advanced partition matching algorithm for partition-wise join

Started by Ashutosh Bapatover 8 years ago154 messages
#1Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
2 attachment(s)

The patch-set in [1]/messages/by-id/CAFjFpRd9Vqh_=-Ldv-XqWY006d07TJ+VXuhXCbdj=P1jukYBrw@mail.gmail.com supports partition-wise join when the partition bounds and
partition keys of the joining tables exactly match. The last two patches in the
last few patch-sets in that thread implement more
advanced partition matching code. In order to avoid mixing reviews for advanced
partition matching and the basic partition-wise join implementation, I am
starting a new thread to discuss the same. I am attaching the last two
patches from that patch set here.

The new partition matching algorithm handles following cases when a
given partition
on one side has at most one matching partition matching on the other side.

1. When the ranges of the joining tables do not match exactly E.g. partition
table t1 has partitions t1p1 (0 - 100), t1p2 (150 - 200) and partition table t2
has partitions t2p1 (0 - 50), t2p2 (100 - 175). In this case (t1p1, t2p1) and
(t1p2, t2p2) form the matching partition pairs, which can be joined. While
matching the pairs, we also compute the partition bounds for the resulting
join. An INNER join between t1 and t2 will have ranges (0 - 50) since no row
with 50 <= key < 100 from t1p1 is going to find a matching row in t2p1 and (150
- 175) since no row with 100 <= key < 150 from t2p2 is going to find a matching
row in t1p2 and no row with 175 <= key < 200 in t1p2 is going to find a
matching row in t1p2. A t1 LEFT join t2 on the other hand will have ranges same
as the outer relation i.e. t1, (0 - 100), (150 - 200) since all rows from t1
will be part of the join. Thus depending upon the type of join the partition
bounds of the resultant join relation change. Similarly for list partitioned
table, when the lists do not match exactly, the algorithm finds matching pairs
of partitions and the lists of resultant join relation. E.g. t1 has
partitions t1p1 ('a',
'b', 'c'), t1p2 ('e', 'f') and t2 has partitions t2p1 ('a', 'b'), t2p2 ('d',
'e', 'f'). In this case (t1p1, t2p1) and (t2p1, t2p2) form the matching
pairs which are joined. Inner join will have bounds ('a','b'), ('e', 'f') and
t1 LEFT JOIN t2 will have bounds same as t1.

2. When one or both side have at least one partition that does not have
matching partition on the other side. E.g. t1 has partitions t1p1 ('a','b'),
t1p2 ('c','d') and t2 has only one partition t2p1 ('a','b') OR t1 has
partitions t1p1 (0 - 100), t1p2 (100 - 200) and t2 has only one partition t2p1
(0 - 100). In this case as well different types of joins will have different
partition bounds for the result using similar rules described above.

3. A combination of 1 and 2 e.g. t1 has partitions t1p1('a','b','c'),
t1p2('d','e','f') and t2 has a single partition t2p1 ('a','b', 'z').

Algorithm
---------
The pairs of matching partitions and the partition bounds of the join are
calculated by an algorithm similar to merge join.

In such a join, it can be observed that every partition on either side,
contributes to at most one partition of the resultant join relation. Thus for
every partition on either side, we keep track of the partition of resultant
join (if any), which it contributes to. If multiple partitions from any of the
joining relations map to a single partition of the resultant join, we need to
gang those partitions together before joining the partition/s from the other
side. Since we do not have infrastructure for ganging multiple arbitrary
RelOptInfos together in a parent RelOptInfo, we do not support such a
partitionw-wise join right now. We stop merging the bounds immediately when we
detect such a case.

For list partitioned tables, we compare list values from both the sides,
starting with the lowest. If the two list values being compared match,
corresponding partitions from both sides form a pair of partitions to be
joined. We record this mapping and also include the list value in join bounds.
If the two list values do not match and the lower of those two comes from the
outer side of the join, we include it in the join bounds. We advance to the
next list value on side with the lower list value continuing the process of
merging till list values on at least one side are exhausted. If the remaining
values are from the outer side, we include those in the join partition bounds.
Every list value included in the join bounds, and its originating partition/s
are associated with appropriate partition of the resultant join. For more
details please see partition_list_bounds_merge() in the attached patch.

In case of range partitioned tables, we compare the ranges of the partitions in
increasing order of their bounds. If two ranges being compared overlap,
corresponding partitions from both sides form a pair of partitions to be
joined. We record this mapping and also include the merged range in the bounds
of resultant join. The overlapping ranges are merged based on the type of join
as described above. If either of the ranges completely precedes the other, and
it's on the outer side, we include that range in the bounds of resultant join.
We advance to the next range on the side with lower upper bound till ranges on
at least one side are exhausted. If the remaining ranges are from the outer
side, we include those in the partition bounds of resultant join. While
including a range in the partition bounds of the resultant join if its lower
bound precedes the upper bound of the last included range, it indicates that
multiple partitions on that side map to one partition on the other side, so we
bail out. Notice that in this method, we always include the ranges in the
partition bounds of the resultant join in the increasing order of their bounds.
Every range included in the join's partition bounds and it's corresponding
partition/s from joining relations are associated with appropriate partition of
the resultant join. For more details please see partition_range_bounds_merge()
in the attached patch.

The partitions from both sides (one partition from each side) which map to the
same partition of the resultant join are joined to form child-joins. The case
when an outer partition may not have a matching partition from the inner side
will be discussed in the next section. Except for the above algorithm to find
the pairs of matching partitions and calculating bounds of the resultant join,
the rest of the partition-wise join algorithm remains the same.

Unsupported case: When a partition from outer side doesn't have matching
partition on the inner side.
--------------------------------------------------------------------------
Consider a join t1 LEFT JOIN t2 where t1 has partitions t1p1 (0 - 100), t1p2
(100 - 200) and t2 has a single partition t2p1(0 - 100). The rows in t1p2 won't
have a matching row in t2 since there is no partition matching t1p2. The result
of the join will have rows in t1p2 with columns from t2 NULLed. In order to
execute this join as a partition-wise join, we need a dummy relation in place
of the missing partition, which we can join with t1p2. We need this placeholder
dummy relation (its targetlist, relids etc.), so that rest of the planner can
work with the resulting child-join.

We notice the missing partitions only while planning the join (during the
execution of make_one_rel()), by which time we have frozen the number of base
relations. Introducing a base relation during join planning is not supported
in current planner. Similarly, a partition can be missing from a partitioned
join relation, in which case we have to add a dummy join relation. This might
need adding corresponding base relations as well. I have not spent time looking
for what it takes to support these cases. For now the patch does not support
partition-wise join in such cases.

TODOs
-----------
1. Add tests for advanced partition matching algorithm
2. Improve code quality, commenting, function names etc.

[1]: /messages/by-id/CAFjFpRd9Vqh_=-Ldv-XqWY006d07TJ+VXuhXCbdj=P1jukYBrw@mail.gmail.com

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

Attachments:

0010-Modify-bound-comparision-functions-to-accept-members.patchtext/x-patch; charset=US-ASCII; name=0010-Modify-bound-comparision-functions-to-accept-members.patchDownload
From 7a9d903deb0475b4f8b0742fbe904a7cf0ce69c1 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Thu, 6 Jul 2017 14:15:22 +0530
Subject: [PATCH 10/11] Modify bound comparision functions to accept members
 of PartitionKey

Functions partition_bound_cmp(), partition_rbound_cmp() and
partition_rbound_datum_cmp() are required to merge partition bounds
from joining relations. While doing so, we do not have access to the
PartitionKey of either relations. So, modify these functions to accept
only required members of PartitionKey so that the functions can be
reused for merging bounds.

Ashutosh Bapat.
---
 src/backend/catalog/partition.c |   76 ++++++++++++++++++++++-----------------
 1 file changed, 44 insertions(+), 32 deletions(-)

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 96a64ce..d42e1b5 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -126,15 +126,17 @@ static List *generate_partition_qual(Relation rel);
 
 static PartitionRangeBound *make_one_range_bound(PartitionKey key, int index,
 					 List *datums, bool lower);
-static int32 partition_rbound_cmp(PartitionKey key,
-					 Datum *datums1, PartitionRangeDatumKind *kind1,
-					 bool lower1, PartitionRangeBound *b2);
-static int32 partition_rbound_datum_cmp(PartitionKey key,
-						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
+static int32 partition_rbound_cmp(int partnatts, FmgrInfo *partsupfunc,
+					 Oid *partcollation, Datum *datums1,
+					 PartitionRangeDatumKind *kind1, bool lower1,
+					 PartitionRangeBound *b2);
+static int32 partition_rbound_datum_cmp(int partnatts, FmgrInfo *partsupfunc,
+						   Oid *partcollation, Datum *rb_datums,
+						   PartitionRangeDatumKind *rb_kind,
 						   Datum *tuple_datums);
 
-static int32 partition_bound_cmp(PartitionKey key,
-					PartitionBoundInfo boundinfo,
+static int32 partition_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+					Oid *partcollation, PartitionBoundInfo boundinfo,
 					int offset, void *probe, bool probe_is_bound);
 static int partition_bound_bsearch(PartitionKey key,
 						PartitionBoundInfo boundinfo,
@@ -719,8 +721,9 @@ check_new_partition_bound(char *relname, Relation parent,
 				 * First check if the resulting range would be empty with
 				 * specified lower and upper bounds
 				 */
-				if (partition_rbound_cmp(key, lower->datums, lower->kind, true,
-										 upper) >= 0)
+				if (partition_rbound_cmp(key->partnatts, key->partsupfunc,
+										 key->partcollation, lower->datums,
+										 lower->kind, true, upper) >= 0)
 				{
 					ereport(ERROR,
 							(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
@@ -771,9 +774,11 @@ check_new_partition_bound(char *relname, Relation parent,
 						{
 							int32		cmpval;
 
-							cmpval = partition_bound_cmp(key, boundinfo,
-														 offset + 1, upper,
-														 true);
+							cmpval = partition_bound_cmp(key->partnatts,
+														 key->partsupfunc,
+														 key->partcollation,
+														 boundinfo, offset + 1,
+														 upper, true);
 							if (cmpval < 0)
 							{
 								/*
@@ -2138,7 +2143,9 @@ qsort_partition_rbound_cmp(const void *a, const void *b, void *arg)
 	PartitionRangeBound *b2 = (*(PartitionRangeBound *const *) b);
 	PartitionKey key = (PartitionKey) arg;
 
-	return partition_rbound_cmp(key, b1->datums, b1->kind, b1->lower, b2);
+	return partition_rbound_cmp(key->partnatts, key->partsupfunc,
+								key->partcollation, b1->datums, b1->kind,
+								b1->lower, b2);
 }
 
 /*
@@ -2155,7 +2162,7 @@ qsort_partition_rbound_cmp(const void *a, const void *b, void *arg)
  * two contiguous partitions.
  */
 static int32
-partition_rbound_cmp(PartitionKey key,
+partition_rbound_cmp(int partnatts, FmgrInfo *partsupfunc, Oid *partcollation,
 					 Datum *datums1, PartitionRangeDatumKind *kind1,
 					 bool lower1, PartitionRangeBound *b2)
 {
@@ -2165,7 +2172,7 @@ partition_rbound_cmp(PartitionKey key,
 	PartitionRangeDatumKind *kind2 = b2->kind;
 	bool		lower2 = b2->lower;
 
-	for (i = 0; i < key->partnatts; i++)
+	for (i = 0; i < partnatts; i++)
 	{
 		/*
 		 * First, handle cases where the column is unbounded, which should not
@@ -2186,8 +2193,8 @@ partition_rbound_cmp(PartitionKey key,
 			 */
 			break;
 
-		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[i],
-												 key->partcollation[i],
+		cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[i],
+												 partcollation[i],
 												 datums1[i],
 												 datums2[i]));
 		if (cmpval != 0)
@@ -2213,22 +2220,23 @@ partition_rbound_cmp(PartitionKey key,
  * is <, =, or > partition key of tuple (tuple_datums)
  */
 static int32
-partition_rbound_datum_cmp(PartitionKey key,
-						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
+partition_rbound_datum_cmp(int partnatts, FmgrInfo *partsupfunc,
+						   Oid *partcollation, Datum *rb_datums,
+						   PartitionRangeDatumKind *rb_kind,
 						   Datum *tuple_datums)
 {
 	int			i;
 	int32		cmpval = -1;
 
-	for (i = 0; i < key->partnatts; i++)
+	for (i = 0; i < partnatts; i++)
 	{
 		if (rb_kind[i] == PARTITION_RANGE_DATUM_MINVALUE)
 			return -1;
 		else if (rb_kind[i] == PARTITION_RANGE_DATUM_MAXVALUE)
 			return 1;
 
-		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[i],
-												 key->partcollation[i],
+		cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[i],
+												 partcollation[i],
 												 rb_datums[i],
 												 tuple_datums[i]));
 		if (cmpval != 0)
@@ -2245,17 +2253,18 @@ partition_rbound_datum_cmp(PartitionKey key,
  * specified in *probe.
  */
 static int32
-partition_bound_cmp(PartitionKey key, PartitionBoundInfo boundinfo,
-					int offset, void *probe, bool probe_is_bound)
+partition_bound_cmp(int partnatts, FmgrInfo *partsupfunc, Oid *partcollation,
+					PartitionBoundInfo boundinfo, int offset, void *probe,
+					bool probe_is_bound)
 {
 	Datum	   *bound_datums = boundinfo->datums[offset];
 	int32		cmpval = -1;
 
-	switch (key->strategy)
+	switch (boundinfo->strategy)
 	{
 		case PARTITION_STRATEGY_LIST:
-			cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
-													 key->partcollation[0],
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
 													 bound_datums[0],
 													 *(Datum *) probe));
 			break;
@@ -2273,12 +2282,14 @@ partition_bound_cmp(PartitionKey key, PartitionBoundInfo boundinfo,
 					 */
 					bool		lower = boundinfo->indexes[offset] < 0;
 
-					cmpval = partition_rbound_cmp(key,
-												  bound_datums, kind, lower,
+					cmpval = partition_rbound_cmp(partnatts, partsupfunc,
+												  partcollation, bound_datums,
+												  kind, lower,
 												  (PartitionRangeBound *) probe);
 				}
 				else
-					cmpval = partition_rbound_datum_cmp(key,
+					cmpval = partition_rbound_datum_cmp(partnatts, partsupfunc,
+														partcollation,
 														bound_datums, kind,
 														(Datum *) probe);
 				break;
@@ -2286,7 +2297,7 @@ partition_bound_cmp(PartitionKey key, PartitionBoundInfo boundinfo,
 
 		default:
 			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) key->strategy);
+				 (int) boundinfo->strategy);
 	}
 
 	return cmpval;
@@ -2320,7 +2331,8 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
 		int32		cmpval;
 
 		mid = (lo + hi + 1) / 2;
-		cmpval = partition_bound_cmp(key, boundinfo, mid, probe,
+		cmpval = partition_bound_cmp(key->partnatts, key->partsupfunc,
+									 key->partcollation, boundinfo, mid, probe,
 									 probe_is_bound);
 		if (cmpval <= 0)
 		{
-- 
1.7.9.5

0011-WIP-Partition-wise-join-for-1-1-1-0-0-1-partition-ma.patchtext/x-patch; charset=US-ASCII; name=0011-WIP-Partition-wise-join-for-1-1-1-0-0-1-partition-ma.patchDownload
From a650bbfa5510aa8db87d36be9def50d265779a3e Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Wed, 9 Aug 2017 12:30:34 +0530
Subject: [PATCH 11/11] WIP Partition-wise join for 1:1, 1:0, 0:1 partition
 matching.

Earlier version of partition-wise join implementation allowed
partition-wise join between two relations with exactly same partition
bounds. This commit allows partition-wise join to be applied under
following conditions

1. the partition bounds of joining relations are such that rows from
given partition on one side join can join with rows from maximum one
partition on the other side i.e. bounds of a given partition on one
side match/overlap with those of maximum one partition on the other
side. If the mapping happens to be m:n where m > 1 or n > 1, we have
to gang multiple partition relations together into a single relation.
This means that we have to add simple relations during join
processing, something which is not supported right now.  ALso, in such
a case, different pairs of joining relations can produce different
partition bounds for the same join relation, which again is not
supported right now.

2. For every partition on outer side that can contribute to the result
of an OUTER side, there exists at least one (taken along with item 1,
it means exactly one)  matching partition on the inner side. To
support partition-wise join when the inner matching partition doesn't
exist, we have to add a dummy base relation corresponding to the
non-existent inner partition. We don't have support add base relations
during join processing.

This commit is not complete yet.

Ashutosh Bapat.
---
 src/backend/catalog/partition.c       | 1231 +++++++++++++++++++++++++++++++++
 src/backend/optimizer/path/joinrels.c |   77 ++-
 src/backend/optimizer/util/relnode.c  |   42 +-
 src/include/catalog/partition.h       |    6 +
 4 files changed, 1325 insertions(+), 31 deletions(-)

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index d42e1b5..eb35fab 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -141,6 +141,38 @@ static int32 partition_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
 static int partition_bound_bsearch(PartitionKey key,
 						PartitionBoundInfo boundinfo,
 						void *probe, bool probe_is_bound, bool *is_equal);
+static PartitionBoundInfo partition_range_bounds_merge(int partnatts,
+							 FmgrInfo *supfuncs, Oid *collations,
+							 PartitionBoundInfo boundinfo1, int nparts1,
+							 PartitionBoundInfo boundinfo2, int nparts2,
+							 JoinType jointype, List **parts1, List **parts2);
+static PartitionBoundInfo partition_list_bounds_merge(int partnatts,
+							FmgrInfo *partsupfunc, Oid *collations,
+							PartitionBoundInfo boundinfo1, int nparts1,
+							PartitionBoundInfo boundinfo2, int nparts2,
+							JoinType jointype, List **parts1, List **parts2);
+static void generate_matching_part_pairs(int *mergemap1, int npart1,
+							 int *mergemap2, int nparts2, JoinType jointype,
+							 int nparts, List **parts1, List **parts2);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index);
+static int map_and_merge_partitions(int *partmap1, int *mergemap1, int index1,
+						 int *partmap2, int *mergemap2, int index2,
+						 int *next_index);
+static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *collations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2);
+static int partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
+						   Oid *collations, PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, bool *overlap);
+static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes);
 
 /*
  * RelationBuildPartitionDesc
@@ -2348,3 +2380,1202 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
 
 	return lo;
 }
+
+/*
+ * Merge the given partition bounds.
+ *
+ * If given partition bounds can not be merged, return NULL.
+ *
+ * The function also returns two lists of partition indexes one for each of the
+ * joining relations. Both the lists contain the same number of elements. The
+ * partition indexes at the same positions in the list indicate partitions from
+ * each side to be joined and their position corresponds to the index of
+ * partition to which the results of the child-join belong in the partitioned
+ * join.
+ */
+extern PartitionBoundInfo
+partition_bounds_merge(int partnatts, FmgrInfo *partsupfunc, Oid *partcollation,
+					   PartitionBoundInfo boundinfo1, int nparts1,
+					   PartitionBoundInfo boundinfo2, int nparts2,
+					   JoinType jointype, List **parts1, List **parts2)
+{
+	PartitionBoundInfo merged_bounds;
+	char		strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (boundinfo1->strategy != boundinfo2->strategy)
+		return NULL;
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+	strategy = boundinfo1->strategy;
+	if (strategy == PARTITION_STRATEGY_LIST)
+		merged_bounds = partition_list_bounds_merge(partnatts, partsupfunc,
+													partcollation, boundinfo1,
+													nparts1, boundinfo2,
+													nparts2, jointype, parts1,
+													parts2);
+	else if (strategy == PARTITION_STRATEGY_RANGE)
+		merged_bounds = partition_range_bounds_merge(partnatts, partsupfunc,
+													 partcollation, boundinfo1,
+													 nparts1, boundinfo2,
+													 nparts2, jointype, parts1,
+													 parts2);
+	else
+		elog(ERROR, "unexpected partition strategy: %d", strategy);
+
+	Assert(merged_bounds || (*parts1 == NIL && *parts2 == NIL));
+	return merged_bounds;
+}
+
+/*
+ * partition_get_range_bounds
+ *
+ * Given the index of lower bound in datums array, return lower and upper
+ * bounds and the index of the partition with that lower bound.
+ */
+static int
+partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
+						   PartitionRangeBound *lower,
+						   PartitionRangeBound *upper)
+{
+	int			part_index;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The lower bound should correspond to a valid partition. */
+	part_index = bi->indexes[lb_index + 1];
+	Assert(part_index >= 0);
+
+	lower->kind = bi->kind[lb_index];
+	lower->datums = bi->datums[lb_index];
+	lower->lower = true;
+	upper->kind = bi->kind[lb_index + 1];
+	upper->datums = bi->datums[lb_index + 1];
+	upper->lower = false;
+
+	return part_index;
+}
+
+/*
+ * partition_range_get_next_lb_index
+ *
+ * Given the index of lower bound in datums array return the
+ * index of lower bound of the next partition. When the given index corresponds
+ * to the last partition, return -1.
+ */
+static int
+partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
+{
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The lower bound should correspond to a valid partition. */
+	Assert(bi->indexes[lb_index + 1] >= 0);
+
+	/*
+	 * If there are no bounds left beyond the upper bound, we have reached the
+	 * last partition.
+	 */
+	if (lb_index + 2 < bi->ndatums)
+	{
+		/*
+		 * If the bound next to the upper bound corresponds to no partition,
+		 * that's the next lower bound of the next partition. Otherwise, the
+		 * current upper bound is the lower bound of the next partition.
+		 */
+		if (bi->indexes[lb_index + 2] < 0)
+			return lb_index + 2;
+		else
+			return lb_index + 1;
+	}
+	else
+		return -1;
+}
+
+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc, Oid *collations,
+						  PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2)
+{
+	return partition_rbound_cmp(partnatts, partsupfunc, collations,
+								bound1->datums, bound1->kind, bound1->lower,
+								bound2);
+}
+
+/*
+ * partition_range_cmp
+ *
+ * Compare the bounds of two range partitions and return <, = or > 0, if the
+ * first partition's upper bound is lower than, equal to or higher than the
+ * second partition's upper bound resp.
+ *
+ * Also, set overlaps to true, if the ranges overlap, otherwise set it to
+ * false.
+ */
+static int
+partition_range_cmp(int partnatts, FmgrInfo *supfuncs, Oid *collations,
+						   PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, bool *overlap)
+{
+	/*
+	 * Compare upper bound of the first partition with the lower bound of the
+	 * second and vice-versa. If lower bound is higher than the upper bound,
+	 * the partitions are not overlapping. All other cases indicate overlapping
+	 * partitions.
+	 * TODO: Add a testcase which has lower and upper bound matching exactly.
+	 * Lower bound is inclusive and upper bound is exclusive, so even if the
+	 * datums match, the bounds do not match exactly.
+	 */
+	if (partition_range_bound_cmp(partnatts, supfuncs, collations,
+								  lower_bound1, upper_bound2) > 0)
+	{
+		*overlap = false;
+		return 1;
+	}
+	else if (partition_range_bound_cmp(partnatts, supfuncs, collations,
+									   lower_bound2, upper_bound1) > 0)
+	{
+		*overlap = false;
+		return -1;
+	}
+	else
+	{
+		*overlap = true;
+		return partition_range_bound_cmp(partnatts, supfuncs, collations,
+										 upper_bound1, upper_bound2);
+	}
+}
+
+/*
+ * partition_range_merge
+ *
+ * Merge the partition bounds of given two partitions such that the join
+ * between the given two partitions fits merged bounds.
+ *
+ * "merged_upper" will be set to one of the given upper bounds and
+ * "merged_lower" will be set to one of the given lower bounds.
+ */
+static void
+partition_range_merge(int partnatts, FmgrInfo *supfuncs,
+							 Oid *collations, JoinType jointype,
+							 PartitionRangeBound *left_lb,
+							 PartitionRangeBound *left_ub,
+							 PartitionRangeBound *right_lb,
+							 PartitionRangeBound *right_ub,
+							 PartitionRangeBound **merged_lb,
+							 PartitionRangeBound **merged_ub)
+{
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = left_ub;
+			*merged_lb = left_lb;
+			break;
+
+		case JOIN_RIGHT:
+			*merged_ub = right_ub;
+			*merged_lb = right_lb;
+			break;
+
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			if (partition_range_bound_cmp(partnatts, supfuncs, collations,
+										  left_ub, right_ub) < 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, supfuncs, collations,
+										  left_lb, right_lb) > 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		case JOIN_FULL:
+			if (partition_range_bound_cmp(partnatts, supfuncs, collations,
+										  left_ub, right_ub) > 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, supfuncs, collations,
+										  left_lb, right_lb) < 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		default:
+			elog(ERROR, "Unexpected join type %d", jointype);
+	}
+
+	return;
+}
+
+/*
+ * Add the lower bound of the next range to the list of bounds, if the lower
+ * bound is higher or equal to the previous upper bound. If successful return
+ * true, otherwise false.
+ */
+static bool
+partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		prev_ub.datums = llast(*merged_datums);
+		prev_ub.kind = llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		/*
+		 * TODO: explain why do we pass lower to be false for the next lower
+		 * bound.
+		 */
+		cmpval = partition_rbound_cmp(partnatts, supfuncs, collations,
+									  next_lb_datums, next_lb_kind, false,
+									  &prev_ub);
+	}
+
+	/*
+	 * The lower bound is lower than the last upper bound, thus does not fit
+	 * the bounds created so far and hence can not be merged with the existing
+	 * bounds.
+	 */
+	if (cmpval < 0)
+		return false;
+
+	/*
+	 * Add bounds of the new merged partition. If the next lower bound is
+	 * higher than the last upper bound, add new range with index
+	 * corresponding to the lower bound as -1. If the merged lower bound
+	 * is same as the last merged upper bound, the last upper bound will be
+	 * reused as the lower bound of the next range.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, next_lb_datums);
+		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	return true;
+}
+
+/*
+ * Merge given two range partition bounds.
+ *
+ * Work horse function for partition_bounds_merge() for range partitioned
+ * tables.
+ *
+ * TODO: for an anti-join, the caller is supposed to send the outer relation as
+ * left relation. May be we should rename left and right as inner and outer. We
+ * don't need to handle RIGHT joins in this function, so renaming them as outer
+ * and inner is fine.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(int partnatts, FmgrInfo *supfuncs, Oid *collations,
+							 PartitionBoundInfo left_bi, int left_nparts,
+							 PartitionBoundInfo right_bi, int right_nparts,
+							 JoinType jointype, List **left_parts, List **right_parts)
+{
+	int		   *left_pmap;
+	int		   *left_mmap;
+	int		   *right_pmap;
+	int		   *right_mmap;
+	int			cnt1;
+	int			cnt2;
+	int			left_part;
+	int			right_part;
+	PartitionBoundInfo merged_bounds = NULL;
+	bool		merged = true;	/* By default we ranges are merge-able. */
+	int			left_lb_index;
+	int			right_lb_index;
+	int			next_index;
+	int			cmpval;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	List	   *merged_kinds = NIL;
+
+	Assert(left_bi->strategy == right_bi->strategy &&
+		   left_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	*left_parts = NIL;
+	*right_parts = NIL;
+
+	/* Allocate and initialize partition maps. */
+	left_pmap = (int *) palloc(sizeof(int) * left_nparts);
+	left_mmap = (int *) palloc(sizeof(int) * left_nparts);
+	right_pmap = (int *) palloc(sizeof(int) * right_nparts);
+	right_mmap = (int *) palloc(sizeof(int) * right_nparts);
+
+	for (cnt1 = 0; cnt1 < left_nparts; cnt1++)
+	{
+		left_pmap[cnt1] = -1;
+		left_mmap[cnt1] = -1;
+	}
+	for (cnt2 = 0; cnt2 < right_nparts; cnt2++)
+	{
+		right_pmap[cnt2] = -1;
+		right_mmap[cnt2] = -1;
+	}
+
+	left_lb_index = 0;
+	right_lb_index = 0;
+	next_index = 0;
+	while (left_lb_index >= 0 && right_lb_index >= 0)
+	{
+		PartitionRangeBound left_lb;
+		PartitionRangeBound left_ub;
+		PartitionRangeBound right_lb;
+		PartitionRangeBound right_ub;
+		PartitionRangeBound *merged_lb = NULL;
+		PartitionRangeBound *merged_ub = NULL;
+		int			merged_index = -1;
+		bool		overlap;
+
+		/* Get the range bounds of the next partition. */
+		left_part = partition_get_range_bounds(left_bi, left_lb_index,
+											   &left_lb, &left_ub);
+		right_part = partition_get_range_bounds(right_bi, right_lb_index,
+												&right_lb, &right_ub);
+
+		cmpval = partition_range_cmp(partnatts, supfuncs, collations,
+									 &left_lb, &left_ub, &right_lb, &right_ub,
+									 &overlap);
+
+		if (overlap)
+		{
+			/* Overlapping ranges, try merging. */
+			partition_range_merge(partnatts, supfuncs, collations, jointype,
+								  &left_lb, &left_ub, &right_lb, &right_ub,
+								  &merged_lb, &merged_ub);
+			merged_index = map_and_merge_partitions(left_pmap, left_mmap,
+													left_part, right_pmap,
+													right_mmap, right_part,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				merged = false;
+				break;
+			}
+		}
+
+		if (cmpval == 0)
+		{
+			Assert(overlap);
+
+			/* Move to the next pair of partitions. */
+			left_lb_index = partition_range_get_next_lb_index(left_bi,
+															  left_lb_index);
+			right_lb_index = partition_range_get_next_lb_index(right_bi,
+															   right_lb_index);
+		}
+		else if (cmpval < 0)
+		{
+			/*
+			 * If the partition on the left was not mapped to any partition on
+			 * the right. Any row from that partition will not have a matching
+			 * row from the other relation. So the partition will be part of
+			 * the join if it's an anti-join or the left side is the outer
+			 * side.
+			 */
+			if (jointype == JOIN_INNER || jointype == JOIN_SEMI ||
+				jointype == JOIN_RIGHT)
+			{
+				/* Nothing to do. */
+			}
+			else if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+					 jointype == JOIN_ANTI)
+			{
+				if (left_mmap[left_part] < 0)
+				{
+					left_mmap[left_part] = next_index++;
+					merged_index = left_mmap[left_part];
+					merged_lb = &left_lb;
+					merged_ub = &left_ub;
+				}
+			}
+			else
+			{
+				/* Don't know what to do with other join types. Bail out. */
+				merged = false;
+			}
+
+			/* Move to the next partition on the left side. */
+			left_lb_index = partition_range_get_next_lb_index(left_bi,
+															  left_lb_index);
+		}
+		else
+		{
+			Assert(cmpval > 0);
+
+			/*
+			 * If the partition on the right was not mapped to any partition on
+			 * the left. Any row from that partition will not have a matching
+			 * row from the other relation. So the partition will be part of
+			 * the join if the right side is the outer side.
+			 */
+			if (jointype == JOIN_INNER || jointype == JOIN_SEMI ||
+				jointype == JOIN_LEFT || jointype == JOIN_ANTI)
+				merged_index = -1;
+			else if (jointype == JOIN_RIGHT || jointype == JOIN_FULL)
+			{
+				if (right_mmap[right_part] < 0)
+				{
+					right_mmap[right_part] = next_index++;
+					merged_index = right_mmap[right_part];
+					merged_lb = &right_lb;
+					merged_ub = &right_ub;
+				}
+			}
+			else
+			{
+				/* Don't know what to do with other join types. Bail out. */
+				merged = false;
+			}
+
+			/* Move to the next partition on the right side. */
+			right_lb_index = partition_range_get_next_lb_index(right_bi,
+															   right_lb_index);
+		}
+
+		if (!merged)
+			break;
+
+		/* A skipped partition is not added to merged bounds. */
+		if (merged_index < 0)
+			continue;
+
+		/*
+		 * We have a valid partition index for the next partition of join. The
+		 * partition should have valid range.
+		 */
+		Assert(merged_lb && merged_ub);
+
+		/* Try merging merged lower bound with the last upper bound. */
+		merged = partition_range_merge_next_lb(partnatts, supfuncs,
+											   collations, merged_lb->datums,
+											   merged_lb->kind, &merged_datums,
+											   &merged_kinds, &merged_indexes);
+		if (merged)
+		{
+			/* Add upper bound with the merged partition index. */
+			merged_datums = lappend(merged_datums, merged_ub->datums);
+			merged_kinds = lappend(merged_kinds, merged_ub->kind);
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+		}
+		else
+			break;
+	}
+
+	/*
+	 * We will run the above loop till we exhaust ranges of at least one side
+	 * unless we failed to merge the ranges.
+	 */
+	Assert (!merged || (left_lb_index < 0 || right_lb_index < 0));
+
+	/*
+	 * Handle any remaining partition bounds.  If remaining partitions fall on
+	 * the inner side of the join, none of the rows in those partition are
+	 * going to be joined with any row on the outer side and hence those
+	 * partitions will not be part of the join result. Hence only consider the
+	 * remaining partitions on the outer side of the join.
+	 */
+	if (merged &&
+		((left_lb_index >= 0 &&
+		  (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+		   jointype == JOIN_LEFT)) ||
+		 (right_lb_index >= 0 &&
+		  (jointype == JOIN_RIGHT || jointype == JOIN_FULL))))
+	{
+		int			bound_index = -1;
+		PartitionBoundInfo rem_bi = NULL;
+		int			*mmap = NULL;
+
+		if (left_lb_index >= 0)
+		{
+			Assert(jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+				   jointype == JOIN_ANTI);
+			bound_index = left_lb_index;
+			rem_bi = left_bi;
+			mmap = left_mmap;
+		}
+		else if (right_lb_index >= 0)
+		{
+			Assert(jointype == JOIN_RIGHT || jointype == JOIN_FULL);
+			bound_index = right_lb_index;
+			rem_bi = right_bi;
+			mmap = right_mmap;
+		}
+
+		Assert(bound_index >= 0 && rem_bi && mmap);
+
+		/*
+		 * Merge lower bound of the next range with the upper bound of last
+		 * range.
+		 */
+		merged = partition_range_merge_next_lb(partnatts, supfuncs, collations,
+											   rem_bi->datums[bound_index],
+											   rem_bi->kind[bound_index],
+											   &merged_datums, &merged_kinds,
+											   &merged_indexes);
+
+		/*
+		 * Rest of the bounds correspond to valid ranges so add them after
+		 * remapping their partitions as required.
+		 */
+		for (bound_index++; merged && bound_index < rem_bi->ndatums;
+			 bound_index++)
+		{
+			Datum	   *datums = rem_bi->datums[bound_index];
+			int			index = rem_bi->indexes[bound_index];
+			int			part_index;
+
+			if (index < 0)
+				part_index = index;
+			else
+			{
+				if (mmap[index] < 0)
+					mmap[index] = next_index++;
+				part_index = mmap[index];
+			}
+
+			merged_indexes = lappend_int(merged_indexes, part_index);
+			merged_datums = lappend(merged_datums, datums);
+			merged_kinds = lappend(merged_kinds,
+								   rem_bi->kind[bound_index]);
+		}
+	}
+
+	/* Create PartitionBoundInfo */
+	if (merged)
+	{
+		/* Use maps to match partition from the joining relations. */
+		generate_matching_part_pairs(left_mmap, left_nparts, right_mmap,
+									 right_nparts, jointype, next_index,
+									 left_parts, right_parts);
+
+		/* Craft a PartitionBoundInfo to return. */
+		if (*left_parts && *right_parts)
+		{
+			Assert(list_length(*left_parts) == list_length(*right_parts));
+			Assert(list_length(*left_parts) == next_index);
+			merged_bounds = build_merged_partition_bounds(left_bi->strategy,
+														  merged_datums,
+														  merged_indexes,
+														  merged_kinds,
+														  -1);
+		}
+	}
+
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	pfree(left_pmap);
+	pfree(right_pmap);
+	pfree(left_mmap);
+	pfree(right_mmap);
+
+	/* Free any memory we used in this function. */
+	return merged_bounds;
+}
+
+/*
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ * The function builds the maps of matching partitions from either relation. It
+ * builds the list of partition key values that may appear in the join result
+ * alongwith the list of indexes of partitions of join to which those values
+ * belong. It then crafts a PartitionBoundInfo structure representing the
+ * partition bounds of the join result.
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(int partnatts, FmgrInfo *partsupfunc,
+							Oid *partcollation, PartitionBoundInfo left_bi,
+							int left_nparts, PartitionBoundInfo right_bi,
+							int right_nparts, JoinType jointype, List **left_parts,
+							List **right_parts)
+{
+	int		   *left_pmap;	/* left to right partition map */
+	int		   *left_mmap;	/* left to merged partition map */
+	int		   *right_pmap;	/* right to left partition map */
+	int		   *right_mmap;	/* right to merged partition map */
+	int			cntl;
+	int			cntr;
+	bool		merged = true;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			next_index = 0;
+	int			null_index;
+	PartitionBoundInfo merged_bounds = NULL;
+	int		   *left_indexes = left_bi->indexes;
+	int		   *right_indexes = right_bi->indexes;
+	int			left_ni = left_bi->null_index;
+	int			right_ni = right_bi->null_index;
+
+	Assert(left_bi->strategy == right_bi->strategy &&
+		   left_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!left_bi->kind && !right_bi->kind);
+
+	/* Allocate and initialize partition maps. */
+	left_pmap = (int *) palloc(sizeof(int) * left_nparts);
+	left_mmap = (int *) palloc(sizeof(int) * left_nparts);
+	right_pmap = (int *) palloc(sizeof(int) * right_nparts);
+	right_mmap = (int *) palloc(sizeof(int) * right_nparts);
+
+	/* Initialize partition maps. */
+	for (cntl = 0; cntl < left_nparts; cntl++)
+	{
+		left_pmap[cntl] = -1;
+		left_mmap[cntl] = -1;
+	}
+	for (cntr = 0; cntr < right_nparts; cntr++)
+	{
+		right_pmap[cntr] = -1;
+		right_mmap[cntr] = -1;
+	}
+
+	cntl = cntr = 0;
+	while (cntl < left_bi->ndatums && cntr < right_bi->ndatums)
+	{
+		Datum	   *ldatums = left_bi->datums[cntl];
+		Datum	   *rdatums = right_bi->datums[cntr];
+		int			l_index = left_indexes[cntl];
+		int			r_index = right_indexes[cntr];
+		int			cmpval;
+		int			merged_index;
+		Datum	   *merged_datum;
+
+		/* Every list datum should map to a valid partition index. */
+		Assert(l_index >= 0 && r_index >= 0);
+
+		cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+												 partcollation[0], ldatums[0],
+												 rdatums[0]));
+		if (cmpval == 0)
+		{
+			/*
+			 * Try matching partitions containing the matching datums. If
+			 * successful, add the datum to the merged bounds with index of
+			 * merged partition containing it.
+			 */
+			merged_datum = ldatums;
+			merged_index = map_and_merge_partitions(left_pmap, left_mmap, l_index,
+													right_pmap, right_mmap, r_index,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				merged = false;
+				break;
+			}
+
+			/* Move to the next pair of bounds. */
+			cntl++;
+			cntr++;
+		}
+		else if (cmpval < 0)
+		{
+			/*
+			 * This list datum is present in the left side but not the right
+			 * side. So it will appear in the join when the left side is outer
+			 * side.
+			 */
+			if (jointype == JOIN_INNER || jointype == JOIN_RIGHT ||
+				jointype == JOIN_SEMI)
+				merged_index = -1;
+			else if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+					 jointype == JOIN_ANTI)
+			{
+				if (left_mmap[l_index] < 0)
+					left_mmap[l_index] = next_index++;
+				merged_index = left_mmap[l_index];
+				merged_datum = ldatums;
+			}
+			else
+			{
+				/* Don't know what to do with other join types. */
+				merged = false;
+				break;
+			}
+
+			/* Move to the next datum on the left side. */
+			cntl++;
+		}
+		else
+		{
+			Assert(cmpval > 0);
+
+			/*
+			 * This list datum is present in the right side but not the left
+			 * side. So it will appear in the join when the right side is outer
+			 * side.
+			 */
+			if (jointype == JOIN_INNER || jointype == JOIN_LEFT ||
+				jointype == JOIN_SEMI || jointype == JOIN_ANTI)
+				merged_index = -1;
+			else if (jointype == JOIN_RIGHT || jointype == JOIN_FULL)
+			{
+				/*
+				 * Every list value on the outer side will appear in the
+				 * join.  Find the merged partition to which this value
+				 * belongs.
+				 */
+				if (right_mmap[r_index] < 0)
+					right_mmap[r_index] = next_index++;
+				merged_index = right_mmap[r_index];
+				merged_datum = rdatums;
+			}
+			else
+			{
+				/* Don't know what to do with other join types. */
+				merged = false;
+				break;
+			}
+
+			/* Move to the next datum on the right side. */
+			cntr++;
+		}
+
+		/*
+		 * Add the datum with appropriate index in the list of datums, if the
+		 * rows containing that datum are deemed to be part of the join.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	/*
+	 * If merge is unsuccessful, bail out without any further processing.
+	 * That leaks the memory allocated in this function. So, try not to leak
+	 * memory.
+	 */
+	if (!merged)
+		goto merge_failed;
+
+	/* We should have exhausted datums on at least one side. */
+	Assert(cntr >= right_bi->ndatums || cntl >= left_bi->ndatums);
+
+	/*
+	 * Add any remaining list values on the outer side, assigning partition
+	 * indexes if required.
+	 */
+	if (jointype == JOIN_LEFT || jointype == JOIN_FULL || jointype == JOIN_ANTI)
+	{
+		for (;cntl < left_bi->ndatums; cntl++)
+		{
+			Datum	   *ldatums = left_bi->datums[cntl];
+			int			l_index = left_indexes[cntl];
+
+			if (left_mmap[l_index] < 0)
+				left_mmap[l_index] = next_index++;
+			merged_indexes = lappend_int(merged_indexes, left_mmap[l_index]);
+			merged_datums = lappend(merged_datums, ldatums);
+		}
+	}
+
+	if (jointype == JOIN_RIGHT || jointype == JOIN_FULL)
+	{
+		for (;cntr < right_bi->ndatums; cntr++)
+		{
+			Datum	   *rdatums = right_bi->datums[cntr];
+			int			r_index = right_indexes[cntr];
+
+			if (right_mmap[r_index] < 0)
+				right_mmap[r_index] = next_index++;
+			merged_indexes = lappend_int(merged_indexes, right_mmap[r_index]);
+			merged_datums = lappend(merged_datums, rdatums);
+		}
+	}
+
+	/*
+	 * Merge NULL partitions if any. Find the index of merged partition to
+	 * which the NULL values belong in the join result. We can eliminate a NULL
+	 * partition when it appears only in the inner relation.
+	 */
+	if (!partition_bound_accepts_nulls(left_bi) &&
+		!partition_bound_accepts_nulls(right_bi))
+		null_index = -1;
+	else if (partition_bound_accepts_nulls(left_bi) &&
+			 !partition_bound_accepts_nulls(right_bi))
+	{
+		if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+			jointype == JOIN_ANTI)
+		{
+			if (left_mmap[left_ni] < 0)
+				left_mmap[left_ni] = next_index++;
+			null_index = left_mmap[left_ni];
+		}
+		else
+			null_index = -1;
+	}
+	else if (!partition_bound_accepts_nulls(left_bi) &&
+			 partition_bound_accepts_nulls(right_bi))
+	{
+		if (jointype == JOIN_RIGHT || jointype == JOIN_FULL)
+		{
+			if (right_mmap[right_ni] < 0)
+				right_mmap[right_ni] = next_index++;
+			null_index = right_mmap[right_ni];
+		}
+		else
+			null_index = -1;
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		null_index = map_and_merge_partitions(left_pmap, left_mmap,
+											  left_ni, right_pmap,
+											  right_mmap, right_ni,
+											  &next_index);
+		if (null_index < 0)
+			merged = false;
+	}
+
+	/* If successful build the output structures. */
+	if (merged)
+	{
+
+		/* Use maps to match partition from the joining relations. */
+		generate_matching_part_pairs(left_mmap, left_nparts, right_mmap,
+									 right_nparts, jointype, next_index,
+									 left_parts, right_parts);
+		/* Craft a PartitionBoundInfo to return. */
+		if (*left_parts && *right_parts)
+		{
+			Assert(list_length(*left_parts) == list_length(*right_parts));
+			Assert(list_length(*left_parts) == next_index);
+			merged_bounds = build_merged_partition_bounds(left_bi->strategy,
+														  merged_datums,
+														  merged_indexes, NIL,
+														  null_index);
+		}
+	}
+
+merge_failed:
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	pfree(left_pmap);
+	pfree(right_pmap);
+	pfree(left_mmap);
+	pfree(right_mmap);
+
+	return merged_bounds;
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * If the two given partitions (given by index1 and index2 resp.) are already
+ * mapped to each other return the index of corresponding partition in the
+ * merged set of partitions.  If they do not have a merged partition associated
+ * with them, assign a new merged partition index.  If the partitions are
+ * already mapped and their mapped partitions are different from each other,
+ * they can not be merged, so return -1.
+ *
+ * partmap1[i] gives the partition of relation 2 which matches ith partition of
+ * relation 1. Similarly for partmap2.
+ *
+ * mergemap1[i] gives the partition in the merged set to which ith partition of
+ * relation 1 maps to. Similarly for mergemap2.
+ *
+ * index1 and index2 are the indexes of matching partition from respective
+ * relations.
+ *
+ * *next_index is used and incremented when the given partitions require a new
+ * merged partition.
+ */
+static int
+map_and_merge_partitions(int *partmap1, int *mergemap1, int index1,
+						 int *partmap2, int *mergemap2, int index2,
+						 int *next_index)
+{
+	int			merged_index;
+
+	/*
+	 * If both the partitions are not mapped to each other, update the
+	 * maps.
+	 */
+	if (partmap1[index1] < 0 && partmap2[index2] < 0)
+	{
+		partmap1[index1] = index2;
+		partmap2[index2] = index1;
+	}
+
+	/*
+	 * If the given to partitions map to each other, find the corresponding
+	 * merged partition index .
+	 */
+	if (partmap1[index1] == index2 && partmap2[index2] == index1)
+	{
+		/*
+		 * If both the partitions are mapped to the same merged partition, get
+		 * the index of merged partition.
+		 */
+		if (mergemap1[index1] == mergemap2[index2])
+		{
+			merged_index = mergemap1[index1];
+
+			/*
+			 * If the given two partitions do not have a merged partition
+			 * associated with them, allocate a new merged partition.
+			 */
+			if (merged_index < 0)
+			{
+				merged_index = *next_index;
+				*next_index = *next_index + 1;
+				mergemap1[index1] = merged_index;
+				mergemap2[index2] = merged_index;
+			}
+		}
+
+		/*
+		 * If partition from one relation was mapped to a merged partition but
+		 * not the partition from the other relation, map the same merged
+		 * partition to the partition from other relation, since matching
+		 * partitions map to the same merged partition.
+		 */
+		else if (mergemap1[index1] >= 0 && mergemap2[index2] < 0)
+		{
+			mergemap2[index2] = mergemap1[index1];
+			merged_index = mergemap1[index1];
+		}
+		else if (mergemap1[index1] < 0 && mergemap2[index2] >= 0)
+		{
+			mergemap1[index1] = mergemap2[index2];
+			merged_index = mergemap2[index2];
+		}
+		else
+		{
+			Assert(mergemap1[index1] != mergemap2[index2] &&
+				   mergemap1[index1] >= 0 && mergemap2[index2] >= 0);
+
+			/*
+			 * Both the partitions map to different merged partitions. This
+			 * means that multiple partitions from one relation matches to one
+			 * partition from the other relation. Partition-wise join does not
+			 * handle this case right now, since it requires ganging multiple
+			 * partitions together (into one RelOptInfo).
+			 */
+			merged_index = -1;
+		}
+	}
+	else
+	{
+		/*
+		 * Multiple partitions from one relation map to one partition from the
+		 * other relation. Partition-wise join does not handle this case right
+		 * now, since it requires ganging multiple partitions together (into
+		 * one RelOptInfo).
+		 */
+		merged_index = -1;
+	}
+
+	return merged_index;
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * Given the merged partition to which partition on either side of join map,
+ * produce the list pairs of partitions which when joined produce the merged
+ * partitions in the order of merged partition indexes.
+ *
+ * If successful, the pairs of partitions are returned as two separate lists
+ * one for each side. Otherwise, those lists will be set to NIL.
+ *
+ * TODO: rename the sides as outer and inner. You may not need to support
+ * JOIN_RIGHT, since we won't see that type here.
+ */
+static void
+generate_matching_part_pairs(int *mergemap1, int nparts1, int *mergemap2,
+							 int nparts2, JoinType jointype, int nparts,
+							 List **parts1, List **parts2)
+{
+	bool		merged = true;
+	int		  **matching_parts;
+	int			cnt1;
+	int			cnt2;
+
+	matching_parts = (int **) palloc(sizeof(int *) * 2);
+	matching_parts[0] = (int *) palloc(sizeof(int) * nparts);
+	matching_parts[1] = (int *) palloc(sizeof(int) * nparts);
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt1 = 0; cnt1 < nparts; cnt1++)
+	{
+		matching_parts[0][cnt1] = -1;
+		matching_parts[1][cnt1] = -1;
+	}
+
+	/* Set pairs of matching partitions. */
+	for (cnt1 = 0; cnt1 < nparts1; cnt1++)
+	{
+		if (mergemap1[cnt1] >= 0)
+		{
+			Assert(mergemap1[cnt1] < nparts);
+			matching_parts[0][mergemap1[cnt1]] = cnt1;
+		}
+	}
+	for (cnt2 = 0; cnt2 < nparts2; cnt2++)
+	{
+		if (mergemap2[cnt2] >= 0)
+		{
+			Assert(mergemap2[cnt2] < nparts);
+			matching_parts[1][mergemap2[cnt2]] = cnt2;
+		}
+	}
+
+	/*
+	 * If we have a partition missing on an inner side, we need to add a dummy
+	 * relation which joins with the outer partition. If the inner relation
+	 * happens to be a base relation, it will require adding a dummy child
+	 * base relation during join processing. Right now, we freeze the base
+	 * relation arrays like PlannerInfo::simple_rte_array after planning for
+	 * base relations. Adding a new (dummy) base relation would require some
+	 * changes to that. So, right now, we do not implement partition-wise join
+	 * in such cases.
+	 */
+	for (cnt1 = 0; cnt1 < nparts; cnt1++)
+	{
+		int			part1 = matching_parts[0][cnt1];
+		int			part2 = matching_parts[1][cnt1];
+
+		/* At least one of the partitions should exist. */
+		Assert(part1 >= 0 || part2 >= 0);
+
+		switch (jointype)
+		{
+			case JOIN_INNER:
+			case JOIN_SEMI:
+
+				/*
+				 * An inner or semi join can not return any row when the
+				 * matching partition on either side is missing. We should
+				 * have eliminated all such cases while merging the bounds.
+				 */
+				Assert(part1 >= 0 && part2 >= 0);
+				break;
+
+			case JOIN_LEFT:
+			case JOIN_ANTI:
+				Assert(part1 >= 0);
+				if (part2 < 0)
+					merged = false;
+				break;
+
+			case JOIN_RIGHT:
+				Assert(part2 >= 0);
+				if (part1 < 0)
+					merged = false;
+				break;
+
+			case JOIN_FULL:
+				if (part1 < 0 || part2 < 0)
+					merged = false;
+				break;
+
+			default:
+				/* We do not know what to do in this case. Bail out. */
+				merged = false;
+		}
+
+		if (!merged)
+			break;
+
+		*parts1 = lappend_int(*parts1, part1);
+		*parts2 = lappend_int(*parts2, part2);
+	}
+
+	pfree(matching_parts[0]);
+	pfree(matching_parts[1]);
+	pfree(matching_parts);
+
+	if (!merged)
+	{
+		list_free(*parts1);
+		list_free(*parts2);
+		*parts1 = NIL;
+		*parts2 = NIL;
+	}
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind = (PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+															   list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+
+	return merged_bounds;
+}
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index a97a895..fb8d752 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1308,8 +1308,13 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 						RelOptInfo *joinrel, SpecialJoinInfo *parent_sjinfo,
 						List *parent_restrictlist)
 {
-	int			nparts;
 	int			cnt_parts;
+	PartitionScheme part_scheme;
+	PartitionBoundInfo join_boundinfo;
+	List	   *parts1;
+	List	   *parts2;
+	ListCell   *lc1;
+	ListCell   *lc2;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
@@ -1353,39 +1358,54 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	 */
 	Assert(joinrel->part_scheme == rel1->part_scheme &&
 		   joinrel->part_scheme == rel2->part_scheme);
+	part_scheme = joinrel->part_scheme;
 
 	/*
-	 * Since we allow partition-wise join only when the partition bounds of
-	 * the joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * Get the list of matching partitions from both sides of the join. While
+	 * doing so, we also build the partition bounds of the join relation,
+	 * which should match the bounds calculated for other pairs. TODO: why
+	 * should every pair result in the same partition bounds?
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
-
-	nparts = joinrel->nparts;
-
+	join_boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											part_scheme->partsupfunc,
+											part_scheme->parttypcoll,
+											rel1->boundinfo, rel1->nparts,
+											rel2->boundinfo, rel2->nparts,
+											parent_sjinfo->jointype,
+											&parts1, &parts2);
+
+	Assert(join_boundinfo);
+	Assert(partition_bounds_equal(part_scheme->partnatts,
+								  part_scheme->parttyplen,
+								  part_scheme->parttypbyval, join_boundinfo,
+								  joinrel->boundinfo));
 	elog(DEBUG3, "join between relations %s and %s is considered for partition-wise join.",
 		 bmsToString(rel1->relids), bmsToString(rel2->relids));
 
-	/* Allocate space to hold child-joins RelOptInfos, if not already done. */
+	/*
+	 * Every pair of joining relations should result in the same number of
+	 * child-joins.
+	 */
+	Assert(joinrel->nparts == list_length(parts1));
+	Assert(joinrel->nparts == list_length(parts2));
+
+	/* Allocate space for hold child-joins RelOptInfos, if not already done. */
 	if (!joinrel->part_rels)
-		joinrel->part_rels = (RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+		joinrel->part_rels = (RelOptInfo **) palloc0(sizeof(RelOptInfo *) *
+													 joinrel->nparts);
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	cnt_parts = 0;
+	forboth(lc1, parts1, lc2, parts2)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
+		int			part1 = lfirst_int(lc1);
+		int			part2 = lfirst_int(lc2);
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1393,6 +1413,10 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		Assert(part1 >= 0 && part2 >= 0);
+		child_rel1 = rel1->part_rels[part1];
+		child_rel2 = rel2->part_rels[part2];
+
 		/* We should never try to join two overlapping sets of rels. */
 		Assert(!bms_overlap(child_rel1->relids, child_rel2->relids));
 		child_joinrelids = bms_union(child_rel1->relids, child_rel2->relids);
@@ -1425,6 +1449,15 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 			joinrel->part_rels[cnt_parts] = child_joinrel;
 		}
 
+		/*
+		 * For every pair of joining relations, the set of matching partitions
+		 * would change. However, the base relation partitions constituting
+		 * the given child should remain same for all the joining pairs. Since
+		 * the order in which children are stored in the array of child-joins,
+		 * depends upon partition bounds of the join, which are same for all
+		 * the joining pairs, every joining pair yields the child-joins in the
+		 * same order.
+		 */
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
 
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
@@ -1437,7 +1470,11 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		 */
 		try_partition_wise_join(root, child_rel1, child_rel2, child_joinrel,
 								child_sjinfo, child_restrictlist);
+
+		cnt_parts++;
 	}
+
+	Assert(cnt_parts == joinrel->nparts);
 }
 
 /*
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 72b6832..c58b00c 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1623,6 +1623,9 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	int			partnatts;
 	int			cnt;
 	PartitionScheme part_scheme;
+	PartitionBoundInfo join_boundinfo;
+	List	   *parts1;
+	List	   *parts2;
 
 	/* Nothing to do if partition-wise join technique is disabled. */
 	if (!enable_partition_wise_join)
@@ -1663,17 +1666,26 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 		   REL_HAS_ALL_PART_PROPS(inner_rel));
 
 	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
+	 * Every pair of joining relations would yield the same partition bounds
+	 * for a given join (TODO: why?) so we compute the bounds only the first
+	 * time. Then for every pair we find the pairs of matching partitions from
+	 * the joining relations and join those. TODO: Needs a better explanation
+	 * of why is this true.  TODO: Also there is no reason to have
+	 * part_indexes1 and part_indexes2 pulled here just to be freed up later.
+	 * So, we might want to do something better.
 	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
+	join_boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											part_scheme->partsupfunc,
+											part_scheme->parttypcoll,
+											outer_rel->boundinfo,
+											outer_rel->nparts,
+											inner_rel->boundinfo,
+											inner_rel->nparts,
+											jointype, &parts1, &parts2);
+	if (!join_boundinfo)
 	{
 		Assert(!IS_PARTITIONED_REL(joinrel));
+		Assert(!parts1 && !parts2);
 		return;
 	}
 
@@ -1686,13 +1698,16 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 		   !joinrel->nullable_partexprs && !joinrel->part_rels &&
 		   !joinrel->boundinfo);
 
+	Assert(list_length(parts1) == list_length(parts2));
+
 	/*
 	 * Join relation is partitioned using same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations. It will have as many partitions as the pairs of
+	 * matching partitions we found.
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
-	joinrel->nparts = outer_rel->nparts;
+	joinrel->nparts = list_length(parts1);
+	joinrel->boundinfo = join_boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 
 	/*
@@ -1813,4 +1828,9 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 			joinrel->nullable_partexprs[cnt] = nullable_partexprs;
 		}
 	}
+
+	/* TODO: OR we could actually create the child-join relations here.*/
+	list_free(parts1);
+	list_free(parts2);
+
 }
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 2283c67..056a4f9 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -99,4 +99,10 @@ extern int get_partition_for_tuple(PartitionDispatch *pd,
 						EState *estate,
 						PartitionDispatchData **failed_at,
 						TupleTableSlot **failed_slot);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   FmgrInfo *supfuncs, Oid *collations,
+					   PartitionBoundInfo boundinfo1, int nparts1,
+					   PartitionBoundInfo boundinfo2, int nparts2,
+					   JoinType jointype, List **parts1, List **parts2);
+
 #endif							/* PARTITION_H */
-- 
1.7.9.5

#2Rajkumar Raghuwanshi
rajkumar.raghuwanshi@enterprisedb.com
In reply to: Ashutosh Bapat (#1)
1 attachment(s)
Re: advanced partition matching algorithm for partition-wise join

On Mon, Aug 21, 2017 at 12:43 PM, Ashutosh Bapat <
ashutosh.bapat@enterprisedb.com> wrote:

TODOs
-----------
1. Add tests for advanced partition matching algorithm

Hi Ashutosh,

I have applied all partition-wise-join patches (v26) and tested feature. I
have modified partition_join.sql file and added extra test cases to test
partition matching.

Attaching WIP test case patch which as of now have some server crashes and
a data corruptions issue which is commented in the file itself and need to
be removed once issue got solved. Also some of queries is not picking or
picking partition-wise-join as per expectation which may need some
adjustment.

Attachments:

advanced_partition_matching_test.patchtext/x-patch; charset=US-ASCII; name=advanced_partition_matching_test.patchDownload
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index a246d87..e9dd539 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -5,6 +5,9 @@
 -- Enable partition-wise join, which by default is disabled.
 SET enable_partition_wise_join to true;
 --
+-- tests for range partitioned tables.
+--
+--
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
@@ -16,15 +19,20 @@ CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
 ANALYZE prt1;
+-- prt2 have missing starting 0-50 range and missing ending 550-600
+-- range bounds
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
-CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
+CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (50) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
-CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (550);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0+50, 599-50, 3) i;
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap 
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -56,10 +64,10 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
   a  |  c   |  b  |  c   
 -----+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
+  50 | 0050 |  50 | 0050
+ 200 | 0200 | 200 | 0200
+ 350 | 0350 | 350 | 0350
+ 500 | 0500 | 500 | 0500
 (4 rows)
 
 -- left outer join, with whole-row reference
@@ -94,25 +102,25 @@ SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
       t1      |      t2      
 --------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
+ (0,0,0000)   | 
+ (50,0,0050)  | (0,50,0050)
  (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
+ (150,0,0150) | 
+ (200,0,0200) | (0,200,0200)
  (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
+ (300,0,0300) | 
+ (350,0,0350) | (0,350,0350)
  (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
+ (450,0,0450) | 
+ (500,0,0500) | (0,500,0500)
  (550,0,0550) | 
 (12 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-                             QUERY PLAN                              
----------------------------------------------------------------------
+                       QUERY PLAN                       
+--------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b
    ->  Result
@@ -129,171 +137,150 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
                      ->  Hash
                            ->  Seq Scan on prt2_p2 t2_1
                                  Filter: (a = 0)
-               ->  Nested Loop Left Join
-                     ->  Seq Scan on prt2_p3 t2_2
-                           Filter: (a = 0)
-                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_1
-                           Index Cond: (a = t2_2.b)
-(21 rows)
+               ->  Hash Right Join
+                     Hash Cond: (t1_1.a = t2_2.b)
+                     ->  Seq Scan on prt1_p3 t1_1
+                     ->  Hash
+                           ->  Seq Scan on prt2_p3 t2_2
+                                 Filter: (a = 0)
+(22 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
   a  |  c   |  b  |  c   
 -----+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  50 | 0050 |  50 | 0050
+ 200 | 0200 | 200 | 0200
+ 350 | 0350 | 350 | 0350
+ 500 | 0500 | 500 | 0500
+     |      | 125 | 0125
+     |      | 275 | 0275
+     |      | 425 | 0425
+(7 rows)
 
 -- full outer join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                 QUERY PLAN                  
+---------------------------------------------
  Sort
    Sort Key: prt1_p1.a, prt2_p1.b
-   ->  Append
-         ->  Hash Full Join
-               Hash Cond: (prt1_p1.a = prt2_p1.b)
+   ->  Hash Full Join
+         Hash Cond: (prt1_p1.a = prt2_p1.b)
+         ->  Append
                ->  Seq Scan on prt1_p1
                      Filter: (b = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_p1
-                           Filter: (a = 0)
-         ->  Hash Full Join
-               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               ->  Seq Scan on prt1_p3
+                     Filter: (b = 0)
                ->  Seq Scan on prt1_p2
                      Filter: (b = 0)
-               ->  Hash
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p1
+                           Filter: (a = 0)
                      ->  Seq Scan on prt2_p2
                            Filter: (a = 0)
-         ->  Hash Full Join
-               Hash Cond: (prt1_p3.a = prt2_p3.b)
-               ->  Seq Scan on prt1_p3
-                     Filter: (b = 0)
-               ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(24 rows)
+(19 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
   a  |  c   |  b  |  c   
 -----+------+-----+------
-   0 | 0000 |   0 | 0000
-  50 | 0050 |     | 
+   0 | 0000 |     | 
+  50 | 0050 |  50 | 0050
  100 | 0100 |     | 
- 150 | 0150 | 150 | 0150
- 200 | 0200 |     | 
+ 150 | 0150 |     | 
+ 200 | 0200 | 200 | 0200
  250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
+ 300 | 0300 |     | 
+ 350 | 0350 | 350 | 0350
  400 | 0400 |     | 
- 450 | 0450 | 450 | 0450
- 500 | 0500 |     | 
+ 450 | 0450 |     | 
+ 500 | 0500 | 500 | 0500
  550 | 0550 |     | 
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(16 rows)
+     |      | 125 | 0125
+     |      | 275 | 0275
+     |      | 425 | 0425
+(15 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
-                            QUERY PLAN                            
-------------------------------------------------------------------
+                         QUERY PLAN                         
+------------------------------------------------------------
  Sort
    Sort Key: prt1_p1.a, prt2_p1.b
-   ->  Append
-         ->  Hash Full Join
-               Hash Cond: (prt1_p1.a = prt2_p1.b)
-               Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
+   ->  Hash Full Join
+         Hash Cond: (prt1_p1.a = prt2_p1.b)
+         Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
+         ->  Append
                ->  Seq Scan on prt1_p1
                      Filter: (b = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_p1
-                           Filter: (a = 0)
-         ->  Hash Full Join
-               Hash Cond: (prt1_p2.a = prt2_p2.b)
-               Filter: (((50) = prt1_p2.a) OR ((75) = prt2_p2.b))
+               ->  Seq Scan on prt1_p3
+                     Filter: (b = 0)
                ->  Seq Scan on prt1_p2
                      Filter: (b = 0)
-               ->  Hash
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p1
+                           Filter: (a = 0)
                      ->  Seq Scan on prt2_p2
                            Filter: (a = 0)
-         ->  Hash Full Join
-               Hash Cond: (prt1_p3.a = prt2_p3.b)
-               Filter: (((50) = prt1_p3.a) OR ((75) = prt2_p3.b))
-               ->  Seq Scan on prt1_p3
-                     Filter: (b = 0)
-               ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(27 rows)
+(20 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
 ----+------+----+------
- 50 | 0050 |    | 
-    |      | 75 | 0075
-(2 rows)
+ 50 | 0050 | 50 | 0050
+(1 row)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                 QUERY PLAN                  
+---------------------------------------------
  Sort
    Sort Key: prt1_p1.a, prt2_p1.b
-   ->  Result
+   ->  Hash Full Join
+         Hash Cond: (prt1_p1.a = prt2_p1.b)
          ->  Append
-               ->  Hash Full Join
-                     Hash Cond: (prt1_p1.a = prt2_p1.b)
-                     ->  Seq Scan on prt1_p1
-                           Filter: (b = 0)
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1
-                                 Filter: (a = 0)
-               ->  Hash Full Join
-                     Hash Cond: (prt1_p2.a = prt2_p2.b)
-                     ->  Seq Scan on prt1_p2
-                           Filter: (b = 0)
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2
-                                 Filter: (a = 0)
-               ->  Hash Full Join
-                     Hash Cond: (prt1_p3.a = prt2_p3.b)
-                     ->  Seq Scan on prt1_p3
-                           Filter: (b = 0)
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3
-                                 Filter: (a = 0)
-(25 rows)
+               ->  Seq Scan on prt1_p1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p3
+                           Filter: (a = 0)
+(19 rows)
 
 SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
   a  |  c   | phv |  b  |  c   | phv 
 -----+------+-----+-----+------+-----
-   0 | 0000 |  25 |   0 | 0000 |  50
-  50 | 0050 |  25 |     |      |    
+   0 | 0000 |  25 |     |      |    
+  50 | 0050 |  25 |  50 | 0050 |  50
  100 | 0100 |  25 |     |      |    
- 150 | 0150 |  25 | 150 | 0150 |  50
- 200 | 0200 |  25 |     |      |    
+ 150 | 0150 |  25 |     |      |    
+ 200 | 0200 |  25 | 200 | 0200 |  50
  250 | 0250 |  25 |     |      |    
- 300 | 0300 |  25 | 300 | 0300 |  50
- 350 | 0350 |  25 |     |      |    
+ 300 | 0300 |  25 |     |      |    
+ 350 | 0350 |  25 | 350 | 0350 |  50
  400 | 0400 |  25 |     |      |    
- 450 | 0450 |  25 | 450 | 0450 |  50
- 500 | 0500 |  25 |     |      |    
+ 450 | 0450 |  25 |     |      |    
+ 500 | 0500 |  25 | 500 | 0500 |  50
  550 | 0550 |  25 |     |      |    
-     |      |     |  75 | 0075 |  50
-     |      |     | 225 | 0225 |  50
-     |      |     | 375 | 0375 |  50
-     |      |     | 525 | 0525 |  50
-(16 rows)
+     |      |     | 125 | 0125 |  50
+     |      |     | 275 | 0275 |  50
+     |      |     | 425 | 0425 |  50
+(15 rows)
 
 -- Join with pruned partitions from joining relations
 EXPLAIN (COSTS OFF)
@@ -315,7 +302,7 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a <
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a < 450 AND t2.b > 250 AND t1.b = 0 ORDER BY t1.a, t2.b;
   a  |  c   |  b  |  c   
 -----+------+-----+------
- 300 | 0300 | 300 | 0300
+ 350 | 0350 | 350 | 0350
 (1 row)
 
 EXPLAIN (COSTS OFF)
@@ -350,43 +337,32 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
  150 | 0150 |     | 
  200 | 0200 |     | 
  250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
+ 300 | 0300 |     | 
+ 350 | 0350 | 350 | 0350
  400 | 0400 |     | 
 (9 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-                         QUERY PLAN                         
-------------------------------------------------------------
+                      QUERY PLAN                      
+------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
-   ->  Append
-         ->  Hash Full Join
-               Hash Cond: (prt1_p1.a = b)
-               Filter: ((prt1_p1.b = 0) OR (a = 0))
+   Sort Key: prt1_p1.a, prt2_p2.b
+   ->  Hash Full Join
+         Hash Cond: (prt1_p1.a = prt2_p2.b)
+         Filter: ((prt1_p1.b = 0) OR (prt2_p2.a = 0))
+         ->  Append
                ->  Seq Scan on prt1_p1
                      Filter: (a < 450)
-               ->  Hash
-                     ->  Result
-                           One-Time Filter: false
-         ->  Hash Full Join
-               Hash Cond: (prt1_p2.a = prt2_p2.b)
-               Filter: ((prt1_p2.b = 0) OR (prt2_p2.a = 0))
                ->  Seq Scan on prt1_p2
                      Filter: (a < 450)
-               ->  Hash
+         ->  Hash
+               ->  Append
                      ->  Seq Scan on prt2_p2
                            Filter: (b > 250)
-         ->  Hash Full Join
-               Hash Cond: (prt2_p3.b = a)
-               Filter: ((b = 0) OR (prt2_p3.a = 0))
-               ->  Seq Scan on prt2_p3
-                     Filter: (b > 250)
-               ->  Hash
-                     ->  Result
-                           One-Time Filter: false
-(27 rows)
+                     ->  Seq Scan on prt2_p3
+                           Filter: (b > 250)
+(16 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
   a  |  c   |  b  |  c   
@@ -397,12 +373,12 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
  150 | 0150 |     | 
  200 | 0200 |     | 
  250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
+ 300 | 0300 |     | 
+ 350 | 0350 | 350 | 0350
  400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
+     |      | 275 | 0275
+     |      | 425 | 0425
+     |      | 500 | 0500
 (12 rows)
 
 -- Semi-join
@@ -439,10 +415,46 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
   a  | b |  c   
 -----+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
+  50 | 0 | 0050
+ 200 | 0 | 0200
+ 350 | 0 | 0350
+ 500 | 0 | 0500
+(4 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Append
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Seq Scan on prt2_p1 t1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p1 t2
+                     Filter: (b = 0)
+   ->  Hash Semi Join
+         Hash Cond: (t1_1.b = t2_2.a)
+         ->  Seq Scan on prt2_p2 t1_1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p2 t2_2
+                     Filter: (b = 0)
+   ->  Nested Loop Semi Join
+         Join Filter: (t1_2.b = t2_1.a)
+         ->  Seq Scan on prt2_p3 t1_2
+               Filter: (a = 0)
+         ->  Seq Scan on prt1_p3 t2_1
+               Filter: (b = 0)
+(21 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.a;
+ a |  b  |  c   
+---+-----+------
+ 0 |  50 | 0050
+ 0 | 200 | 0200
+ 0 | 350 | 0350
+ 0 | 500 | 0500
 (4 rows)
 
 -- Anti-join with aggregates
@@ -472,7 +484,31 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
   sum  |         avg          | sum  |         avg         
 -------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+ 64584 | 299.0000000000000000 | 2584 | 11.9629629629629630
+(1 row)
+
+EXPLAIN (COSTS OFF)
+SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a);
+                    QUERY PLAN                    
+--------------------------------------------------
+ Aggregate
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p1 t1
+               ->  Seq Scan on prt2_p2 t1_1
+               ->  Seq Scan on prt2_p3 t1_2
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p1 t2
+                     ->  Seq Scan on prt1_p3 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+(12 rows)
+
+SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a);
+ sum |         avg         |  sum  |         avg          
+-----+---------------------+-------+----------------------
+ 992 | 11.9518072289156627 | 24817 | 299.0000000000000000
 (1 row)
 
 -- lateral reference
@@ -484,50 +520,46 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 --------------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
-   ->  Result
+   ->  Nested Loop Left Join
          ->  Append
-               ->  Nested Loop Left Join
-                     ->  Seq Scan on prt1_p1 t1
-                           Filter: (b = 0)
-                     ->  Nested Loop
+               ->  Seq Scan on prt1_p1 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t3.b = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt2_p1 t3
+                     ->  Seq Scan on prt2_p2 t3_1
+                     ->  Seq Scan on prt2_p3 t3_2
+               ->  Hash
+                     ->  Append
                            ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
                                  Index Cond: (a = t1.a)
-                           ->  Index Scan using iprt2_p1_b on prt2_p1 t3
-                                 Index Cond: (b = t2.a)
-               ->  Nested Loop Left Join
-                     ->  Seq Scan on prt1_p2 t1_2
-                           Filter: (b = 0)
-                     ->  Nested Loop
-                           ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
-                                 Index Cond: (a = t1_2.a)
-                           ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
-                                 Index Cond: (b = t2_2.a)
-               ->  Nested Loop Left Join
-                     ->  Seq Scan on prt1_p3 t1_1
-                           Filter: (b = 0)
-                     ->  Nested Loop
                            ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_1
-                                 Index Cond: (a = t1_1.a)
-                           ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
-                                 Index Cond: (b = t2_1.a)
-(28 rows)
+                                 Index Cond: (a = t1.a)
+                           ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                                 Index Cond: (a = t1.a)
+(24 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
   a  | b |  c   | t2a | t3a | least 
 -----+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
+   0 | 0 | 0000 |     |     |      
+  50 | 0 | 0050 |  50 |   0 |    50
  100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
+ 150 | 0 | 0150 |     |     |      
+ 200 | 0 | 0200 | 200 |   0 |   200
  250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
+ 300 | 0 | 0300 |     |     |      
+ 350 | 0 | 0350 | 350 |   0 |   350
  400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
+ 450 | 0 | 0450 |     |     |      
+ 500 | 0 | 0500 | 500 |   0 |   500
  550 | 0 | 0550 |     |     |      
 (12 rows)
 
@@ -570,17 +602,17 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
   a  | t2a | t2c  
 -----+-----+------
-   0 |   0 | 0000
-  50 |     | 
+   0 |     | 
+  50 |  50 | 0050
  100 |     | 
- 150 | 150 | 0150
- 200 |     | 
+ 150 |     | 
+ 200 | 200 | 0200
  250 |     | 
- 300 | 300 | 0300
- 350 |     | 
+ 300 |     | 
+ 350 | 350 | 0350
  400 |     | 
- 450 | 450 | 0450
- 500 |     | 
+ 450 |     | 
+ 500 | 500 | 0500
  550 |     | 
 (12 rows)
 
@@ -641,55 +673,50 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
 --
 -- N-way join
 --
+--TODO: remove below comments and enable partition_wise_join
+--one issue got fixed
+--Server crashed with below queries
+--setting partition-wise-join to off
+SET enable_partition_wise_join to false;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-                                QUERY PLAN                                 
----------------------------------------------------------------------------
+                          QUERY PLAN                          
+--------------------------------------------------------------
  Sort
    Sort Key: t1.a
-   ->  Result
+   ->  Hash Join
+         Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
          ->  Append
-               ->  Nested Loop
-                     Join Filter: (t1.a = ((t3.a + t3.b) / 2))
-                     ->  Hash Join
-                           Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt1_e_p1 t3
+               ->  Seq Scan on prt1_e_p2 t3_1
+               ->  Seq Scan on prt1_e_p3 t3_2
+         ->  Hash
+               ->  Hash Join
+                     Hash Cond: (t2.b = t1.a)
+                     ->  Append
                            ->  Seq Scan on prt2_p1 t2
-                           ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
-                                       Filter: (b = 0)
-                     ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                           Index Cond: (((a + b) / 2) = t2.b)
-               ->  Nested Loop
-                     Join Filter: (t1_2.a = ((t3_1.a + t3_1.b) / 2))
-                     ->  Hash Join
-                           Hash Cond: (t2_1.b = t1_2.a)
                            ->  Seq Scan on prt2_p2 t2_1
-                           ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_2
-                                       Filter: (b = 0)
-                     ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
-                           Index Cond: (((a + b) / 2) = t2_1.b)
-               ->  Nested Loop
-                     Join Filter: (t1_1.a = ((t3_2.a + t3_2.b) / 2))
-                     ->  Hash Join
-                           Hash Cond: (t2_2.b = t1_1.a)
                            ->  Seq Scan on prt2_p3 t2_2
-                           ->  Hash
+                     ->  Hash
+                           ->  Append
+                                 ->  Seq Scan on prt1_p1 t1
+                                       Filter: (b = 0)
                                  ->  Seq Scan on prt1_p3 t1_1
                                        Filter: (b = 0)
-                     ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
-                           Index Cond: (((a + b) / 2) = t2_2.b)
-(34 rows)
+                                 ->  Seq Scan on prt1_p2 t1_2
+                                       Filter: (b = 0)
+(23 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
   a  |  c   |  b  |  c   | ?column? | c 
 -----+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
+  50 | 0050 |  50 | 0050 |      100 | 0
+ 200 | 0200 | 200 | 0200 |      400 | 0
+ 350 | 0350 | 350 | 0350 |      700 | 0
+ 500 | 0500 | 500 | 0500 |     1000 | 0
 (4 rows)
 
+SET enable_partition_wise_join to true;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
                              QUERY PLAN                             
@@ -733,17 +760,17 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
   a  |  c   |  b  |  c   | ?column? | c 
 -----+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
+   0 | 0000 |     |      |        0 | 0
+  50 | 0050 |  50 | 0050 |      100 | 0
  100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
+ 150 | 0150 |     |      |      300 | 0
+ 200 | 0200 | 200 | 0200 |      400 | 0
  250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
+ 300 | 0300 |     |      |      600 | 0
+ 350 | 0350 | 350 | 0350 |      700 | 0
  400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
+ 450 | 0450 |     |      |      900 | 0
+ 500 | 0500 | 500 | 0500 |     1000 | 0
  550 | 0550 |     |      |     1100 | 0
 (12 rows)
 
@@ -787,17 +814,17 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
   a  |  c   |  b  |  c   | ?column? | c 
 -----+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
+   0 | 0000 |     |      |        0 | 0
+  50 | 0050 |  50 | 0050 |      100 | 0
  100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
+ 150 | 0150 |     |      |      300 | 0
+ 200 | 0200 | 200 | 0200 |      400 | 0
  250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
+ 300 | 0300 |     |      |      600 | 0
+ 350 | 0350 | 350 | 0350 |      700 | 0
  400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
+ 450 | 0450 |     |      |      900 | 0
+ 500 | 0500 | 500 | 0500 |     1000 | 0
  550 | 0550 |     |      |     1100 | 0
 (12 rows)
 
@@ -805,65 +832,51 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
 -- make sure these go to null as expected
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
-                                                      QUERY PLAN                                                      
-----------------------------------------------------------------------------------------------------------------------
+                                                QUERY PLAN                                                
+----------------------------------------------------------------------------------------------------------
  Sort
    Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
-   ->  Result
-         ->  Append
-               ->  Hash Full Join
-                     Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
-                     Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
-                     ->  Hash Full Join
-                           Hash Cond: (prt1_p1.a = prt2_p1.b)
-                           ->  Seq Scan on prt1_p1
-                                 Filter: (b = 0)
-                           ->  Hash
-                                 ->  Seq Scan on prt2_p1
-                                       Filter: (a = 0)
-                     ->  Hash
-                           ->  Seq Scan on prt1_e_p1
-                                 Filter: (c = 0)
-               ->  Hash Full Join
-                     Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
-                     Filter: ((prt1_p2.a = (50)) OR (prt2_p2.b = (75)) OR (((prt1_e_p2.a + prt1_e_p2.b) / 2) = (50)))
-                     ->  Hash Full Join
-                           Hash Cond: (prt1_p2.a = prt2_p2.b)
-                           ->  Seq Scan on prt1_p2
-                                 Filter: (b = 0)
-                           ->  Hash
-                                 ->  Seq Scan on prt2_p2
-                                       Filter: (a = 0)
-                     ->  Hash
-                           ->  Seq Scan on prt1_e_p2
-                                 Filter: (c = 0)
-               ->  Hash Full Join
-                     Hash Cond: (prt1_p3.a = ((prt1_e_p3.a + prt1_e_p3.b) / 2))
-                     Filter: ((prt1_p3.a = (50)) OR (prt2_p3.b = (75)) OR (((prt1_e_p3.a + prt1_e_p3.b) / 2) = (50)))
-                     ->  Hash Full Join
-                           Hash Cond: (prt1_p3.a = prt2_p3.b)
-                           ->  Seq Scan on prt1_p3
-                                 Filter: (b = 0)
-                           ->  Hash
-                                 ->  Seq Scan on prt2_p3
-                                       Filter: (a = 0)
-                     ->  Hash
-                           ->  Seq Scan on prt1_e_p3
-                                 Filter: (c = 0)
-(43 rows)
+   ->  Hash Full Join
+         Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
+         Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+         ->  Hash Full Join
+               Hash Cond: (prt1_p1.a = prt2_p1.b)
+               ->  Append
+                     ->  Seq Scan on prt1_p1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2
+                           Filter: (b = 0)
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt2_p1
+                                 Filter: (a = 0)
+                           ->  Seq Scan on prt2_p2
+                                 Filter: (a = 0)
+                           ->  Seq Scan on prt2_p3
+                                 Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_e_p1
+                           Filter: (c = 0)
+                     ->  Seq Scan on prt1_e_p2
+                           Filter: (c = 0)
+                     ->  Seq Scan on prt1_e_p3
+                           Filter: (c = 0)
+(30 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
 ----+-----+----+-----+----------+-----
- 50 |  50 |    |     |      100 |  50
-    |     | 75 |  75 |          |    
-(2 rows)
+ 50 |  50 | 50 |  75 |      100 |  50
+(1 row)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-                                   QUERY PLAN                                    
----------------------------------------------------------------------------------
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
@@ -897,23 +910,24 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
                Join Filter: (t1_1.a = t1_5.b)
                ->  HashAggregate
                      Group Key: t1_5.b
-                     ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
-                                 Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
+                     ->  Hash Join
+                           Hash Cond: (((t2_2.a + t2_2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p3 t2_2
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t1_5
+                                       Filter: (a = 0)
                ->  Index Scan using iprt1_p3_a on prt1_p3 t1_1
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
+(42 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
   a  | b |  c   
 -----+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
+  50 | 0 | 0050
+ 200 | 0 | 0200
+ 350 | 0 | 0350
+ 500 | 0 | 0500
 (4 rows)
 
 EXPLAIN (COSTS OFF)
@@ -965,10 +979,10 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
   a  | b |  c   
 -----+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
+  50 | 0 | 0050
+ 200 | 0 | 0200
+ 350 | 0 | 0350
+ 500 | 0 | 0500
 (4 rows)
 
 -- test merge joins
@@ -1030,10 +1044,10 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
   a  | b |  c   
 -----+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
+  50 | 0 | 0050
+ 200 | 0 | 0200
+ 350 | 0 | 0350
+ 500 | 0 | 0500
 (4 rows)
 
 EXPLAIN (COSTS OFF)
@@ -1097,22 +1111,899 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
   a  |  c   |  b  |  c   | ?column? | c 
 -----+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
+   0 | 0000 |     |      |        0 | 0
+  50 | 0050 |  50 | 0050 |      100 | 0
  100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
+ 150 | 0150 |     |      |      300 | 0
+ 200 | 0200 | 200 | 0200 |      400 | 0
  250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
+ 300 | 0300 |     |      |      600 | 0
+ 350 | 0350 | 350 | 0350 |      700 | 0
  400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
+ 450 | 0450 |     |      |      900 | 0
+ 500 | 0500 | 500 | 0500 |     1000 | 0
  550 | 0550 |     |      |     1100 | 0
 (12 rows)
 
 RESET enable_hashjoin;
 RESET enable_nestloop;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 699, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p3 t1_1
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a  |  c   | a |  c   
+-----+------+---+------
+  50 | 0050 | 0 | 0050
+ 200 | 0200 | 0 | 0200
+ 350 | 0350 | 0 | 0350
+ 500 | 0500 | 0 | 0500
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p3 t1_1
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a  |  c   | a |  c   
+-----+------+---+------
+   0 | 0000 |   | 
+  50 | 0050 | 0 | 0050
+ 100 | 0100 |   | 
+ 150 | 0150 |   | 
+ 200 | 0200 | 0 | 0200
+ 250 | 0250 |   | 
+ 300 | 0300 |   | 
+ 350 | 0350 | 0 | 0350
+ 400 | 0400 |   | 
+ 450 | 0450 |   | 
+ 500 | 0500 | 0 | 0500
+ 550 | 0550 |   | 
+(12 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p1 t1
+         ->  Seq Scan on prt1_p3 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p1 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_3
+                     Filter: (a = 0)
+(16 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a;
+  a  |  c   | a |  c   
+-----+------+---+------
+  50 | 0050 | 0 | 0050
+ 200 | 0200 | 0 | 0200
+ 350 | 0350 | 0 | 0350
+ 500 | 0500 | 0 | 0500
+     |      | 0 | 0275
+     |      | 0 | 0425
+     |      | 0 | 0125
+     |      | 0 | 0600
+     |      | 0 | 0675
+(9 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.b)
+               Join Filter: ((t1.b + t2.a) = 0)
+               ->  Seq Scan on prt1_p1 t1
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2
+         ->  Hash Join
+               Hash Cond: (t1_2.a = t2_1.b)
+               Join Filter: ((t1_2.b + t2_1.a) = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_1
+         ->  Hash Join
+               Hash Cond: (t1_1.a = t2_2.b)
+               Join Filter: ((t1_1.b + t2_2.a) = 0)
+               ->  Seq Scan on prt1_p3 t1_1
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_2
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+  a  |  c   | a |  c   
+-----+------+---+------
+  50 | 0050 | 0 | 0050
+ 200 | 0200 | 0 | 0200
+ 350 | 0350 | 0 | 0350
+ 500 | 0500 | 0 | 0500
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p1 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_1.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_2.b)
+               ->  Seq Scan on prt1_p3 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_2
+(21 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  50 | 0 | 0050
+ 200 | 0 | 0200
+ 350 | 0 | 0350
+ 500 | 0 | 0500
+(4 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p1 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_1
+                     Index Cond: (a = t1_2.b)
+(20 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b  |  c   
+---+-----+------
+ 0 |  50 | 0050
+ 0 | 200 | 0200
+ 0 | 350 | 0350
+ 0 | 500 | 0500
+(4 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p1 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_1.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_2.b)
+               ->  Seq Scan on prt1_p3 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_2
+(21 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+ 100 | 0 | 0100
+ 150 | 0 | 0150
+ 250 | 0 | 0250
+ 300 | 0 | 0300
+ 400 | 0 | 0400
+ 450 | 0 | 0450
+ 550 | 0 | 0550
+(8 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p1 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt2_p3 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_1
+                     Index Cond: (a = t1_2.b)
+(20 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b  |  c   
+---+-----+------
+ 0 | 125 | 0125
+ 0 | 275 | 0275
+ 0 | 425 | 0425
+(3 rows)
+
+--TODO: remove below comments and enable partition_wise_join
+--one issue got fixed
+--Getting wrong output with partition wise join
+--setting partition-wise-join to off to get correct output
+SET enable_partition_wise_join to false;
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b  |  c   
+---+-----+------
+ 0 | 125 | 0125
+ 0 | 275 | 0275
+ 0 | 425 | 0425
+ 0 | 600 | 0600
+ 0 | 675 | 0675
+(5 rows)
+
+SET enable_partition_wise_join to true;
+-- N-Way joins
+--TODO: remove below comments and enable partition_wise_join
+--one issue got fixed
+--Server crashed with below queries
+--setting partition-wise-join to off
+SET enable_partition_wise_join to false;
+-- t1 join t2 qualify partition-wise-join and t2 join t3 qualify partition-wise-join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt2 t3 ON t2.b = t3.b WHERE t1.b = 0 ORDER BY 1,2,3,4,5,6;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c, t2.a, t2.c, t3.a, t3.c
+   ->  Hash Right Join
+         Hash Cond: (t3.b = t2.b)
+         ->  Append
+               ->  Seq Scan on prt2_p1 t3
+               ->  Seq Scan on prt2_p2 t3_1
+               ->  Seq Scan on prt2_p3 t3_2
+               ->  Seq Scan on prt2_p4 t3_3
+         ->  Hash
+               ->  Hash Join
+                     Hash Cond: (t2.b = t1.a)
+                     ->  Append
+                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p4 t2_3
+                     ->  Hash
+                           ->  Append
+                                 ->  Seq Scan on prt1_p1 t1
+                                       Filter: (b = 0)
+                                 ->  Seq Scan on prt1_p3 t1_1
+                                       Filter: (b = 0)
+                                 ->  Seq Scan on prt1_p2 t1_2
+                                       Filter: (b = 0)
+(25 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt2 t3 ON t2.b = t3.b WHERE t1.b = 0 ORDER BY 1,2,3,4,5,6;
+  a  |  c   | a |  c   | a |  c   
+-----+------+---+------+---+------
+  50 | 0050 | 0 | 0050 | 0 | 0050
+ 200 | 0200 | 0 | 0200 | 0 | 0200
+ 350 | 0350 | 0 | 0350 | 0 | 0350
+ 500 | 0500 | 0 | 0500 | 0 | 0500
+(4 rows)
+
+-- t1 join t2 qualify partition-wise-join and t2 join t3 not qualify partition-wise-join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt2 t3 ON t2.b = t3.b WHERE t1.b = 0 ORDER BY 1,2,3,4,5,6;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c, t2.a, t2.c, t3.a, t3.c
+   ->  Hash Join
+         Hash Cond: (t3.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p1 t3
+               ->  Seq Scan on prt2_p2 t3_1
+               ->  Seq Scan on prt2_p3 t3_2
+               ->  Seq Scan on prt2_p4 t3_3
+         ->  Hash
+               ->  Hash Join
+                     Hash Cond: (t2.b = t1.a)
+                     ->  Append
+                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p4 t2_3
+                     ->  Hash
+                           ->  Append
+                                 ->  Seq Scan on prt1_p1 t1
+                                       Filter: (b = 0)
+                                 ->  Seq Scan on prt1_p3 t1_1
+                                       Filter: (b = 0)
+                                 ->  Seq Scan on prt1_p2 t1_2
+                                       Filter: (b = 0)
+(25 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt2 t3 ON t2.b = t3.b WHERE t1.b = 0 ORDER BY 1,2,3,4,5,6;
+  a  |  c   | a |  c   | a |  c   
+-----+------+---+------+---+------
+  50 | 0050 | 0 | 0050 | 0 | 0050
+ 200 | 0200 | 0 | 0200 | 0 | 0200
+ 350 | 0350 | 0 | 0350 | 0 | 0350
+ 500 | 0500 | 0 | 0500 | 0 | 0500
+(4 rows)
+
+-- t1 join t2 not qualify partition-wise-join and t2 join t3 qualify partition-wise-join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) INNER JOIN prt2 t3 ON t2.b = t3.b WHERE t1.b = 0 ORDER BY 1,2,3,4,5,6;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c, t2.a, t2.c, t3.a, t3.c
+   ->  Hash Join
+         Hash Cond: (t3.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p1 t3
+               ->  Seq Scan on prt2_p2 t3_1
+               ->  Seq Scan on prt2_p3 t3_2
+               ->  Seq Scan on prt2_p4 t3_3
+         ->  Hash
+               ->  Hash Join
+                     Hash Cond: (t2.b = t1.a)
+                     ->  Append
+                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p4 t2_3
+                     ->  Hash
+                           ->  Append
+                                 ->  Seq Scan on prt1_p1 t1
+                                       Filter: (b = 0)
+                                 ->  Seq Scan on prt1_p3 t1_1
+                                       Filter: (b = 0)
+                                 ->  Seq Scan on prt1_p2 t1_2
+                                       Filter: (b = 0)
+(25 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) INNER JOIN prt2 t3 ON t2.b = t3.b WHERE t1.b = 0 ORDER BY 1,2,3,4,5,6;
+  a  |  c   | a |  c   | a |  c   
+-----+------+---+------+---+------
+  50 | 0050 | 0 | 0050 | 0 | 0050
+ 200 | 0200 | 0 | 0200 | 0 | 0200
+ 350 | 0350 | 0 | 0350 | 0 | 0350
+ 500 | 0500 | 0 | 0500 | 0 | 0500
+(4 rows)
+
+-- t1 join t2 not qualify partition-wise-join and t2 join t3 not qualify partition-wise-join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) FULL JOIN prt2 t3 ON t2.b = t3.b WHERE t1.b = 0 ORDER BY 1,2,3,4,5,6;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c, t2.a, t2.c, t3.a, t3.c
+   ->  Hash Right Join
+         Hash Cond: (t3.b = t2.b)
+         ->  Append
+               ->  Seq Scan on prt2_p1 t3
+               ->  Seq Scan on prt2_p2 t3_1
+               ->  Seq Scan on prt2_p3 t3_2
+               ->  Seq Scan on prt2_p4 t3_3
+         ->  Hash
+               ->  Hash Join
+                     Hash Cond: (t2.b = t1.a)
+                     ->  Append
+                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p4 t2_3
+                     ->  Hash
+                           ->  Append
+                                 ->  Seq Scan on prt1_p1 t1
+                                       Filter: (b = 0)
+                                 ->  Seq Scan on prt1_p3 t1_1
+                                       Filter: (b = 0)
+                                 ->  Seq Scan on prt1_p2 t1_2
+                                       Filter: (b = 0)
+(25 rows)
+
+SET enable_partition_wise_join to true;
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (550) TO (700);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(550, 699, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p4 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p4 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p1 t1
+         ->  Seq Scan on prt1_p3 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p1 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_3
+                     Filter: (a = 0)
+(16 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p3 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p4 t2_3
+(15 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p1 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p4 t2_3
+(17 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p1 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_3
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p1 t2
+                     ->  Seq Scan on prt1_p3 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+(18 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p1 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p4 t2_3
+(17 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p1 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_3
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p1 t2
+                     ->  Seq Scan on prt1_p3 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+(18 rows)
+
+-- N-Way joins
+-- t1 join t2 not qualify partition-wise-join and t2 join t3 qualify partition-wise-join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b) INNER JOIN prt1 t3 ON t2.b = t3.a WHERE t1.b = 0 ORDER BY 1,2,3,4,5,6;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c, t2.a, t2.c, t3.c
+   ->  Hash Join
+         Hash Cond: (t3.a = t1.a)
+         ->  Append
+               ->  Seq Scan on prt1_p1 t3
+               ->  Seq Scan on prt1_p3 t3_1
+               ->  Seq Scan on prt1_p2 t3_2
+         ->  Hash
+               ->  Hash Join
+                     Hash Cond: (t2.b = t1.a)
+                     ->  Append
+                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p4 t2_3
+                     ->  Hash
+                           ->  Append
+                                 ->  Seq Scan on prt1_p1 t1
+                                       Filter: (b = 0)
+                                 ->  Seq Scan on prt1_p3 t1_1
+                                       Filter: (b = 0)
+                                 ->  Seq Scan on prt1_p2 t1_2
+                                       Filter: (b = 0)
+(24 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b) INNER JOIN prt1 t3 ON t2.b = t3.a WHERE t1.b = 0 ORDER BY 1,2,3,4,5,6;
+  a  |  c   | a |  c   |  a  |  c   
+-----+------+---+------+-----+------
+  50 | 0050 | 0 | 0050 |  50 | 0050
+ 200 | 0200 | 0 | 0200 | 200 | 0200
+ 350 | 0350 | 0 | 0350 | 350 | 0350
+ 500 | 0500 | 0 | 0500 | 500 | 0500
+ 550 | 0550 | 0 | 0550 | 550 | 0550
+(5 rows)
+
+-- t1 join t2 not qualify partition-wise-join and t2 join t3 not qualify partition-wise-join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt2 t3 ON t2.b = t3.b WHERE t1.b = 0 ORDER BY 1,2,3,4,5,6;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c, t2.a, t2.c, t3.a, t3.c
+   ->  Hash Join
+         Hash Cond: (t3.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p1 t3
+               ->  Seq Scan on prt2_p2 t3_1
+               ->  Seq Scan on prt2_p3 t3_2
+               ->  Seq Scan on prt2_p4 t3_3
+         ->  Hash
+               ->  Hash Join
+                     Hash Cond: (t2.b = t1.a)
+                     ->  Append
+                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p4 t2_3
+                     ->  Hash
+                           ->  Append
+                                 ->  Seq Scan on prt1_p1 t1
+                                       Filter: (b = 0)
+                                 ->  Seq Scan on prt1_p3 t1_1
+                                       Filter: (b = 0)
+                                 ->  Seq Scan on prt1_p2 t1_2
+                                       Filter: (b = 0)
+(25 rows)
+
+-- Partition-wise join with minvalue/maxvalue bounds
+DROP TABLE prt2_p1;
+DROP TABLE prt2_p3;
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (MINVALUE) TO (250);
+CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 250, 3) i;
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(500, 600, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p3 t1_1
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a  |  c   | a |  c   
+-----+------+---+------
+   0 | 0000 | 0 | 0000
+ 150 | 0150 | 0 | 0150
+ 350 | 0350 | 0 | 0350
+ 500 | 0500 | 0 | 0500
+(4 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2_1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t1.a = t2_1.b)
+               Join Filter: ((t1.b + t2_1.a) = 0)
+               ->  Seq Scan on prt1_p1 t1
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Join
+               Hash Cond: (t1_2.a = t2.b)
+               Join Filter: ((t1_2.b + t2.a) = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2
+         ->  Hash Join
+               Hash Cond: (t1_1.a = t2_2.b)
+               Join Filter: ((t1_1.b + t2_2.a) = 0)
+               ->  Seq Scan on prt1_p3 t1_1
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_2
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+  a  |  c   | a |  c   
+-----+------+---+------
+   0 | 0000 | 0 | 0000
+ 150 | 0150 | 0 | 0150
+ 350 | 0350 | 0 | 0350
+ 500 | 0500 | 0 | 0500
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_2.b)
+               ->  Seq Scan on prt1_p3 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_2
+(21 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+ 150 | 0 | 0150
+ 350 | 0 | 0350
+ 500 | 0 | 0500
+(4 rows)
+
 --
 -- partitioned by multiple columns
 --
@@ -1182,83 +2073,2174 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009'); 
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap 
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p4 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2_3
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1
+         ->  Hash Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_2
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2_3
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1
+         ->  Hash Left Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_2
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Sort Key: t2_3.a
+   ->  Result
+         ->  Append
+               ->  Hash Right Join
+                     Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+                     Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+                     ->  Seq Scan on plt1_p4 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt2_p4 t2_3
+               ->  Hash Left Join
+                     Hash Cond: ((t2.c)::text = (t1.c)::text)
+                     Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+                     ->  Seq Scan on plt2_p1 t2
+                     ->  Hash
+                           ->  Seq Scan on plt1_p1 t1
+               ->  Hash Left Join
+                     Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+                     Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+                     ->  Seq Scan on plt2_p2 t2_1
+                     ->  Hash
+                           ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash Right Join
+                     Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+                     Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+                     ->  Seq Scan on plt1_p3 t1_2
+                     ->  Hash
+                           ->  Seq Scan on plt2_p3 t2_2
+(28 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a, t2_3.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2_3
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_2
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a, t1_3.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p4 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  HashAggregate
+                           Group Key: (t2_3.c)::text
+                           ->  Seq Scan on plt2_p4 t2_3
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p1 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p2 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p3 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_2
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a, t1_3.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_3.c)::text
+                     ->  Seq Scan on plt1_p4 t2_3
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1_3
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p1 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p2 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p3 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_2
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a, t1_3.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p4 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2_3
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p1 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p2 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p3 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_2
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a, t1_3.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p4 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2_3
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p1 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p2 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p3 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_2
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
+-- N-Way joins
+-- inner-inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1 t1,plt2 t2,plt1 t3 WHERE t1.c = t2.c AND t2.c = t3.c AND t1.b + t2.b = 0 AND t2.b + t3.b = 0 ORDER BY t1.a,t2.b;
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a, t2_3.b
+   ->  Result
+         ->  Append
+               ->  Merge Join
+                     Merge Cond: ((t2_3.c)::text = (t1_3.c)::text)
+                     Join Filter: ((t1_3.b + t2_3.b) = 0)
+                     ->  Merge Join
+                           Merge Cond: ((t3_3.c)::text = (t2_3.c)::text)
+                           Join Filter: ((t2_3.b + t3_3.b) = 0)
+                           ->  Sort
+                                 Sort Key: t3_3.c
+                                 ->  Seq Scan on plt1_p4 t3_3
+                           ->  Sort
+                                 Sort Key: t2_3.c
+                                 ->  Seq Scan on plt2_p4 t2_3
+                     ->  Sort
+                           Sort Key: t1_3.c
+                           ->  Seq Scan on plt1_p4 t1_3
+               ->  Hash Join
+                     Hash Cond: ((t3.c)::text = (t1.c)::text)
+                     Join Filter: ((t2.b + t3.b) = 0)
+                     ->  Seq Scan on plt1_p1 t3
+                     ->  Hash
+                           ->  Hash Join
+                                 Hash Cond: ((t2.c)::text = (t1.c)::text)
+                                 Join Filter: ((t1.b + t2.b) = 0)
+                                 ->  Seq Scan on plt2_p1 t2
+                                 ->  Hash
+                                       ->  Seq Scan on plt1_p1 t1
+               ->  Hash Join
+                     Hash Cond: ((t3_1.c)::text = (t1_1.c)::text)
+                     Join Filter: ((t2_1.b + t3_1.b) = 0)
+                     ->  Seq Scan on plt1_p2 t3_1
+                     ->  Hash
+                           ->  Hash Join
+                                 Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+                                 Join Filter: ((t1_1.b + t2_1.b) = 0)
+                                 ->  Seq Scan on plt1_p2 t1_1
+                                 ->  Hash
+                                       ->  Seq Scan on plt2_p2 t2_1
+               ->  Hash Join
+                     Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+                     Join Filter: ((t1_2.b + t2_2.b) = 0)
+                     ->  Seq Scan on plt1_p3 t1_2
+                     ->  Hash
+                           ->  Hash Join
+                                 Hash Cond: ((t3_2.c)::text = (t2_2.c)::text)
+                                 Join Filter: ((t2_2.b + t3_2.b) = 0)
+                                 ->  Seq Scan on plt1_p3 t3_2
+                                 ->  Hash
+                                       ->  Seq Scan on plt2_p3 t2_2
+(52 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1 t1,plt2 t2,plt1 t3 WHERE t1.c = t2.c AND t2.c = t3.c AND t1.b + t2.b = 0 AND t2.b + t3.b = 0 ORDER BY t1.a,t2.b;
+  a  |  c   |  a  |  c   |  a  |  c   
+-----+------+-----+------+-----+------
+   0 | 0000 |   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left-left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c LEFT JOIN plt2 t3 ON (t2.c = t3.c) WHERE t1.b + coalesce(t2.b, 0) = 0 AND t2.b + coalesce(t3.b, 0) = 0 ORDER BY t1.a, t2.b;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a, t2_3.b
+   ->  Result
+         ->  Append
+               ->  Hash Right Join
+                     Hash Cond: ((t3_3.c)::text = (t2_3.c)::text)
+                     Filter: ((t2_3.b + COALESCE(t3_3.b, 0)) = 0)
+                     ->  Seq Scan on plt2_p4 t3_3
+                     ->  Hash
+                           ->  Hash Join
+                                 Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+                                 Join Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+                                 ->  Seq Scan on plt1_p4 t1_3
+                                 ->  Hash
+                                       ->  Seq Scan on plt2_p4 t2_3
+               ->  Hash Join
+                     Hash Cond: ((t1.c)::text = (t2.c)::text)
+                     Join Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+                     ->  Seq Scan on plt1_p1 t1
+                     ->  Hash
+                           ->  Hash Left Join
+                                 Hash Cond: ((t2.c)::text = (t3.c)::text)
+                                 Filter: ((t2.b + COALESCE(t3.b, 0)) = 0)
+                                 ->  Seq Scan on plt2_p1 t2
+                                 ->  Hash
+                                       ->  Seq Scan on plt2_p1 t3
+               ->  Hash Right Join
+                     Hash Cond: ((t3_1.c)::text = (t2_1.c)::text)
+                     Filter: ((t2_1.b + COALESCE(t3_1.b, 0)) = 0)
+                     ->  Seq Scan on plt2_p2 t3_1
+                     ->  Hash
+                           ->  Hash Join
+                                 Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+                                 Join Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+                                 ->  Seq Scan on plt1_p2 t1_1
+                                 ->  Hash
+                                       ->  Seq Scan on plt2_p2 t2_1
+               ->  Hash Right Join
+                     Hash Cond: ((t3_2.c)::text = (t2_2.c)::text)
+                     Filter: ((t2_2.b + COALESCE(t3_2.b, 0)) = 0)
+                     ->  Seq Scan on plt2_p3 t3_2
+                     ->  Hash
+                           ->  Hash Join
+                                 Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+                                 Join Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+                                 ->  Seq Scan on plt1_p3 t1_2
+                                 ->  Hash
+                                       ->  Seq Scan on plt2_p3 t2_2
+(48 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c LEFT JOIN plt2 t3 ON (t2.c = t3.c) WHERE t1.b + coalesce(t2.b, 0) = 0 AND t2.b + coalesce(t3.b, 0) = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  a  |  c   |  a  |  c   
+-----+------+-----+------+-----+------
+   0 | 0000 |   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left-right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c RIGHT JOIN plt1 t3 ON (t2.c = t3.c) WHERE t1.b + coalesce(t2.b,0) = 0 AND coalesce(t1.b, 0) + t3.b = 0 ORDER BY t1.a, t2.b;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a, t2_3.b
+   ->  Result
+         ->  Append
+               ->  Hash Join
+                     Hash Cond: ((t3_3.c)::text = (t1_3.c)::text)
+                     Join Filter: ((COALESCE(t1_3.b, 0) + t3_3.b) = 0)
+                     ->  Seq Scan on plt1_p4 t3_3
+                     ->  Hash
+                           ->  Hash Join
+                                 Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+                                 Join Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+                                 ->  Seq Scan on plt1_p4 t1_3
+                                 ->  Hash
+                                       ->  Seq Scan on plt2_p4 t2_3
+               ->  Hash Join
+                     Hash Cond: ((t3.c)::text = (t1.c)::text)
+                     Join Filter: ((COALESCE(t1.b, 0) + t3.b) = 0)
+                     ->  Seq Scan on plt1_p1 t3
+                     ->  Hash
+                           ->  Hash Join
+                                 Hash Cond: ((t2.c)::text = (t1.c)::text)
+                                 Join Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+                                 ->  Seq Scan on plt2_p1 t2
+                                 ->  Hash
+                                       ->  Seq Scan on plt1_p1 t1
+               ->  Hash Join
+                     Hash Cond: ((t3_1.c)::text = (t1_1.c)::text)
+                     Join Filter: ((COALESCE(t1_1.b, 0) + t3_1.b) = 0)
+                     ->  Seq Scan on plt1_p2 t3_1
+                     ->  Hash
+                           ->  Hash Join
+                                 Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+                                 Join Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+                                 ->  Seq Scan on plt1_p2 t1_1
+                                 ->  Hash
+                                       ->  Seq Scan on plt2_p2 t2_1
+               ->  Hash Join
+                     Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+                     Join Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+                     ->  Seq Scan on plt2_p3 t2_2
+                     ->  Hash
+                           ->  Hash Join
+                                 Hash Cond: ((t1_2.c)::text = (t3_2.c)::text)
+                                 Join Filter: ((COALESCE(t1_2.b, 0) + t3_2.b) = 0)
+                                 ->  Seq Scan on plt1_p3 t1_2
+                                 ->  Hash
+                                       ->  Seq Scan on plt1_p3 t3_2
+(48 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c RIGHT JOIN plt1 t3 ON (t2.c = t3.c) WHERE t1.b + coalesce(t2.b,0) = 0 AND coalesce(t1.b, 0) + t3.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  a  |  c   |  a  |  c   
+-----+------+-----+------+-----+------
+   0 | 0000 |   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- right-full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c FULL JOIN plt1 t3 ON (t2.c = t3.c) WHERE coalesce(t1.b,0) + t2.b = 0 AND coalesce(t2.b,0) + coalesce(t3.b,0) = 0 ORDER BY t1.a, t2.b;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a, t2_3.b
+   ->  Hash Right Join
+         Hash Cond: ((t3.c)::text = (t2_3.c)::text)
+         Filter: ((COALESCE(t2_3.b, 0) + COALESCE(t3.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p1 t3
+               ->  Seq Scan on plt1_p2 t3_1
+               ->  Seq Scan on plt1_p3 t3_2
+               ->  Seq Scan on plt1_p4 t3_3
+         ->  Hash
+               ->  Append
+                     ->  Hash Right Join
+                           Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+                           Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+                           ->  Seq Scan on plt1_p4 t1_3
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p4 t2_3
+                     ->  Hash Left Join
+                           Hash Cond: ((t2.c)::text = (t1.c)::text)
+                           Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+                           ->  Seq Scan on plt2_p1 t2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p1 t1
+                     ->  Hash Left Join
+                           Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+                           Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+                           ->  Seq Scan on plt2_p2 t2_1
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p2 t1_1
+                     ->  Hash Right Join
+                           Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+                           Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p3 t2_2
+(36 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c FULL JOIN plt1 t3 ON (t2.c = t3.c) WHERE coalesce(t1.b,0) + t2.b = 0 AND coalesce(t2.b,0) + coalesce(t3.b,0) = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  a  |  c   |  a  |  c   
+-----+------+-----+------+-----+------
+   0 | 0000 |   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002 | 376 | 0002
+     |      | 470 | 0011 |     | 
+(6 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT t3.c FROM plt1 t3 WHERE t3.b = 0)) AND t1.b = 0 ORDER BY t1.a;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1_3.c)::text = (t1_7.c)::text)
+               ->  HashAggregate
+                     Group Key: (t1_7.c)::text
+                     ->  Hash Join
+                           Hash Cond: ((t1_7.c)::text = (t3_3.c)::text)
+                           ->  Seq Scan on plt2_p4 t1_7
+                           ->  Hash
+                                 ->  HashAggregate
+                                       Group Key: (t3_3.c)::text
+                                       ->  Seq Scan on plt1_p4 t3_3
+                                             Filter: (b = 0)
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1_3
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t1_4.c)::text)
+               ->  HashAggregate
+                     Group Key: (t1_4.c)::text
+                     ->  Hash Join
+                           Hash Cond: ((t1_4.c)::text = (t3.c)::text)
+                           ->  Seq Scan on plt2_p1 t1_4
+                           ->  Hash
+                                 ->  HashAggregate
+                                       Group Key: (t3.c)::text
+                                       ->  Seq Scan on plt1_p1 t3
+                                             Filter: (b = 0)
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t1_5.c)::text)
+               ->  HashAggregate
+                     Group Key: (t1_5.c)::text
+                     ->  Hash Join
+                           Hash Cond: ((t1_5.c)::text = (t3_1.c)::text)
+                           ->  Seq Scan on plt2_p2 t1_5
+                           ->  Hash
+                                 ->  HashAggregate
+                                       Group Key: (t3_1.c)::text
+                                       ->  Seq Scan on plt1_p2 t3_1
+                                             Filter: (b = 0)
+               ->  Materialize
+                     ->  Seq Scan on plt1_p2 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t1_6.c)::text)
+               ->  Seq Scan on plt1_p3 t1_2
+                     Filter: (b = 0)
+               ->  Hash Semi Join
+                     Hash Cond: ((t1_6.c)::text = (t3_2.c)::text)
+                     ->  Seq Scan on plt2_p3 t1_6
+                     ->  Hash
+                           ->  Seq Scan on plt1_p3 t3_2
+                                 Filter: (b = 0)
+(58 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT t3.c FROM plt1 t3 WHERE t3.b = 0)) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
--- test partition matching with N-way join
+-- test partition matching with N-way join with expression
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                      QUERY PLAN                                      
---------------------------------------------------------------------------------------
+                                       QUERY PLAN                                       
+----------------------------------------------------------------------------------------
  Sort
-   Sort Key: t1.c, t3.c
+   Sort Key: t1_3.c, t3_3.c
    ->  HashAggregate
-         Group Key: t1.c, t2.c, t3.c
+         Group Key: t1_3.c, t2_3.c, t3_3.c
          ->  Result
                ->  Append
                      ->  Hash Join
-                           Hash Cond: (t1.c = t2.c)
-                           ->  Seq Scan on plt1_p1 t1
+                           Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+                           ->  Hash Join
+                                 Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                                 ->  Seq Scan on plt1_p4 t1_3
+                                 ->  Hash
+                                       ->  Seq Scan on plt1_e_p4 t3_3
                            ->  Hash
-                                 ->  Hash Join
-                                       Hash Cond: (t2.c = ltrim(t3.c, 'A'::text))
-                                       ->  Seq Scan on plt2_p1 t2
-                                       ->  Hash
-                                             ->  Seq Scan on plt1_e_p1 t3
+                                 ->  Seq Scan on plt2_p4 t2_3
                      ->  Hash Join
-                           Hash Cond: (t1_1.c = t2_1.c)
-                           ->  Seq Scan on plt1_p2 t1_1
+                           Hash Cond: ((t2.c)::text = (t1.c)::text)
+                           ->  Hash Join
+                                 Hash Cond: ((t2.c)::text = ltrim(t3.c, 'A'::text))
+                                 ->  Seq Scan on plt2_p1 t2
+                                 ->  Hash
+                                       ->  Seq Scan on plt1_e_p1 t3
                            ->  Hash
-                                 ->  Hash Join
-                                       Hash Cond: (t2_1.c = ltrim(t3_1.c, 'A'::text))
-                                       ->  Seq Scan on plt2_p2 t2_1
-                                       ->  Hash
-                                             ->  Seq Scan on plt1_e_p2 t3_1
+                                 ->  Seq Scan on plt1_p1 t1
                      ->  Hash Join
-                           Hash Cond: (t1_2.c = t2_2.c)
-                           ->  Seq Scan on plt1_p3 t1_2
+                           Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+                           ->  Hash Join
+                                 Hash Cond: ((t1_1.c)::text = ltrim(t3_1.c, 'A'::text))
+                                 ->  Seq Scan on plt1_p2 t1_1
+                                 ->  Hash
+                                       ->  Seq Scan on plt1_e_p2 t3_1
                            ->  Hash
-                                 ->  Hash Join
-                                       Hash Cond: (t2_2.c = ltrim(t3_2.c, 'A'::text))
-                                       ->  Seq Scan on plt2_p3 t2_2
-                                       ->  Hash
-                                             ->  Seq Scan on plt1_e_p3 t3_2
-(33 rows)
+                                 ->  Seq Scan on plt2_p2 t2_1
+                     ->  Hash Join
+                           Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+                           ->  Hash Join
+                                 Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
+                                 ->  Seq Scan on plt1_p3 t1_2
+                                 ->  Hash
+                                       ->  Seq Scan on plt1_e_p3 t3_2
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p3 t2_2
+(42 rows)
 
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  74.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 124.0000000000000000 | 124.5000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 224.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 274.0000000000000000 | 274.5000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 374.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 424.0000000000000000 | 424.5000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 524.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 574.0000000000000000 | 574.5000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join 
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p4 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2_3
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1
+         ->  Hash Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_2
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2_3
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1
+         ->  Hash Left Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_2
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p1 t1
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Seq Scan on plt1_p3 t1_2
+               ->  Seq Scan on plt1_p4 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p1 t2
+                     ->  Seq Scan on plt2_p2 t2_1
+                     ->  Seq Scan on plt2_p3 t2_2
+                     ->  Seq Scan on plt2_p4 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p1 t1
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Seq Scan on plt1_p3 t1_2
+               ->  Seq Scan on plt1_p4 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p1 t2
+                     ->  Seq Scan on plt2_p2 t2_1
+                     ->  Seq Scan on plt2_p3 t2_2
+                     ->  Seq Scan on plt2_p4 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join 
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a, t1_3.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p4 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  HashAggregate
+                           Group Key: (t2_3.c)::text
+                           ->  Seq Scan on plt2_p4 t2_3
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p1 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p2 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p3 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_2
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a, t1_3.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_3.c)::text
+                     ->  Seq Scan on plt1_p4 t2_3
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1_3
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p1 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p2 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p3 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_2
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a, t1_3.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p4 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2_3
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p1 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p2 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p3 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_2
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p1 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p1 t2
+                     ->  Seq Scan on plt1_p2 t2_1
+                     ->  Seq Scan on plt1_p3 t2_2
+                     ->  Seq Scan on plt1_p4 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- N-Way joins
+-- t1 join t2 qualify partition-wise-join and t2 join t3 qualify partition-wise-join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c) LEFT JOIN plt2 t3 ON t2.c = t3.c WHERE t1.b + t2.b = 0 AND t2.b + coalesce(t3.b,0) = 0 ORDER BY t1.a,t2.b;
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a, t2_3.b
+   ->  Result
+         ->  Append
+               ->  Hash Right Join
+                     Hash Cond: ((t3_3.c)::text = (t2_3.c)::text)
+                     Filter: ((t2_3.b + COALESCE(t3_3.b, 0)) = 0)
+                     ->  Seq Scan on plt2_p4 t3_3
+                     ->  Hash
+                           ->  Hash Join
+                                 Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+                                 Join Filter: ((t1_3.b + t2_3.b) = 0)
+                                 ->  Seq Scan on plt1_p4 t1_3
+                                 ->  Hash
+                                       ->  Seq Scan on plt2_p4 t2_3
+               ->  Hash Join
+                     Hash Cond: ((t1.c)::text = (t2.c)::text)
+                     Join Filter: ((t1.b + t2.b) = 0)
+                     ->  Seq Scan on plt1_p1 t1
+                     ->  Hash
+                           ->  Hash Left Join
+                                 Hash Cond: ((t2.c)::text = (t3.c)::text)
+                                 Filter: ((t2.b + COALESCE(t3.b, 0)) = 0)
+                                 ->  Seq Scan on plt2_p1 t2
+                                 ->  Hash
+                                       ->  Seq Scan on plt2_p1 t3
+               ->  Hash Right Join
+                     Hash Cond: ((t3_1.c)::text = (t2_1.c)::text)
+                     Filter: ((t2_1.b + COALESCE(t3_1.b, 0)) = 0)
+                     ->  Seq Scan on plt2_p2 t3_1
+                     ->  Hash
+                           ->  Hash Join
+                                 Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+                                 Join Filter: ((t1_1.b + t2_1.b) = 0)
+                                 ->  Seq Scan on plt1_p2 t1_1
+                                 ->  Hash
+                                       ->  Seq Scan on plt2_p2 t2_1
+               ->  Hash Right Join
+                     Hash Cond: ((t3_2.c)::text = (t2_2.c)::text)
+                     Filter: ((t2_2.b + COALESCE(t3_2.b, 0)) = 0)
+                     ->  Seq Scan on plt2_p3 t3_2
+                     ->  Hash
+                           ->  Hash Join
+                                 Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+                                 Join Filter: ((t1_2.b + t2_2.b) = 0)
+                                 ->  Seq Scan on plt1_p3 t1_2
+                                 ->  Hash
+                                       ->  Seq Scan on plt2_p3 t2_2
+(48 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c) LEFT JOIN plt2 t3 ON t2.c = t3.c WHERE t1.b + t2.b = 0 AND t2.b + coalesce(t3.b,0) = 0 ORDER BY t1.a,t2.b;
+  a  |  c   |  a  |  c   |  a  |  c   
+-----+------+-----+------+-----+------
+   0 | 0000 |   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- t1 join t2 qualify partition-wise-join and t2 join t3 not qualify partition-wise-join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c) RIGHT JOIN plt2 t3 ON (t2.c = t3.c) WHERE t1.b + coalesce(t2.b,0) = 0 AND coalesce(t2.b,0) + t3.b = 0 ORDER BY t1.a, t2.b;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a, t2_3.b
+   ->  Result
+         ->  Append
+               ->  Hash Join
+                     Hash Cond: ((t3_3.c)::text = (t1_3.c)::text)
+                     Join Filter: ((COALESCE(t2_3.b, 0) + t3_3.b) = 0)
+                     ->  Seq Scan on plt2_p4 t3_3
+                     ->  Hash
+                           ->  Hash Join
+                                 Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+                                 Join Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+                                 ->  Seq Scan on plt1_p4 t1_3
+                                 ->  Hash
+                                       ->  Seq Scan on plt2_p4 t2_3
+               ->  Hash Join
+                     Hash Cond: ((t3.c)::text = (t1.c)::text)
+                     Join Filter: ((COALESCE(t2.b, 0) + t3.b) = 0)
+                     ->  Seq Scan on plt2_p1 t3
+                     ->  Hash
+                           ->  Hash Join
+                                 Hash Cond: ((t2.c)::text = (t1.c)::text)
+                                 Join Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+                                 ->  Seq Scan on plt2_p1 t2
+                                 ->  Hash
+                                       ->  Seq Scan on plt1_p1 t1
+               ->  Hash Join
+                     Hash Cond: ((t3_1.c)::text = (t1_1.c)::text)
+                     Join Filter: ((COALESCE(t2_1.b, 0) + t3_1.b) = 0)
+                     ->  Seq Scan on plt2_p2 t3_1
+                     ->  Hash
+                           ->  Hash Join
+                                 Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+                                 Join Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+                                 ->  Seq Scan on plt1_p2 t1_1
+                                 ->  Hash
+                                       ->  Seq Scan on plt2_p2 t2_1
+               ->  Hash Join
+                     Hash Cond: ((t3_2.c)::text = (t1_2.c)::text)
+                     Join Filter: ((COALESCE(t2_2.b, 0) + t3_2.b) = 0)
+                     ->  Seq Scan on plt2_p3 t3_2
+                     ->  Hash
+                           ->  Hash Join
+                                 Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+                                 Join Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+                                 ->  Seq Scan on plt1_p3 t1_2
+                                 ->  Hash
+                                       ->  Seq Scan on plt2_p3 t2_2
+(48 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c) RIGHT JOIN plt2 t3 ON (t2.c = t3.c) WHERE t1.b + coalesce(t2.b,0) = 0 AND coalesce(t2.b,0) + t3.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  a  |  c   |  a  |  c   
+-----+------+-----+------+-----+------
+   0 | 0000 |   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- t1 join t2 not qualify partition-wise-join and t2 join t3 qualify partition-wise-join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c) INNER JOIN plt2 t3 ON (t2.c = t3.c) WHERE coalesce(t1.b, 0) + t2.b = 0 AND t2.b + t3.b = 0 ORDER BY t1.a, t2.b;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Hash Join
+         Hash Cond: ((t3.c)::text = (t2.c)::text)
+         Join Filter: ((t2.b + t3.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p1 t3
+               ->  Seq Scan on plt2_p2 t3_1
+               ->  Seq Scan on plt2_p3 t3_2
+               ->  Seq Scan on plt2_p4 t3_3
+               ->  Seq Scan on plt2_p5 t3_4
+         ->  Hash
+               ->  Hash Right Join
+                     Hash Cond: ((t1.c)::text = (t2.c)::text)
+                     Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+                     ->  Append
+                           ->  Seq Scan on plt1_p1 t1
+                           ->  Seq Scan on plt1_p2 t1_1
+                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Seq Scan on plt1_p4 t1_3
+                     ->  Hash
+                           ->  Append
+                                 ->  Seq Scan on plt2_p1 t2
+                                 ->  Seq Scan on plt2_p2 t2_1
+                                 ->  Seq Scan on plt2_p3 t2_2
+                                 ->  Seq Scan on plt2_p4 t2_3
+                                 ->  Seq Scan on plt2_p5 t2_4
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c) INNER JOIN plt2 t3 ON (t2.c = t3.c) WHERE coalesce(t1.b, 0) + t2.b = 0 AND t2.b + t3.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  a  |  c   |  a  |  c   
+-----+------+-----+------+-----+------
+   0 | 0000 |   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002 | 376 | 0002
+     |      | 470 | 0011 | 470 | 0011
+     |      |  47 | 0013 |  47 | 0013
+     |      | 235 | 0014 | 235 | 0014
+(8 rows)
+
+-- t1 join t2 not qualify partition-wise-join and t2 join t3 not qualify partition-wise-join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c) FULL JOIN plt1 t3 ON (t2.c = t3.c) WHERE coalesce(t1.b, 0) + t2.b = 0 AND coalesce(t2.b,0) + coalesce(t3.b,0) = 0 ORDER BY t1.a, t2.b;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Hash Right Join
+         Hash Cond: ((t3.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t2.b, 0) + COALESCE(t3.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p1 t3
+               ->  Seq Scan on plt1_p2 t3_1
+               ->  Seq Scan on plt1_p3 t3_2
+               ->  Seq Scan on plt1_p4 t3_3
+         ->  Hash
+               ->  Hash Right Join
+                     Hash Cond: ((t1.c)::text = (t2.c)::text)
+                     Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+                     ->  Append
+                           ->  Seq Scan on plt1_p1 t1
+                           ->  Seq Scan on plt1_p2 t1_1
+                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Seq Scan on plt1_p4 t1_3
+                     ->  Hash
+                           ->  Append
+                                 ->  Seq Scan on plt2_p1 t2
+                                 ->  Seq Scan on plt2_p2 t2_1
+                                 ->  Seq Scan on plt2_p3 t2_2
+                                 ->  Seq Scan on plt2_p4 t2_3
+                                 ->  Seq Scan on plt2_p5 t2_4
+(26 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p1 t1
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Seq Scan on plt1_p3 t1_2
+               ->  Seq Scan on plt1_p4 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p1 t2
+                     ->  Seq Scan on plt2_p2 t2_1
+                     ->  Seq Scan on plt2_p3 t2_2
+                     ->  Seq Scan on plt2_p4 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p1 t1
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Seq Scan on plt1_p3 t1_2
+               ->  Seq Scan on plt1_p4 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p1 t2
+                     ->  Seq Scan on plt2_p2 t2_1
+                     ->  Seq Scan on plt2_p3 t2_2
+                     ->  Seq Scan on plt2_p4 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p1 t1
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Seq Scan on plt1_p3 t1_2
+               ->  Seq Scan on plt1_p4 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p1 t2
+                     ->  Seq Scan on plt2_p2 t2_1
+                     ->  Seq Scan on plt2_p3 t2_2
+                     ->  Seq Scan on plt2_p4 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p1 t1
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Seq Scan on plt1_p3 t1_2
+               ->  Seq Scan on plt1_p4 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p1 t2
+                     ->  Seq Scan on plt2_p2 t2_1
+                     ->  Seq Scan on plt2_p3 t2_2
+                     ->  Seq Scan on plt2_p4 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p1 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p4 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p1 t2
+                                 ->  Seq Scan on plt2_p2 t2_1
+                                 ->  Seq Scan on plt2_p3 t2_2
+                                 ->  Seq Scan on plt2_p4 t2_3
+                                 ->  Seq Scan on plt2_p5 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p1 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p1 t2
+                                 ->  Seq Scan on plt1_p2 t2_1
+                                 ->  Seq Scan on plt1_p3 t2_2
+                                 ->  Seq Scan on plt1_p4 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p1 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p4 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p1 t2
+                     ->  Seq Scan on plt2_p2 t2_1
+                     ->  Seq Scan on plt2_p3 t2_2
+                     ->  Seq Scan on plt2_p4 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p1 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p1 t2
+                     ->  Seq Scan on plt1_p2 t2_1
+                     ->  Seq Scan on plt1_p3 t2_2
+                     ->  Seq Scan on plt1_p4 t2_3
+(21 rows)
+
+-- N-Way joins
+-- t1 join t2 not qualify partition-wise-join and t2 join t3 qualify partition-wise-join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c) INNER JOIN plt1 t3 ON (t2.c = t3.c) WHERE t1.b + t2.b = 0 AND t2.b + t3.b = 0 ORDER BY t1.a, t2.b;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p1 t1
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Seq Scan on plt1_p3 t1_2
+               ->  Seq Scan on plt1_p4 t1_3
+         ->  Hash
+               ->  Hash Join
+                     Hash Cond: ((t3.c)::text = (t2.c)::text)
+                     Join Filter: ((t2.b + t3.b) = 0)
+                     ->  Append
+                           ->  Seq Scan on plt1_p1 t3
+                           ->  Seq Scan on plt1_p2 t3_1
+                           ->  Seq Scan on plt1_p3 t3_2
+                           ->  Seq Scan on plt1_p4 t3_3
+                     ->  Hash
+                           ->  Append
+                                 ->  Seq Scan on plt2_p1 t2
+                                 ->  Seq Scan on plt2_p2 t2_1
+                                 ->  Seq Scan on plt2_p3 t2_2
+                                 ->  Seq Scan on plt2_p4 t2_3
+                                 ->  Seq Scan on plt2_p5 t2_4
+(26 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c) INNER JOIN plt1 t3 ON (t2.c = t3.c) WHERE t1.b + t2.b = 0 AND t2.b + t3.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  a  |  c   |  a  |  c   
+-----+------+-----+------+-----+------
+   0 | 0000 |   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005 | 141 | 0005
+ 188 | 0001 | 188 | 0001 | 188 | 0001
+ 329 | 0006 | 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002 | 376 | 0002
+(6 rows)
+
+-- t1 join t2 not qualify partition-wise-join and t2 join t3 not qualify partition-wise-join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c) RIGHT JOIN plt2 t3 ON (t2.c = t3.c) WHERE t1.b + coalesce(t2.b, 0) = 0 AND coalesce(t2.b,0) + t3.b = 0 ORDER BY t1.a, t2.b;
+                            QUERY PLAN                             
+-------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Hash Join
+         Hash Cond: ((t3.c)::text = (t1.c)::text)
+         Join Filter: ((COALESCE(t2.b, 0) + t3.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p1 t3
+               ->  Seq Scan on plt2_p2 t3_1
+               ->  Seq Scan on plt2_p3 t3_2
+               ->  Seq Scan on plt2_p4 t3_3
+               ->  Seq Scan on plt2_p5 t3_4
+         ->  Hash
+               ->  Hash Join
+                     Hash Cond: ((t1.c)::text = (t2.c)::text)
+                     Join Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+                     ->  Append
+                           ->  Seq Scan on plt1_p1 t1
+                           ->  Seq Scan on plt1_p2 t1_1
+                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Seq Scan on plt1_p4 t1_3
+                     ->  Hash
+                           ->  Append
+                                 ->  Seq Scan on plt2_p1 t2
+                                 ->  Seq Scan on plt2_p2 t2_1
+                                 ->  Seq Scan on plt2_p3 t2_2
+                                 ->  Seq Scan on plt2_p4 t2_3
+                                 ->  Seq Scan on plt2_p5 t2_4
+(27 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed 
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_3.c)::text = (t1_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt2_p4 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1_3
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1
+         ->  Hash Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_2
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.c)::text = (t1_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1_3
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1
+         ->  Hash Left Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_2
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Sort Key: t2_3.a
+   ->  Result
+         ->  Append
+               ->  Hash Left Join
+                     Hash Cond: ((t2_3.c)::text = (t1_3.c)::text)
+                     Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+                     ->  Seq Scan on plt2_p4 t2_3
+                     ->  Hash
+                           ->  Seq Scan on plt1_p4 t1_3
+               ->  Hash Left Join
+                     Hash Cond: ((t2.c)::text = (t1.c)::text)
+                     Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+                     ->  Seq Scan on plt2_p1 t2
+                     ->  Hash
+                           ->  Seq Scan on plt1_p1 t1
+               ->  Hash Left Join
+                     Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+                     Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+                     ->  Seq Scan on plt2_p2 t2_1
+                     ->  Hash
+                           ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash Right Join
+                     Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+                     Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+                     ->  Seq Scan on plt1_p3 t1_2
+                     ->  Hash
+                           ->  Seq Scan on plt2_p3 t2_2
+(28 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a, t2_3.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2_3.c)::text = (t1_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1_3
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_2
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a, t1_3.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_3.c)::text
+                     ->  Seq Scan on plt2_p4 t2_3
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1_3
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p1 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p2 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p3 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_2
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a, t1_3.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_3.c)::text
+                     ->  Seq Scan on plt1_p4 t2_3
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1_3
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p1 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p2 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p3 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_2
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a, t1_3.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p4 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2_3
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p1 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p2 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p3 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_2
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1_3.a, t1_3.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p4 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2_3
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p1 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p2 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p3 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_2
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed 
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side 
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p1 t2
+               ->  Seq Scan on plt2_p2 t2_1
+               ->  Seq Scan on plt2_p3 t2_2
+               ->  Seq Scan on plt2_p4 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p1 t1
+                     ->  Seq Scan on plt1_p2 t1_1
+                     ->  Seq Scan on plt1_p4 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p1 t2
+               ->  Seq Scan on plt2_p2 t2_1
+               ->  Seq Scan on plt2_p3 t2_2
+               ->  Seq Scan on plt2_p4 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p1 t1
+                     ->  Seq Scan on plt1_p2 t1_1
+                     ->  Seq Scan on plt1_p4 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p1 t2
+               ->  Seq Scan on plt2_p2 t2_1
+               ->  Seq Scan on plt2_p3 t2_2
+               ->  Seq Scan on plt2_p4 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p1 t1
+                     ->  Seq Scan on plt1_p2 t1_1
+                     ->  Seq Scan on plt1_p4 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p1 t2
+               ->  Seq Scan on plt2_p2 t2_1
+               ->  Seq Scan on plt2_p3 t2_2
+               ->  Seq Scan on plt2_p4 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p1 t1
+                     ->  Seq Scan on plt1_p2 t1_1
+                     ->  Seq Scan on plt1_p4 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p1 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p4 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p1 t2
+                                 ->  Seq Scan on plt2_p2 t2_1
+                                 ->  Seq Scan on plt2_p3 t2_2
+                                 ->  Seq Scan on plt2_p4 t2_3
+(22 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p1 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p1 t2
+                                 ->  Seq Scan on plt1_p2 t2_1
+                                 ->  Seq Scan on plt1_p4 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p1 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p4 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p1 t2
+                     ->  Seq Scan on plt2_p2 t2_1
+                     ->  Seq Scan on plt2_p3 t2_2
+                     ->  Seq Scan on plt2_p4 t2_3
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p1 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p1 t2
+                     ->  Seq Scan on plt1_p2 t2_1
+                     ->  Seq Scan on plt1_p4 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1286,9 +4268,9 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p2 t2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
                ->  Seq Scan on prt2_p3 t2_2
                      Filter: (a = 0)
@@ -1306,9 +4288,9 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p2 t2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
                ->  Seq Scan on prt2_p3 t2_2
                      Filter: (a = 0)
@@ -1341,7 +4323,7 @@ CREATE TABLE prt2_l_p3_p2 PARTITION OF prt2_l_p3 FOR VALUES FROM (13) TO (25);
 INSERT INTO prt2_l SELECT i % 25, i, to_char(i % 4, 'FM0000') FROM generate_series(0, 599, 3) i;
 ANALYZE prt2_l;
 -- inner join, qual covering only top-level partitions
-EXPLAIN(COSTS OFF)
+EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1, prt2_l t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
                          QUERY PLAN                          
 -------------------------------------------------------------
@@ -1386,7 +4368,7 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1, prt2_l t2 WHERE t1.a = t2.b AND t1
 (4 rows)
 
 -- left join
-EXPLAIN(COSTS OFF)
+EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1 LEFT JOIN prt2_l t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t1.b = 0 ORDER BY t1.a, t2.b;
                                      QUERY PLAN                                     
 ------------------------------------------------------------------------------------
@@ -1440,7 +4422,7 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1 LEFT JOIN prt2_l t2 ON t1.a = t2.b
 (12 rows)
 
 -- right join
-EXPLAIN(COSTS OFF)
+EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1 RIGHT JOIN prt2_l t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.a = 0 ORDER BY t1.a, t2.b;
                                         QUERY PLAN                                        
 ------------------------------------------------------------------------------------------
@@ -1491,7 +4473,7 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1 RIGHT JOIN prt2_l t2 ON t1.a = t2.b
 (8 rows)
 
 -- full join
-EXPLAIN(COSTS OFF)
+EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.b = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.a = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
                                                      QUERY PLAN                                                     
 --------------------------------------------------------------------------------------------------------------------
@@ -1552,7 +4534,7 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.b = 0) t1
 (16 rows)
 
 -- lateral partition-wise join
-EXPLAIN(COSTS OFF)
+EXPLAIN (COSTS OFF)
 SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.b AS t3b, least(t1.a,t2.a,t3.b) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.c = t3.c)) ss
 			  ON t1.a = ss.t2a AND t1.c = ss.t2c WHERE t1.b = 0 ORDER BY t1.a;
@@ -1626,7 +4608,7 @@ SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
 (12 rows)
 
 -- join with one side empty
-EXPLAIN(COSTS OFF)
+EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a = 1 AND a = 2) t1 RIGHT JOIN prt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c;
                                QUERY PLAN                                
 -------------------------------------------------------------------------
@@ -1699,15 +4681,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a
    ->  Hash
          ->  Append
                ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
+                     Hash Cond: (t1.a = t3_1.b)
                      ->  Seq Scan on prt1_p1 t1
                      ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
+                           ->  Seq Scan on prt2_p1 t3_1
                ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_1.b)
+                     Hash Cond: (t1_2.a = t3.b)
                      ->  Seq Scan on prt1_p2 t1_2
                      ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
+                           ->  Seq Scan on prt2_p2 t3
                ->  Hash Join
                      Hash Cond: (t1_1.a = t3_2.b)
                      ->  Seq Scan on prt1_p3 t1_1
@@ -1719,21 +4701,20 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
          ->  Seq Scan on prt1_p1 t1
          ->  Seq Scan on prt1_p3 t1_1
          ->  Seq Scan on prt1_p2 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (t1.a < b)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (t1.a < b)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (t1.a < b)
-(12 rows)
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p2 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p3 t2_2
+(11 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partition-wise join
@@ -1819,16 +4800,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
                      ->  Seq Scan on plt1_p1 t3
                      ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partition-wise join can not be applied for a join between list and range
 -- partitioned table
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index cd54ea0..7aa53c0 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -7,6 +7,10 @@
 SET enable_partition_wise_join to true;
 
 --
+-- tests for range partitioned tables.
+--
+
+--
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
@@ -19,16 +23,22 @@ CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting 0-50 range and missing ending 550-600
+-- range bounds
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
-CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
+CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (50) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
-CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (550);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0+50, 599-50, 3) i;
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap 
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -77,11 +87,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.a;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a);
+SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a);
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -126,9 +144,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
 --
 -- N-way join
 --
+--TODO: remove below comments and enable partition_wise_join
+--one issue got fixed
+--Server crashed with below queries
+--setting partition-wise-join to off
+SET enable_partition_wise_join to false;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
+SET enable_partition_wise_join to true;
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
@@ -168,6 +192,156 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 699, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+--TODO: remove below comments and enable partition_wise_join
+--one issue got fixed
+--Getting wrong output with partition wise join
+--setting partition-wise-join to off to get correct output
+SET enable_partition_wise_join to false;
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+SET enable_partition_wise_join to true;
+
+-- N-Way joins
+--TODO: remove below comments and enable partition_wise_join
+--one issue got fixed
+--Server crashed with below queries
+--setting partition-wise-join to off
+SET enable_partition_wise_join to false;
+-- t1 join t2 qualify partition-wise-join and t2 join t3 qualify partition-wise-join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt2 t3 ON t2.b = t3.b WHERE t1.b = 0 ORDER BY 1,2,3,4,5,6;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt2 t3 ON t2.b = t3.b WHERE t1.b = 0 ORDER BY 1,2,3,4,5,6;
+
+-- t1 join t2 qualify partition-wise-join and t2 join t3 not qualify partition-wise-join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt2 t3 ON t2.b = t3.b WHERE t1.b = 0 ORDER BY 1,2,3,4,5,6;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt2 t3 ON t2.b = t3.b WHERE t1.b = 0 ORDER BY 1,2,3,4,5,6;
+
+-- t1 join t2 not qualify partition-wise-join and t2 join t3 qualify partition-wise-join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) INNER JOIN prt2 t3 ON t2.b = t3.b WHERE t1.b = 0 ORDER BY 1,2,3,4,5,6;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) INNER JOIN prt2 t3 ON t2.b = t3.b WHERE t1.b = 0 ORDER BY 1,2,3,4,5,6;
+
+-- t1 join t2 not qualify partition-wise-join and t2 join t3 not qualify partition-wise-join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) FULL JOIN prt2 t3 ON t2.b = t3.b WHERE t1.b = 0 ORDER BY 1,2,3,4,5,6;
+SET enable_partition_wise_join to true;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (550) TO (700);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(550, 699, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- N-Way joins
+-- t1 join t2 not qualify partition-wise-join and t2 join t3 qualify partition-wise-join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b) INNER JOIN prt1 t3 ON t2.b = t3.a WHERE t1.b = 0 ORDER BY 1,2,3,4,5,6;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b) INNER JOIN prt1 t3 ON t2.b = t3.a WHERE t1.b = 0 ORDER BY 1,2,3,4,5,6;
+-- t1 join t2 not qualify partition-wise-join and t2 join t3 not qualify partition-wise-join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt2 t3 ON t2.b = t3.b WHERE t1.b = 0 ORDER BY 1,2,3,4,5,6;
+
+-- Partition-wise join with minvalue/maxvalue bounds
+DROP TABLE prt2_p1;
+DROP TABLE prt2_p3;
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (MINVALUE) TO (250);
+CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 250, 3) i;
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(500, 600, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -192,35 +366,311 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009'); 
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap 
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- N-Way joins
+-- inner-inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1 t1,plt2 t2,plt1 t3 WHERE t1.c = t2.c AND t2.c = t3.c AND t1.b + t2.b = 0 AND t2.b + t3.b = 0 ORDER BY t1.a,t2.b;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1 t1,plt2 t2,plt1 t3 WHERE t1.c = t2.c AND t2.c = t3.c AND t1.b + t2.b = 0 AND t2.b + t3.b = 0 ORDER BY t1.a,t2.b;
+
+-- left-left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c LEFT JOIN plt2 t3 ON (t2.c = t3.c) WHERE t1.b + coalesce(t2.b, 0) = 0 AND t2.b + coalesce(t3.b, 0) = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c LEFT JOIN plt2 t3 ON (t2.c = t3.c) WHERE t1.b + coalesce(t2.b, 0) = 0 AND t2.b + coalesce(t3.b, 0) = 0 ORDER BY t1.a, t2.b;
+
+-- left-right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c RIGHT JOIN plt1 t3 ON (t2.c = t3.c) WHERE t1.b + coalesce(t2.b,0) = 0 AND coalesce(t1.b, 0) + t3.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c RIGHT JOIN plt1 t3 ON (t2.c = t3.c) WHERE t1.b + coalesce(t2.b,0) = 0 AND coalesce(t1.b, 0) + t3.b = 0 ORDER BY t1.a, t2.b;
+
+-- right-full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c FULL JOIN plt1 t3 ON (t2.c = t3.c) WHERE coalesce(t1.b,0) + t2.b = 0 AND coalesce(t2.b,0) + coalesce(t3.b,0) = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c FULL JOIN plt1 t3 ON (t2.c = t3.c) WHERE coalesce(t1.b,0) + t2.b = 0 AND coalesce(t2.b,0) + coalesce(t3.b,0) = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT t3.c FROM plt1 t3 WHERE t3.b = 0)) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT t3.c FROM plt1 t3 WHERE t3.b = 0)) AND t1.b = 0 ORDER BY t1.a;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
--- test partition matching with N-way join
+-- test partition matching with N-way join with expression
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join 
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join 
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- N-Way joins
+-- t1 join t2 qualify partition-wise-join and t2 join t3 qualify partition-wise-join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c) LEFT JOIN plt2 t3 ON t2.c = t3.c WHERE t1.b + t2.b = 0 AND t2.b + coalesce(t3.b,0) = 0 ORDER BY t1.a,t2.b;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c) LEFT JOIN plt2 t3 ON t2.c = t3.c WHERE t1.b + t2.b = 0 AND t2.b + coalesce(t3.b,0) = 0 ORDER BY t1.a,t2.b;
+
+-- t1 join t2 qualify partition-wise-join and t2 join t3 not qualify partition-wise-join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c) RIGHT JOIN plt2 t3 ON (t2.c = t3.c) WHERE t1.b + coalesce(t2.b,0) = 0 AND coalesce(t2.b,0) + t3.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c) RIGHT JOIN plt2 t3 ON (t2.c = t3.c) WHERE t1.b + coalesce(t2.b,0) = 0 AND coalesce(t2.b,0) + t3.b = 0 ORDER BY t1.a, t2.b;
+
+-- t1 join t2 not qualify partition-wise-join and t2 join t3 qualify partition-wise-join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c) INNER JOIN plt2 t3 ON (t2.c = t3.c) WHERE coalesce(t1.b, 0) + t2.b = 0 AND t2.b + t3.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c) INNER JOIN plt2 t3 ON (t2.c = t3.c) WHERE coalesce(t1.b, 0) + t2.b = 0 AND t2.b + t3.b = 0 ORDER BY t1.a, t2.b;
+
+-- t1 join t2 not qualify partition-wise-join and t2 join t3 not qualify partition-wise-join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c) FULL JOIN plt1 t3 ON (t2.c = t3.c) WHERE coalesce(t1.b, 0) + t2.b = 0 AND coalesce(t2.b,0) + coalesce(t3.b,0) = 0 ORDER BY t1.a, t2.b;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- N-Way joins
+-- t1 join t2 not qualify partition-wise-join and t2 join t3 qualify partition-wise-join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c) INNER JOIN plt1 t3 ON (t2.c = t3.c) WHERE t1.b + t2.b = 0 AND t2.b + t3.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c) INNER JOIN plt1 t3 ON (t2.c = t3.c) WHERE t1.b + t2.b = 0 AND t2.b + t3.b = 0 ORDER BY t1.a, t2.b;
+
+-- t1 join t2 not qualify partition-wise-join and t2 join t3 not qualify partition-wise-join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c) RIGHT JOIN plt2 t3 ON (t2.c = t3.c) WHERE t1.b + coalesce(t2.b, 0) = 0 AND coalesce(t2.b,0) + t3.b = 0 ORDER BY t1.a, t2.b;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed 
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed 
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side 
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
@@ -260,27 +710,27 @@ INSERT INTO prt2_l SELECT i % 25, i, to_char(i % 4, 'FM0000') FROM generate_seri
 ANALYZE prt2_l;
 
 -- inner join, qual covering only top-level partitions
-EXPLAIN(COSTS OFF)
+EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1, prt2_l t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1, prt2_l t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
 
 -- left join
-EXPLAIN(COSTS OFF)
+EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1 LEFT JOIN prt2_l t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t1.b = 0 ORDER BY t1.a, t2.b;
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1 LEFT JOIN prt2_l t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t1.b = 0 ORDER BY t1.a, t2.b;
 
 -- right join
-EXPLAIN(COSTS OFF)
+EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1 RIGHT JOIN prt2_l t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.a = 0 ORDER BY t1.a, t2.b;
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1 RIGHT JOIN prt2_l t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.a = 0 ORDER BY t1.a, t2.b;
 
 -- full join
-EXPLAIN(COSTS OFF)
+EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.b = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.a = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.b = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.a = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
 
 -- lateral partition-wise join
-EXPLAIN(COSTS OFF)
+EXPLAIN (COSTS OFF)
 SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.b AS t3b, least(t1.a,t2.a,t3.b) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.c = t3.c)) ss
 			  ON t1.a = ss.t2a AND t1.c = ss.t2c WHERE t1.b = 0 ORDER BY t1.a;
@@ -289,7 +739,7 @@ SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
 			  ON t1.a = ss.t2a AND t1.c = ss.t2c WHERE t1.b = 0 ORDER BY t1.a;
 
 -- join with one side empty
-EXPLAIN(COSTS OFF)
+EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a = 1 AND a = 2) t1 RIGHT JOIN prt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c;
 
 --
#3Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Rajkumar Raghuwanshi (#2)
2 attachment(s)
Re: advanced partition matching algorithm for partition-wise join

PFA the patches rebased on the latest sources. There are also fixes
for some of the crashes and bugs reported. I haven't yet included the
testcase patch in the main patchset.

On Mon, Aug 28, 2017 at 12:44 PM, Rajkumar Raghuwanshi
<rajkumar.raghuwanshi@enterprisedb.com> wrote:

On Mon, Aug 21, 2017 at 12:43 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

TODOs
-----------
1. Add tests for advanced partition matching algorithm

Hi Ashutosh,

I have applied all partition-wise-join patches (v26) and tested feature. I
have modified partition_join.sql file and added extra test cases to test
partition matching.

Attaching WIP test case patch which as of now have some server crashes and a
data corruptions issue which is commented in the file itself and need to be
removed once issue got solved. Also some of queries is not picking or
picking partition-wise-join as per expectation which may need some
adjustment.

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

Attachments:

0011-Modify-bound-comparision-functions-to-accept-members.patchtext/x-patch; charset=US-ASCII; name=0011-Modify-bound-comparision-functions-to-accept-members.patchDownload
From 865242c79b56f021dc619bc028480097d11bb69a Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Thu, 6 Jul 2017 14:15:22 +0530
Subject: [PATCH 11/12] Modify bound comparision functions to accept members
 of PartitionKey

Functions partition_bound_cmp(), partition_rbound_cmp() and
partition_rbound_datum_cmp() are required to merge partition bounds
from joining relations. While doing so, we do not have access to the
PartitionKey of either relations. So, modify these functions to accept
only required members of PartitionKey so that the functions can be
reused for merging bounds.

Ashutosh Bapat.
---
 src/backend/catalog/partition.c |   76 ++++++++++++++++++++++-----------------
 1 file changed, 44 insertions(+), 32 deletions(-)

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 96a64ce..d42e1b5 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -126,15 +126,17 @@ static List *generate_partition_qual(Relation rel);
 
 static PartitionRangeBound *make_one_range_bound(PartitionKey key, int index,
 					 List *datums, bool lower);
-static int32 partition_rbound_cmp(PartitionKey key,
-					 Datum *datums1, PartitionRangeDatumKind *kind1,
-					 bool lower1, PartitionRangeBound *b2);
-static int32 partition_rbound_datum_cmp(PartitionKey key,
-						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
+static int32 partition_rbound_cmp(int partnatts, FmgrInfo *partsupfunc,
+					 Oid *partcollation, Datum *datums1,
+					 PartitionRangeDatumKind *kind1, bool lower1,
+					 PartitionRangeBound *b2);
+static int32 partition_rbound_datum_cmp(int partnatts, FmgrInfo *partsupfunc,
+						   Oid *partcollation, Datum *rb_datums,
+						   PartitionRangeDatumKind *rb_kind,
 						   Datum *tuple_datums);
 
-static int32 partition_bound_cmp(PartitionKey key,
-					PartitionBoundInfo boundinfo,
+static int32 partition_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+					Oid *partcollation, PartitionBoundInfo boundinfo,
 					int offset, void *probe, bool probe_is_bound);
 static int partition_bound_bsearch(PartitionKey key,
 						PartitionBoundInfo boundinfo,
@@ -719,8 +721,9 @@ check_new_partition_bound(char *relname, Relation parent,
 				 * First check if the resulting range would be empty with
 				 * specified lower and upper bounds
 				 */
-				if (partition_rbound_cmp(key, lower->datums, lower->kind, true,
-										 upper) >= 0)
+				if (partition_rbound_cmp(key->partnatts, key->partsupfunc,
+										 key->partcollation, lower->datums,
+										 lower->kind, true, upper) >= 0)
 				{
 					ereport(ERROR,
 							(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
@@ -771,9 +774,11 @@ check_new_partition_bound(char *relname, Relation parent,
 						{
 							int32		cmpval;
 
-							cmpval = partition_bound_cmp(key, boundinfo,
-														 offset + 1, upper,
-														 true);
+							cmpval = partition_bound_cmp(key->partnatts,
+														 key->partsupfunc,
+														 key->partcollation,
+														 boundinfo, offset + 1,
+														 upper, true);
 							if (cmpval < 0)
 							{
 								/*
@@ -2138,7 +2143,9 @@ qsort_partition_rbound_cmp(const void *a, const void *b, void *arg)
 	PartitionRangeBound *b2 = (*(PartitionRangeBound *const *) b);
 	PartitionKey key = (PartitionKey) arg;
 
-	return partition_rbound_cmp(key, b1->datums, b1->kind, b1->lower, b2);
+	return partition_rbound_cmp(key->partnatts, key->partsupfunc,
+								key->partcollation, b1->datums, b1->kind,
+								b1->lower, b2);
 }
 
 /*
@@ -2155,7 +2162,7 @@ qsort_partition_rbound_cmp(const void *a, const void *b, void *arg)
  * two contiguous partitions.
  */
 static int32
-partition_rbound_cmp(PartitionKey key,
+partition_rbound_cmp(int partnatts, FmgrInfo *partsupfunc, Oid *partcollation,
 					 Datum *datums1, PartitionRangeDatumKind *kind1,
 					 bool lower1, PartitionRangeBound *b2)
 {
@@ -2165,7 +2172,7 @@ partition_rbound_cmp(PartitionKey key,
 	PartitionRangeDatumKind *kind2 = b2->kind;
 	bool		lower2 = b2->lower;
 
-	for (i = 0; i < key->partnatts; i++)
+	for (i = 0; i < partnatts; i++)
 	{
 		/*
 		 * First, handle cases where the column is unbounded, which should not
@@ -2186,8 +2193,8 @@ partition_rbound_cmp(PartitionKey key,
 			 */
 			break;
 
-		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[i],
-												 key->partcollation[i],
+		cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[i],
+												 partcollation[i],
 												 datums1[i],
 												 datums2[i]));
 		if (cmpval != 0)
@@ -2213,22 +2220,23 @@ partition_rbound_cmp(PartitionKey key,
  * is <, =, or > partition key of tuple (tuple_datums)
  */
 static int32
-partition_rbound_datum_cmp(PartitionKey key,
-						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
+partition_rbound_datum_cmp(int partnatts, FmgrInfo *partsupfunc,
+						   Oid *partcollation, Datum *rb_datums,
+						   PartitionRangeDatumKind *rb_kind,
 						   Datum *tuple_datums)
 {
 	int			i;
 	int32		cmpval = -1;
 
-	for (i = 0; i < key->partnatts; i++)
+	for (i = 0; i < partnatts; i++)
 	{
 		if (rb_kind[i] == PARTITION_RANGE_DATUM_MINVALUE)
 			return -1;
 		else if (rb_kind[i] == PARTITION_RANGE_DATUM_MAXVALUE)
 			return 1;
 
-		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[i],
-												 key->partcollation[i],
+		cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[i],
+												 partcollation[i],
 												 rb_datums[i],
 												 tuple_datums[i]));
 		if (cmpval != 0)
@@ -2245,17 +2253,18 @@ partition_rbound_datum_cmp(PartitionKey key,
  * specified in *probe.
  */
 static int32
-partition_bound_cmp(PartitionKey key, PartitionBoundInfo boundinfo,
-					int offset, void *probe, bool probe_is_bound)
+partition_bound_cmp(int partnatts, FmgrInfo *partsupfunc, Oid *partcollation,
+					PartitionBoundInfo boundinfo, int offset, void *probe,
+					bool probe_is_bound)
 {
 	Datum	   *bound_datums = boundinfo->datums[offset];
 	int32		cmpval = -1;
 
-	switch (key->strategy)
+	switch (boundinfo->strategy)
 	{
 		case PARTITION_STRATEGY_LIST:
-			cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
-													 key->partcollation[0],
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
 													 bound_datums[0],
 													 *(Datum *) probe));
 			break;
@@ -2273,12 +2282,14 @@ partition_bound_cmp(PartitionKey key, PartitionBoundInfo boundinfo,
 					 */
 					bool		lower = boundinfo->indexes[offset] < 0;
 
-					cmpval = partition_rbound_cmp(key,
-												  bound_datums, kind, lower,
+					cmpval = partition_rbound_cmp(partnatts, partsupfunc,
+												  partcollation, bound_datums,
+												  kind, lower,
 												  (PartitionRangeBound *) probe);
 				}
 				else
-					cmpval = partition_rbound_datum_cmp(key,
+					cmpval = partition_rbound_datum_cmp(partnatts, partsupfunc,
+														partcollation,
 														bound_datums, kind,
 														(Datum *) probe);
 				break;
@@ -2286,7 +2297,7 @@ partition_bound_cmp(PartitionKey key, PartitionBoundInfo boundinfo,
 
 		default:
 			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) key->strategy);
+				 (int) boundinfo->strategy);
 	}
 
 	return cmpval;
@@ -2320,7 +2331,8 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
 		int32		cmpval;
 
 		mid = (lo + hi + 1) / 2;
-		cmpval = partition_bound_cmp(key, boundinfo, mid, probe,
+		cmpval = partition_bound_cmp(key->partnatts, key->partsupfunc,
+									 key->partcollation, boundinfo, mid, probe,
 									 probe_is_bound);
 		if (cmpval <= 0)
 		{
-- 
1.7.9.5

0012-WIP-Partition-wise-join-for-1-1-1-0-0-1-partition-ma.patchtext/x-patch; charset=US-ASCII; name=0012-WIP-Partition-wise-join-for-1-1-1-0-0-1-partition-ma.patchDownload
From 737299aa79cc5d8e9fbf825ef6396696c485031d Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Wed, 9 Aug 2017 12:30:34 +0530
Subject: [PATCH 12/12] WIP Partition-wise join for 1:1, 1:0, 0:1 partition
 matching.

Earlier version of partition-wise join implementation allowed
partition-wise join between two relations with exactly same partition
bounds. This commit allows partition-wise join to be applied under
following conditions

1. the partition bounds of joining relations are such that rows from
given partition on one side join can join with rows from maximum one
partition on the other side i.e. bounds of a given partition on one
side match/overlap with those of maximum one partition on the other
side. If the mapping happens to be m:n where m > 1 or n > 1, we have
to gang multiple partition relations together into a single relation.
This means that we have to add simple relations during join
processing, something which is not supported right now.  ALso, in such
a case, different pairs of joining relations can produce different
partition bounds for the same join relation, which again is not
supported right now.

2. For every partition on outer side that can contribute to the result
of an OUTER side, there exists at least one (taken along with item 1,
it means exactly one)  matching partition on the inner side. To
support partition-wise join when the inner matching partition doesn't
exist, we have to add a dummy base relation corresponding to the
non-existent inner partition. We don't have support add base relations
during join processing.

This commit is not complete yet.

Ashutosh Bapat.
---
 src/backend/catalog/partition.c       | 1255 +++++++++++++++++++++++++++++++++
 src/backend/optimizer/path/joinrels.c |   77 +-
 src/backend/optimizer/util/relnode.c  |   42 +-
 src/include/catalog/partition.h       |    6 +
 4 files changed, 1349 insertions(+), 31 deletions(-)

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index d42e1b5..2d1a905 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -141,6 +141,38 @@ static int32 partition_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
 static int partition_bound_bsearch(PartitionKey key,
 						PartitionBoundInfo boundinfo,
 						void *probe, bool probe_is_bound, bool *is_equal);
+static PartitionBoundInfo partition_range_bounds_merge(int partnatts,
+							 FmgrInfo *supfuncs, Oid *collations,
+							 PartitionBoundInfo boundinfo1, int nparts1,
+							 PartitionBoundInfo boundinfo2, int nparts2,
+							 JoinType jointype, List **parts1, List **parts2);
+static PartitionBoundInfo partition_list_bounds_merge(int partnatts,
+							FmgrInfo *partsupfunc, Oid *collations,
+							PartitionBoundInfo boundinfo1, int nparts1,
+							PartitionBoundInfo boundinfo2, int nparts2,
+							JoinType jointype, List **parts1, List **parts2);
+static void generate_matching_part_pairs(int *mergemap1, int npart1,
+							 int *mergemap2, int nparts2, JoinType jointype,
+							 int nparts, List **parts1, List **parts2);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index);
+static int map_and_merge_partitions(int *partmap1, int *mergemap1, int index1,
+						 int *partmap2, int *mergemap2, int index2,
+						 int *next_index);
+static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *collations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2);
+static int partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
+						   Oid *collations, PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, bool *overlap);
+static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes);
 
 /*
  * RelationBuildPartitionDesc
@@ -2348,3 +2380,1226 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
 
 	return lo;
 }
+
+/*
+ * Merge the given partition bounds.
+ *
+ * If given partition bounds can not be merged, return NULL.
+ *
+ * The function also returns two lists of partition indexes one for each of the
+ * joining relations. Both the lists contain the same number of elements. The
+ * partition indexes at the same positions in the list indicate partitions from
+ * each side to be joined and their position corresponds to the index of
+ * partition to which the results of the child-join belong in the partitioned
+ * join.
+ */
+extern PartitionBoundInfo
+partition_bounds_merge(int partnatts, FmgrInfo *partsupfunc, Oid *partcollation,
+					   PartitionBoundInfo boundinfo1, int nparts1,
+					   PartitionBoundInfo boundinfo2, int nparts2,
+					   JoinType jointype, List **parts1, List **parts2)
+{
+	PartitionBoundInfo merged_bounds;
+	char		strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (boundinfo1->strategy != boundinfo2->strategy)
+		return NULL;
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+	strategy = boundinfo1->strategy;
+	if (strategy == PARTITION_STRATEGY_LIST)
+		merged_bounds = partition_list_bounds_merge(partnatts, partsupfunc,
+													partcollation, boundinfo1,
+													nparts1, boundinfo2,
+													nparts2, jointype, parts1,
+													parts2);
+	else if (strategy == PARTITION_STRATEGY_RANGE)
+		merged_bounds = partition_range_bounds_merge(partnatts, partsupfunc,
+													 partcollation, boundinfo1,
+													 nparts1, boundinfo2,
+													 nparts2, jointype, parts1,
+													 parts2);
+	else
+		elog(ERROR, "unexpected partition strategy: %d", strategy);
+
+	Assert(merged_bounds || (*parts1 == NIL && *parts2 == NIL));
+	return merged_bounds;
+}
+
+/*
+ * partition_get_range_bounds
+ *
+ * Given the index of lower bound in datums array, return lower and upper
+ * bounds and the index of the partition with that lower bound.
+ */
+static int
+partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
+						   PartitionRangeBound *lower,
+						   PartitionRangeBound *upper)
+{
+	int			part_index;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The lower bound should correspond to a valid partition. */
+	part_index = bi->indexes[lb_index + 1];
+	Assert(part_index >= 0);
+
+	lower->kind = bi->kind[lb_index];
+	lower->datums = bi->datums[lb_index];
+	lower->lower = true;
+	upper->kind = bi->kind[lb_index + 1];
+	upper->datums = bi->datums[lb_index + 1];
+	upper->lower = false;
+
+	return part_index;
+}
+
+/*
+ * partition_range_get_next_lb_index
+ *
+ * Given the index of lower bound in datums array return the
+ * index of lower bound of the next partition. When the given index corresponds
+ * to the last partition, return number of datums (ndatums).
+ */
+static int
+partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
+{
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The partition index corresponding to the upper bound should be valid. */
+	Assert(bi->indexes[lb_index + 1] >= 0);
+
+	/*
+	 * If there are no bounds left beyond the upper bound, we have reached the
+	 * last partition.
+	 */
+	if (lb_index + 2 < bi->ndatums)
+	{
+		/*
+		 * If the bound next to the upper bound corresponds to no partition,
+		 * that's the next lower bound of the next partition. Otherwise, the
+		 * current upper bound is the lower bound of the next partition.
+		 */
+		if (bi->indexes[lb_index + 2] < 0)
+			return lb_index + 2;
+		else
+			return lb_index + 1;
+	}
+	else
+		return bi->ndatums;
+}
+
+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc, Oid *collations,
+						  PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2)
+{
+	return partition_rbound_cmp(partnatts, partsupfunc, collations,
+								bound1->datums, bound1->kind, bound1->lower,
+								bound2);
+}
+
+/*
+ * partition_range_cmp
+ *
+ * Compare the bounds of two range partitions and return <, = or > 0, if the
+ * first partition's upper bound is lower than, equal to or higher than the
+ * second partition's upper bound resp.
+ *
+ * Also, set overlaps to true, if the ranges overlap, otherwise set it to
+ * false.
+ */
+static int
+partition_range_cmp(int partnatts, FmgrInfo *supfuncs, Oid *collations,
+						   PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, bool *overlap)
+{
+	/*
+	 * Compare upper bound of the first partition with the lower bound of the
+	 * second and vice-versa. If lower bound is higher than the upper bound,
+	 * the partitions are not overlapping. All other cases indicate overlapping
+	 * partitions.
+	 * TODO: Add a testcase which has lower and upper bound matching exactly.
+	 * Lower bound is inclusive and upper bound is exclusive, so even if the
+	 * datums match, the bounds do not match exactly.
+	 */
+	if (partition_range_bound_cmp(partnatts, supfuncs, collations,
+								  lower_bound1, upper_bound2) > 0)
+	{
+		*overlap = false;
+		return 1;
+	}
+	else if (partition_range_bound_cmp(partnatts, supfuncs, collations,
+									   lower_bound2, upper_bound1) > 0)
+	{
+		*overlap = false;
+		return -1;
+	}
+	else
+	{
+		*overlap = true;
+		return partition_range_bound_cmp(partnatts, supfuncs, collations,
+										 upper_bound1, upper_bound2);
+	}
+}
+
+/*
+ * partition_range_merge
+ *
+ * Merge the partition bounds of given two partitions such that the join
+ * between the given two partitions fits merged bounds.
+ *
+ * "merged_upper" will be set to one of the given upper bounds and
+ * "merged_lower" will be set to one of the given lower bounds.
+ */
+static void
+partition_range_merge(int partnatts, FmgrInfo *supfuncs,
+							 Oid *collations, JoinType jointype,
+							 PartitionRangeBound *left_lb,
+							 PartitionRangeBound *left_ub,
+							 PartitionRangeBound *right_lb,
+							 PartitionRangeBound *right_ub,
+							 PartitionRangeBound **merged_lb,
+							 PartitionRangeBound **merged_ub)
+{
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = left_ub;
+			*merged_lb = left_lb;
+			break;
+
+		case JOIN_RIGHT:
+			*merged_ub = right_ub;
+			*merged_lb = right_lb;
+			break;
+
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			if (partition_range_bound_cmp(partnatts, supfuncs, collations,
+										  left_ub, right_ub) < 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, supfuncs, collations,
+										  left_lb, right_lb) > 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		case JOIN_FULL:
+			if (partition_range_bound_cmp(partnatts, supfuncs, collations,
+										  left_ub, right_ub) > 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, supfuncs, collations,
+										  left_lb, right_lb) < 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		default:
+			elog(ERROR, "Unexpected join type %d", jointype);
+	}
+
+	return;
+}
+
+/*
+ * Add the lower bound of the next range to the list of bounds, if the lower
+ * bound is higher or equal to the previous upper bound. If successful return
+ * true, otherwise false.
+ */
+static bool
+partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		prev_ub.datums = llast(*merged_datums);
+		prev_ub.kind = llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		/*
+		 * TODO: explain why do we pass lower to be false for the next lower
+		 * bound.
+		 */
+		cmpval = partition_rbound_cmp(partnatts, supfuncs, collations,
+									  next_lb_datums, next_lb_kind, false,
+									  &prev_ub);
+	}
+
+	/*
+	 * The lower bound is lower than the last upper bound, thus does not fit
+	 * the bounds created so far and hence can not be merged with the existing
+	 * bounds.
+	 */
+	if (cmpval < 0)
+		return false;
+
+	/*
+	 * Add bounds of the new merged partition. If the next lower bound is
+	 * higher than the last upper bound, add new range with index
+	 * corresponding to the lower bound as -1. If the merged lower bound
+	 * is same as the last merged upper bound, the last upper bound will be
+	 * reused as the lower bound of the next range.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, next_lb_datums);
+		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	return true;
+}
+
+/*
+ * Merge given two range partition bounds.
+ *
+ * Work horse function for partition_bounds_merge() for range partitioned
+ * tables.
+ *
+ * TODO: for an anti-join, the caller is supposed to send the outer relation as
+ * left relation. May be we should rename left and right as inner and outer. We
+ * don't need to handle RIGHT joins in this function, so renaming them as outer
+ * and inner is fine.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(int partnatts, FmgrInfo *supfuncs, Oid *collations,
+							 PartitionBoundInfo left_bi, int left_nparts,
+							 PartitionBoundInfo right_bi, int right_nparts,
+							 JoinType jointype, List **left_parts, List **right_parts)
+{
+	int		   *left_pmap;
+	int		   *left_mmap;
+	int		   *right_pmap;
+	int		   *right_mmap;
+	int			cnt1;
+	int			cnt2;
+	int			left_part;
+	int			right_part;
+	PartitionBoundInfo merged_bounds = NULL;
+	bool		merged = true;	/* By default we ranges are merge-able. */
+	int			left_lb_index;
+	int			right_lb_index;
+	int			next_index;
+	int			cmpval;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	List	   *merged_kinds = NIL;
+
+	Assert(left_bi->strategy == right_bi->strategy &&
+		   left_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	*left_parts = NIL;
+	*right_parts = NIL;
+
+	/* Allocate and initialize partition maps. */
+	left_pmap = (int *) palloc(sizeof(int) * left_nparts);
+	left_mmap = (int *) palloc(sizeof(int) * left_nparts);
+	right_pmap = (int *) palloc(sizeof(int) * right_nparts);
+	right_mmap = (int *) palloc(sizeof(int) * right_nparts);
+
+	for (cnt1 = 0; cnt1 < left_nparts; cnt1++)
+	{
+		left_pmap[cnt1] = -1;
+		left_mmap[cnt1] = -1;
+	}
+	for (cnt2 = 0; cnt2 < right_nparts; cnt2++)
+	{
+		right_pmap[cnt2] = -1;
+		right_mmap[cnt2] = -1;
+	}
+
+	left_lb_index = 0;
+	right_lb_index = 0;
+	next_index = 0;
+	while (left_lb_index < left_bi->ndatums &&
+		   right_lb_index < right_bi->ndatums)
+	{
+		PartitionRangeBound left_lb;
+		PartitionRangeBound left_ub;
+		PartitionRangeBound right_lb;
+		PartitionRangeBound right_ub;
+		PartitionRangeBound *merged_lb = NULL;
+		PartitionRangeBound *merged_ub = NULL;
+		int			merged_index = -1;
+		bool		overlap;
+
+		/* Get the range bounds of the next partition. */
+		left_part = partition_get_range_bounds(left_bi, left_lb_index,
+											   &left_lb, &left_ub);
+		right_part = partition_get_range_bounds(right_bi, right_lb_index,
+												&right_lb, &right_ub);
+
+		cmpval = partition_range_cmp(partnatts, supfuncs, collations,
+									 &left_lb, &left_ub, &right_lb, &right_ub,
+									 &overlap);
+
+		if (overlap)
+		{
+			/* Overlapping ranges, try merging. */
+			partition_range_merge(partnatts, supfuncs, collations, jointype,
+								  &left_lb, &left_ub, &right_lb, &right_ub,
+								  &merged_lb, &merged_ub);
+			merged_index = map_and_merge_partitions(left_pmap, left_mmap,
+													left_part, right_pmap,
+													right_mmap, right_part,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				merged = false;
+				break;
+			}
+		}
+
+		if (cmpval == 0)
+		{
+			Assert(overlap);
+
+			/* Move to the next pair of partitions. */
+			left_lb_index = partition_range_get_next_lb_index(left_bi,
+															  left_lb_index);
+			right_lb_index = partition_range_get_next_lb_index(right_bi,
+															   right_lb_index);
+		}
+		else if (cmpval < 0)
+		{
+			/*
+			 * If the partition on the left was not mapped to any partition on
+			 * the right. Any row from that partition will not have a matching
+			 * row from the other relation. So the partition will be part of
+			 * the join if it's an anti-join or the left side is the outer
+			 * side.
+			 */
+			if (jointype == JOIN_INNER || jointype == JOIN_SEMI ||
+				jointype == JOIN_RIGHT)
+			{
+				/* Nothing to do. */
+			}
+			else if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+					 jointype == JOIN_ANTI)
+			{
+				if (left_mmap[left_part] < 0)
+				{
+					left_mmap[left_part] = next_index++;
+					merged_index = left_mmap[left_part];
+					merged_lb = &left_lb;
+					merged_ub = &left_ub;
+				}
+			}
+			else
+			{
+				/* Don't know what to do with other join types. Bail out. */
+				merged = false;
+			}
+
+			/* Move to the next partition on the left side. */
+			left_lb_index = partition_range_get_next_lb_index(left_bi,
+															  left_lb_index);
+		}
+		else
+		{
+			Assert(cmpval > 0);
+
+			/*
+			 * If the partition on the right was not mapped to any partition on
+			 * the left. Any row from that partition will not have a matching
+			 * row from the other relation. So the partition will be part of
+			 * the join if the right side is the outer side.
+			 */
+			if (jointype == JOIN_INNER || jointype == JOIN_SEMI ||
+				jointype == JOIN_LEFT || jointype == JOIN_ANTI)
+			{
+				/* Nothing to do. */
+			}
+			else if (jointype == JOIN_RIGHT || jointype == JOIN_FULL)
+			{
+				if (right_mmap[right_part] < 0)
+				{
+					right_mmap[right_part] = next_index++;
+					merged_index = right_mmap[right_part];
+					merged_lb = &right_lb;
+					merged_ub = &right_ub;
+				}
+			}
+			else
+			{
+				/* Don't know what to do with other join types. Bail out. */
+				merged = false;
+			}
+
+			/* Move to the next partition on the right side. */
+			right_lb_index = partition_range_get_next_lb_index(right_bi,
+															   right_lb_index);
+		}
+
+		if (!merged)
+			break;
+
+		/* A skipped partition is not added to merged bounds. */
+		if (merged_index < 0)
+			continue;
+
+		/*
+		 * We have a valid partition index for the next partition of join. The
+		 * partition should have valid range.
+		 */
+		Assert(merged_lb && merged_ub);
+
+		/* Try merging merged lower bound with the last upper bound. */
+		merged = partition_range_merge_next_lb(partnatts, supfuncs,
+											   collations, merged_lb->datums,
+											   merged_lb->kind, &merged_datums,
+											   &merged_kinds, &merged_indexes);
+		if (merged)
+		{
+			/* Add upper bound with the merged partition index. */
+			merged_datums = lappend(merged_datums, merged_ub->datums);
+			merged_kinds = lappend(merged_kinds, merged_ub->kind);
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+		}
+		else
+			break;
+	}
+
+	/*
+	 * We will run the above loop till we exhaust ranges of at least one side
+	 * unless we failed to merge the ranges.
+	 */
+	Assert (!merged || (left_lb_index >= left_bi->ndatums ||
+						right_lb_index >= right_bi->ndatums));
+
+	/*
+	 * Handle any remaining partition bounds.  If remaining partitions fall on
+	 * the inner side of the join, none of the rows in those partition are
+	 * going to be joined with any row on the outer side and hence those
+	 * partitions will not be part of the join result. Hence only consider the
+	 * remaining partitions on the outer side of the join.
+	 */
+	if (merged &&
+		((left_lb_index < left_bi->ndatums &&
+		  (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+		   jointype == JOIN_ANTI)) ||
+		 (right_lb_index < right_bi->ndatums &&
+		  (jointype == JOIN_RIGHT || jointype == JOIN_FULL))))
+	{
+		int			bound_index = -1;
+		PartitionBoundInfo rem_bi = NULL;
+		int		   *mmap = NULL;
+		int			part_index;
+
+		if (left_lb_index < left_bi->ndatums)
+		{
+			Assert(jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+				   jointype == JOIN_ANTI);
+			bound_index = left_lb_index;
+			rem_bi = left_bi;
+			mmap = left_mmap;
+			part_index = left_part;
+		}
+		else if (right_lb_index < right_bi->ndatums)
+		{
+			Assert(jointype == JOIN_RIGHT || jointype == JOIN_FULL);
+			bound_index = right_lb_index;
+			rem_bi = right_bi;
+			mmap = right_mmap;
+			part_index = right_part;
+		}
+		Assert((bound_index >= 0 && bound_index < rem_bi->ndatums) &&
+			   rem_bi && mmap && part_index >= 0);
+
+		/*
+		 * If the partition corresponding to this lower bound has been already
+		 * mapped to a merged partition, don't need to add it again. This may
+		 * happen if the range of the last partition on the inner side overlaps
+		 * with this partition's range and has upper bound lesser than upper
+		 * bound of this partition's range.
+		 */
+		if (mmap[part_index] >= 0)
+			bound_index = partition_range_get_next_lb_index(rem_bi, bound_index);
+
+		/*
+		 * Merge lower bound of the next range with the upper bound of last
+		 * range.
+		 */
+		if (bound_index < rem_bi->ndatums)
+			merged = partition_range_merge_next_lb(partnatts, supfuncs,
+												   collations,
+												   rem_bi->datums[bound_index],
+												   rem_bi->kind[bound_index],
+												   &merged_datums,
+												   &merged_kinds,
+												   &merged_indexes);
+
+		/*
+		 * Rest of the bounds correspond to valid ranges so add them after
+		 * remapping their partitions as required.
+		 */
+		for (bound_index++; merged && bound_index < rem_bi->ndatums;
+			 bound_index++)
+		{
+			Datum	   *datums = rem_bi->datums[bound_index];
+			int			index = rem_bi->indexes[bound_index];
+			int			part_index;
+
+			/*
+			 * Add lower bounds with partition index -1 and assign a new
+			 * partition index to the upper bounds.
+			 */
+			if (index < 0)
+				part_index = index;
+			else
+			{
+				if (mmap[index] < 0)
+					mmap[index] = next_index++;
+				part_index = mmap[index];
+			}
+
+			merged_indexes = lappend_int(merged_indexes, part_index);
+			merged_datums = lappend(merged_datums, datums);
+			merged_kinds = lappend(merged_kinds,
+								   rem_bi->kind[bound_index]);
+		}
+	}
+
+	/* Create PartitionBoundInfo */
+	if (merged)
+	{
+		/* Use maps to match partition from the joining relations. */
+		generate_matching_part_pairs(left_mmap, left_nparts, right_mmap,
+									 right_nparts, jointype, next_index,
+									 left_parts, right_parts);
+
+		/* Craft a PartitionBoundInfo to return. */
+		if (*left_parts && *right_parts)
+		{
+			Assert(list_length(*left_parts) == list_length(*right_parts));
+			Assert(list_length(*left_parts) == next_index);
+			merged_bounds = build_merged_partition_bounds(left_bi->strategy,
+														  merged_datums,
+														  merged_indexes,
+														  merged_kinds,
+														  -1);
+		}
+	}
+
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	pfree(left_pmap);
+	pfree(right_pmap);
+	pfree(left_mmap);
+	pfree(right_mmap);
+
+	/* Free any memory we used in this function. */
+	return merged_bounds;
+}
+
+/*
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ * The function builds the maps of matching partitions from either relation. It
+ * builds the list of partition key values that may appear in the join result
+ * alongwith the list of indexes of partitions of join to which those values
+ * belong. It then crafts a PartitionBoundInfo structure representing the
+ * partition bounds of the join result.
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(int partnatts, FmgrInfo *partsupfunc,
+							Oid *partcollation, PartitionBoundInfo left_bi,
+							int left_nparts, PartitionBoundInfo right_bi,
+							int right_nparts, JoinType jointype, List **left_parts,
+							List **right_parts)
+{
+	int		   *left_pmap;	/* left to right partition map */
+	int		   *left_mmap;	/* left to merged partition map */
+	int		   *right_pmap;	/* right to left partition map */
+	int		   *right_mmap;	/* right to merged partition map */
+	int			cntl;
+	int			cntr;
+	bool		merged = true;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			next_index = 0;
+	int			null_index;
+	PartitionBoundInfo merged_bounds = NULL;
+	int		   *left_indexes = left_bi->indexes;
+	int		   *right_indexes = right_bi->indexes;
+	int			left_ni = left_bi->null_index;
+	int			right_ni = right_bi->null_index;
+
+	Assert(left_bi->strategy == right_bi->strategy &&
+		   left_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!left_bi->kind && !right_bi->kind);
+
+	/* Allocate and initialize partition maps. */
+	left_pmap = (int *) palloc(sizeof(int) * left_nparts);
+	left_mmap = (int *) palloc(sizeof(int) * left_nparts);
+	right_pmap = (int *) palloc(sizeof(int) * right_nparts);
+	right_mmap = (int *) palloc(sizeof(int) * right_nparts);
+
+	/* Initialize partition maps. */
+	for (cntl = 0; cntl < left_nparts; cntl++)
+	{
+		left_pmap[cntl] = -1;
+		left_mmap[cntl] = -1;
+	}
+	for (cntr = 0; cntr < right_nparts; cntr++)
+	{
+		right_pmap[cntr] = -1;
+		right_mmap[cntr] = -1;
+	}
+
+	cntl = cntr = 0;
+	while (cntl < left_bi->ndatums && cntr < right_bi->ndatums)
+	{
+		Datum	   *ldatums = left_bi->datums[cntl];
+		Datum	   *rdatums = right_bi->datums[cntr];
+		int			l_index = left_indexes[cntl];
+		int			r_index = right_indexes[cntr];
+		int			cmpval;
+		int			merged_index;
+		Datum	   *merged_datum;
+
+		/* Every list datum should map to a valid partition index. */
+		Assert(l_index >= 0 && r_index >= 0);
+
+		cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+												 partcollation[0], ldatums[0],
+												 rdatums[0]));
+		if (cmpval == 0)
+		{
+			/*
+			 * Try matching partitions containing the matching datums. If
+			 * successful, add the datum to the merged bounds with index of
+			 * merged partition containing it.
+			 */
+			merged_datum = ldatums;
+			merged_index = map_and_merge_partitions(left_pmap, left_mmap, l_index,
+													right_pmap, right_mmap, r_index,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				merged = false;
+				break;
+			}
+
+			/* Move to the next pair of bounds. */
+			cntl++;
+			cntr++;
+		}
+		else if (cmpval < 0)
+		{
+			/*
+			 * This list datum is present in the left side but not the right
+			 * side. So it will appear in the join when the left side is outer
+			 * side.
+			 */
+			if (jointype == JOIN_INNER || jointype == JOIN_RIGHT ||
+				jointype == JOIN_SEMI)
+				merged_index = -1;
+			else if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+					 jointype == JOIN_ANTI)
+			{
+				if (left_mmap[l_index] < 0)
+					left_mmap[l_index] = next_index++;
+				merged_index = left_mmap[l_index];
+				merged_datum = ldatums;
+			}
+			else
+			{
+				/* Don't know what to do with other join types. */
+				merged = false;
+				break;
+			}
+
+			/* Move to the next datum on the left side. */
+			cntl++;
+		}
+		else
+		{
+			Assert(cmpval > 0);
+
+			/*
+			 * This list datum is present in the right side but not the left
+			 * side. So it will appear in the join when the right side is outer
+			 * side.
+			 */
+			if (jointype == JOIN_INNER || jointype == JOIN_LEFT ||
+				jointype == JOIN_SEMI || jointype == JOIN_ANTI)
+				merged_index = -1;
+			else if (jointype == JOIN_RIGHT || jointype == JOIN_FULL)
+			{
+				/*
+				 * Every list value on the outer side will appear in the
+				 * join.  Find the merged partition to which this value
+				 * belongs.
+				 */
+				if (right_mmap[r_index] < 0)
+					right_mmap[r_index] = next_index++;
+				merged_index = right_mmap[r_index];
+				merged_datum = rdatums;
+			}
+			else
+			{
+				/* Don't know what to do with other join types. */
+				merged = false;
+				break;
+			}
+
+			/* Move to the next datum on the right side. */
+			cntr++;
+		}
+
+		/*
+		 * Add the datum with appropriate index in the list of datums, if the
+		 * rows containing that datum are deemed to be part of the join.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	/*
+	 * If merge is unsuccessful, bail out without any further processing.
+	 * That leaks the memory allocated in this function. So, try not to leak
+	 * memory.
+	 */
+	if (!merged)
+		goto merge_failed;
+
+	/* We should have exhausted datums on at least one side. */
+	Assert(cntr >= right_bi->ndatums || cntl >= left_bi->ndatums);
+
+	/*
+	 * Add any remaining list values on the outer side, assigning partition
+	 * indexes if required.
+	 */
+	if (jointype == JOIN_LEFT || jointype == JOIN_FULL || jointype == JOIN_ANTI)
+	{
+		for (;cntl < left_bi->ndatums; cntl++)
+		{
+			Datum	   *ldatums = left_bi->datums[cntl];
+			int			l_index = left_indexes[cntl];
+
+			if (left_mmap[l_index] < 0)
+				left_mmap[l_index] = next_index++;
+			merged_indexes = lappend_int(merged_indexes, left_mmap[l_index]);
+			merged_datums = lappend(merged_datums, ldatums);
+		}
+	}
+
+	if (jointype == JOIN_RIGHT || jointype == JOIN_FULL)
+	{
+		for (;cntr < right_bi->ndatums; cntr++)
+		{
+			Datum	   *rdatums = right_bi->datums[cntr];
+			int			r_index = right_indexes[cntr];
+
+			if (right_mmap[r_index] < 0)
+				right_mmap[r_index] = next_index++;
+			merged_indexes = lappend_int(merged_indexes, right_mmap[r_index]);
+			merged_datums = lappend(merged_datums, rdatums);
+		}
+	}
+
+	/*
+	 * Merge NULL partitions if any. Find the index of merged partition to
+	 * which the NULL values belong in the join result. We can eliminate a NULL
+	 * partition when it appears only in the inner relation.
+	 */
+	if (!partition_bound_accepts_nulls(left_bi) &&
+		!partition_bound_accepts_nulls(right_bi))
+		null_index = -1;
+	else if (partition_bound_accepts_nulls(left_bi) &&
+			 !partition_bound_accepts_nulls(right_bi))
+	{
+		if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+			jointype == JOIN_ANTI)
+		{
+			if (left_mmap[left_ni] < 0)
+				left_mmap[left_ni] = next_index++;
+			null_index = left_mmap[left_ni];
+		}
+		else
+			null_index = -1;
+	}
+	else if (!partition_bound_accepts_nulls(left_bi) &&
+			 partition_bound_accepts_nulls(right_bi))
+	{
+		if (jointype == JOIN_RIGHT || jointype == JOIN_FULL)
+		{
+			if (right_mmap[right_ni] < 0)
+				right_mmap[right_ni] = next_index++;
+			null_index = right_mmap[right_ni];
+		}
+		else
+			null_index = -1;
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		null_index = map_and_merge_partitions(left_pmap, left_mmap,
+											  left_ni, right_pmap,
+											  right_mmap, right_ni,
+											  &next_index);
+		if (null_index < 0)
+			merged = false;
+	}
+
+	/* If successful build the output structures. */
+	if (merged)
+	{
+
+		/* Use maps to match partition from the joining relations. */
+		generate_matching_part_pairs(left_mmap, left_nparts, right_mmap,
+									 right_nparts, jointype, next_index,
+									 left_parts, right_parts);
+		/* Craft a PartitionBoundInfo to return. */
+		if (*left_parts && *right_parts)
+		{
+			Assert(list_length(*left_parts) == list_length(*right_parts));
+			Assert(list_length(*left_parts) == next_index);
+			merged_bounds = build_merged_partition_bounds(left_bi->strategy,
+														  merged_datums,
+														  merged_indexes, NIL,
+														  null_index);
+		}
+	}
+
+merge_failed:
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	pfree(left_pmap);
+	pfree(right_pmap);
+	pfree(left_mmap);
+	pfree(right_mmap);
+
+	return merged_bounds;
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * If the two given partitions (given by index1 and index2 resp.) are already
+ * mapped to each other return the index of corresponding partition in the
+ * merged set of partitions.  If they do not have a merged partition associated
+ * with them, assign a new merged partition index.  If the partitions are
+ * already mapped and their mapped partitions are different from each other,
+ * they can not be merged, so return -1.
+ *
+ * partmap1[i] gives the partition of relation 2 which matches ith partition of
+ * relation 1. Similarly for partmap2.
+ *
+ * mergemap1[i] gives the partition in the merged set to which ith partition of
+ * relation 1 maps to. Similarly for mergemap2.
+ *
+ * index1 and index2 are the indexes of matching partition from respective
+ * relations.
+ *
+ * *next_index is used and incremented when the given partitions require a new
+ * merged partition.
+ */
+static int
+map_and_merge_partitions(int *partmap1, int *mergemap1, int index1,
+						 int *partmap2, int *mergemap2, int index2,
+						 int *next_index)
+{
+	int			merged_index;
+
+	/*
+	 * If both the partitions are not mapped to each other, update the
+	 * maps.
+	 */
+	if (partmap1[index1] < 0 && partmap2[index2] < 0)
+	{
+		partmap1[index1] = index2;
+		partmap2[index2] = index1;
+	}
+
+	/*
+	 * If the given to partitions map to each other, find the corresponding
+	 * merged partition index .
+	 */
+	if (partmap1[index1] == index2 && partmap2[index2] == index1)
+	{
+		/*
+		 * If both the partitions are mapped to the same merged partition, get
+		 * the index of merged partition.
+		 */
+		if (mergemap1[index1] == mergemap2[index2])
+		{
+			merged_index = mergemap1[index1];
+
+			/*
+			 * If the given two partitions do not have a merged partition
+			 * associated with them, allocate a new merged partition.
+			 */
+			if (merged_index < 0)
+			{
+				merged_index = *next_index;
+				*next_index = *next_index + 1;
+				mergemap1[index1] = merged_index;
+				mergemap2[index2] = merged_index;
+			}
+		}
+
+		/*
+		 * If partition from one relation was mapped to a merged partition but
+		 * not the partition from the other relation, map the same merged
+		 * partition to the partition from other relation, since matching
+		 * partitions map to the same merged partition.
+		 */
+		else if (mergemap1[index1] >= 0 && mergemap2[index2] < 0)
+		{
+			mergemap2[index2] = mergemap1[index1];
+			merged_index = mergemap1[index1];
+		}
+		else if (mergemap1[index1] < 0 && mergemap2[index2] >= 0)
+		{
+			mergemap1[index1] = mergemap2[index2];
+			merged_index = mergemap2[index2];
+		}
+		else
+		{
+			Assert(mergemap1[index1] != mergemap2[index2] &&
+				   mergemap1[index1] >= 0 && mergemap2[index2] >= 0);
+
+			/*
+			 * Both the partitions map to different merged partitions. This
+			 * means that multiple partitions from one relation matches to one
+			 * partition from the other relation. Partition-wise join does not
+			 * handle this case right now, since it requires ganging multiple
+			 * partitions together (into one RelOptInfo).
+			 */
+			merged_index = -1;
+		}
+	}
+	else
+	{
+		/*
+		 * Multiple partitions from one relation map to one partition from the
+		 * other relation. Partition-wise join does not handle this case right
+		 * now, since it requires ganging multiple partitions together (into
+		 * one RelOptInfo).
+		 */
+		merged_index = -1;
+	}
+
+	return merged_index;
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * Given the merged partition to which partition on either side of join map,
+ * produce the list pairs of partitions which when joined produce the merged
+ * partitions in the order of merged partition indexes.
+ *
+ * If successful, the pairs of partitions are returned as two separate lists
+ * one for each side. Otherwise, those lists will be set to NIL.
+ *
+ * TODO: rename the sides as outer and inner. You may not need to support
+ * JOIN_RIGHT, since we won't see that type here.
+ */
+static void
+generate_matching_part_pairs(int *mergemap1, int nparts1, int *mergemap2,
+							 int nparts2, JoinType jointype, int nparts,
+							 List **parts1, List **parts2)
+{
+	bool		merged = true;
+	int		  **matching_parts;
+	int			cnt1;
+	int			cnt2;
+
+	matching_parts = (int **) palloc(sizeof(int *) * 2);
+	matching_parts[0] = (int *) palloc(sizeof(int) * nparts);
+	matching_parts[1] = (int *) palloc(sizeof(int) * nparts);
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt1 = 0; cnt1 < nparts; cnt1++)
+	{
+		matching_parts[0][cnt1] = -1;
+		matching_parts[1][cnt1] = -1;
+	}
+
+	/* Set pairs of matching partitions. */
+	for (cnt1 = 0; cnt1 < nparts1; cnt1++)
+	{
+		if (mergemap1[cnt1] >= 0)
+		{
+			Assert(mergemap1[cnt1] < nparts);
+			matching_parts[0][mergemap1[cnt1]] = cnt1;
+		}
+	}
+	for (cnt2 = 0; cnt2 < nparts2; cnt2++)
+	{
+		if (mergemap2[cnt2] >= 0)
+		{
+			Assert(mergemap2[cnt2] < nparts);
+			matching_parts[1][mergemap2[cnt2]] = cnt2;
+		}
+	}
+
+	/*
+	 * If we have a partition missing on an inner side, we need to add a dummy
+	 * relation which joins with the outer partition. If the inner relation
+	 * happens to be a base relation, it will require adding a dummy child
+	 * base relation during join processing. Right now, we freeze the base
+	 * relation arrays like PlannerInfo::simple_rte_array after planning for
+	 * base relations. Adding a new (dummy) base relation would require some
+	 * changes to that. So, right now, we do not implement partition-wise join
+	 * in such cases.
+	 */
+	for (cnt1 = 0; cnt1 < nparts; cnt1++)
+	{
+		int			part1 = matching_parts[0][cnt1];
+		int			part2 = matching_parts[1][cnt1];
+
+		/* At least one of the partitions should exist. */
+		Assert(part1 >= 0 || part2 >= 0);
+
+		switch (jointype)
+		{
+			case JOIN_INNER:
+			case JOIN_SEMI:
+
+				/*
+				 * An inner or semi join can not return any row when the
+				 * matching partition on either side is missing. We should
+				 * have eliminated all such cases while merging the bounds.
+				 */
+				Assert(part1 >= 0 && part2 >= 0);
+				break;
+
+			case JOIN_LEFT:
+			case JOIN_ANTI:
+				Assert(part1 >= 0);
+				if (part2 < 0)
+					merged = false;
+				break;
+
+			case JOIN_RIGHT:
+				Assert(part2 >= 0);
+				if (part1 < 0)
+					merged = false;
+				break;
+
+			case JOIN_FULL:
+				if (part1 < 0 || part2 < 0)
+					merged = false;
+				break;
+
+			default:
+				/* We do not know what to do in this case. Bail out. */
+				merged = false;
+		}
+
+		if (!merged)
+			break;
+
+		*parts1 = lappend_int(*parts1, part1);
+		*parts2 = lappend_int(*parts2, part2);
+	}
+
+	pfree(matching_parts[0]);
+	pfree(matching_parts[1]);
+	pfree(matching_parts);
+
+	if (!merged)
+	{
+		list_free(*parts1);
+		list_free(*parts2);
+		*parts1 = NIL;
+		*parts2 = NIL;
+	}
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind = (PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+															   list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+
+	return merged_bounds;
+}
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 79468d2..d4e3047 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1308,8 +1308,13 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 						RelOptInfo *joinrel, SpecialJoinInfo *parent_sjinfo,
 						List *parent_restrictlist)
 {
-	int			nparts;
 	int			cnt_parts;
+	PartitionScheme part_scheme;
+	PartitionBoundInfo join_boundinfo;
+	List	   *parts1;
+	List	   *parts2;
+	ListCell   *lc1;
+	ListCell   *lc2;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
@@ -1359,39 +1364,54 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	 */
 	Assert(joinrel->part_scheme == rel1->part_scheme &&
 		   joinrel->part_scheme == rel2->part_scheme);
+	part_scheme = joinrel->part_scheme;
 
 	/*
-	 * Since we allow partition-wise join only when the partition bounds of
-	 * the joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * Get the list of matching partitions from both sides of the join. While
+	 * doing so, we also build the partition bounds of the join relation,
+	 * which should match the bounds calculated for other pairs. TODO: why
+	 * should every pair result in the same partition bounds?
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
-
-	nparts = joinrel->nparts;
-
+	join_boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											part_scheme->partsupfunc,
+											part_scheme->parttypcoll,
+											rel1->boundinfo, rel1->nparts,
+											rel2->boundinfo, rel2->nparts,
+											parent_sjinfo->jointype,
+											&parts1, &parts2);
+
+	Assert(join_boundinfo);
+	Assert(partition_bounds_equal(part_scheme->partnatts,
+								  part_scheme->parttyplen,
+								  part_scheme->parttypbyval, join_boundinfo,
+								  joinrel->boundinfo));
 	elog(DEBUG3, "join between relations %s and %s is considered for partition-wise join.",
 		 bmsToString(rel1->relids), bmsToString(rel2->relids));
 
-	/* Allocate space to hold child-joins RelOptInfos, if not already done. */
+	/*
+	 * Every pair of joining relations should result in the same number of
+	 * child-joins.
+	 */
+	Assert(joinrel->nparts == list_length(parts1));
+	Assert(joinrel->nparts == list_length(parts2));
+
+	/* Allocate space for hold child-joins RelOptInfos, if not already done. */
 	if (!joinrel->part_rels)
-		joinrel->part_rels = (RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+		joinrel->part_rels = (RelOptInfo **) palloc0(sizeof(RelOptInfo *) *
+													 joinrel->nparts);
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	cnt_parts = 0;
+	forboth(lc1, parts1, lc2, parts2)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
+		int			part1 = lfirst_int(lc1);
+		int			part2 = lfirst_int(lc2);
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1399,6 +1419,10 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		Assert(part1 >= 0 && part2 >= 0);
+		child_rel1 = rel1->part_rels[part1];
+		child_rel2 = rel2->part_rels[part2];
+
 		/* We should never try to join two overlapping sets of rels. */
 		Assert(!bms_overlap(child_rel1->relids, child_rel2->relids));
 		child_joinrelids = bms_union(child_rel1->relids, child_rel2->relids);
@@ -1431,6 +1455,15 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 			joinrel->part_rels[cnt_parts] = child_joinrel;
 		}
 
+		/*
+		 * For every pair of joining relations, the set of matching partitions
+		 * would change. However, the base relation partitions constituting
+		 * the given child should remain same for all the joining pairs. Since
+		 * the order in which children are stored in the array of child-joins,
+		 * depends upon partition bounds of the join, which are same for all
+		 * the joining pairs, every joining pair yields the child-joins in the
+		 * same order.
+		 */
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
 
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
@@ -1443,7 +1476,11 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		 */
 		try_partition_wise_join(root, child_rel1, child_rel2, child_joinrel,
 								child_sjinfo, child_restrictlist);
+
+		cnt_parts++;
 	}
+
+	Assert(cnt_parts == joinrel->nparts);
 }
 
 /*
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 81bc2b1..5d1992e 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1613,6 +1613,9 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	int			partnatts;
 	int			cnt;
 	PartitionScheme part_scheme;
+	PartitionBoundInfo join_boundinfo;
+	List	   *parts1;
+	List	   *parts2;
 
 	/* Nothing to do if partition-wise join technique is disabled. */
 	if (!enable_partition_wise_join)
@@ -1653,17 +1656,26 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 		   REL_HAS_ALL_PART_PROPS(inner_rel));
 
 	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
+	 * Every pair of joining relations would yield the same partition bounds
+	 * for a given join (TODO: why?) so we compute the bounds only the first
+	 * time. Then for every pair we find the pairs of matching partitions from
+	 * the joining relations and join those. TODO: Needs a better explanation
+	 * of why is this true.  TODO: Also there is no reason to have
+	 * part_indexes1 and part_indexes2 pulled here just to be freed up later.
+	 * So, we might want to do something better.
 	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
+	join_boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											part_scheme->partsupfunc,
+											part_scheme->parttypcoll,
+											outer_rel->boundinfo,
+											outer_rel->nparts,
+											inner_rel->boundinfo,
+											inner_rel->nparts,
+											jointype, &parts1, &parts2);
+	if (!join_boundinfo)
 	{
 		Assert(!IS_PARTITIONED_REL(joinrel));
+		Assert(!parts1 && !parts2);
 		return;
 	}
 
@@ -1676,13 +1688,16 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 		   !joinrel->nullable_partexprs && !joinrel->part_rels &&
 		   !joinrel->boundinfo);
 
+	Assert(list_length(parts1) == list_length(parts2));
+
 	/*
 	 * Join relation is partitioned using same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations. It will have as many partitions as the pairs of
+	 * matching partitions we found.
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
-	joinrel->nparts = outer_rel->nparts;
+	joinrel->nparts = list_length(parts1);
+	joinrel->boundinfo = join_boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 
 	/*
@@ -1803,4 +1818,9 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 			joinrel->nullable_partexprs[cnt] = nullable_partexprs;
 		}
 	}
+
+	/* TODO: OR we could actually create the child-join relations here.*/
+	list_free(parts1);
+	list_free(parts2);
+
 }
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 2283c67..056a4f9 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -99,4 +99,10 @@ extern int get_partition_for_tuple(PartitionDispatch *pd,
 						EState *estate,
 						PartitionDispatchData **failed_at,
 						TupleTableSlot **failed_slot);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   FmgrInfo *supfuncs, Oid *collations,
+					   PartitionBoundInfo boundinfo1, int nparts1,
+					   PartitionBoundInfo boundinfo2, int nparts2,
+					   JoinType jointype, List **parts1, List **parts2);
+
 #endif							/* PARTITION_H */
-- 
1.7.9.5

#4Rajkumar Raghuwanshi
rajkumar.raghuwanshi@enterprisedb.com
In reply to: Ashutosh Bapat (#3)
1 attachment(s)
Re: advanced partition matching algorithm for partition-wise join

On Sat, Sep 2, 2017 at 12:42 AM, Ashutosh Bapat <
ashutosh.bapat@enterprisedb.com> wrote:

PFA the patches rebased on the latest sources. There are also fixes
for some of the crashes and bugs reported. I haven't yet included the
testcase patch in the main patchset.

On Mon, Aug 28, 2017 at 12:44 PM, Rajkumar Raghuwanshi
<rajkumar.raghuwanshi@enterprisedb.com> wrote:

On Mon, Aug 21, 2017 at 12:43 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

TODOs
-----------
1. Add tests for advanced partition matching algorithm

Hi Ashutosh,

I have applied all partition-wise-join patches (v26) and tested feature.

I

have modified partition_join.sql file and added extra test cases to test
partition matching.

Attaching WIP test case patch which as of now have some server crashes

and a

data corruptions issue which is commented in the file itself and need to

be

removed once issue got solved. Also some of queries is not picking or
picking partition-wise-join as per expectation which may need some
adjustment.

I have applied v27 patches and tested feature. Also tried to reduce
regression diff with the
existing partition_join.sql by adding new partition instead of changing
original partition bounds.

Attached WIP patch have a server crash and some wrong output which need to
be fixed. I have
commented these issue in patch itself, Please take a look and let me know
if it need more
changes.

Attachments:

advance_partition_matching_test_v1.patchtext/x-patch; charset=US-ASCII; name=advance_partition_matching_test_v1.patchDownload
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index b92fbb9..0dd0b59 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -8,105 +8,159 @@ SET enable_partition_wise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
 INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 799, 3) i;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-(4 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -250 | -0250 | -250 | -0250
+ -100 | -0100 | -100 | -0100
+    0 | 0000  |    0 | 0000
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+(9 rows)
 
 -- left outer join, with whole-row reference
 EXPLAIN (COSTS OFF)
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b
    ->  Result
          ->  Append
                ->  Hash Right Join
                      Hash Cond: (t2.b = t1.a)
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                      ->  Hash
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                                  Filter: (b = 0)
                ->  Hash Right Join
                      Hash Cond: (t2_1.b = t1_1.a)
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                                  Filter: (b = 0)
                ->  Hash Right Join
                      Hash Cond: (t2_2.b = t1_2.a)
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                                  Filter: (b = 0)
-(22 rows)
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (t1_3.a = b)
+               ->  Hash Right Join
+                     Hash Cond: (t2_4.b = t1_4.a)
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_p4 t1_4
+                                 Filter: (b = 0)
+(33 rows)
 
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-      t1      |      t2      
---------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
- (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
- (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
- (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
- (550,0,0550) | 
-(12 rows)
+       t1       |       t2       
+----------------+----------------
+ (-250,0,-0250) | (0,-250,-0250)
+ (-200,0,-0200) | 
+ (-150,0,-0150) | 
+ (-100,0,-0100) | (0,-100,-0100)
+ (-50,0,-0050)  | 
+ (0,0,0000)     | (0,0,0000)
+ (0,0,0000)     | (0,0,0000)
+ (50,0,0050)    | 
+ (100,0,0100)   | 
+ (150,0,0150)   | (0,150,0150)
+ (200,0,0200)   | 
+ (250,0,0250)   | 
+ (300,0,0300)   | (0,300,0300)
+ (350,0,0350)   | 
+ (400,0,0400)   | 
+ (450,0,0450)   | (0,450,0450)
+ (500,0,0500)   | 
+ (550,0,0550)   | 
+ (600,0,0600)   | (0,600,0600)
+ (650,0,0650)   | 
+ (700,0,0700)   | 
+ (750,0,0750)   | (0,750,0750)
+(22 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
@@ -119,35 +173,55 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
          ->  Append
                ->  Hash Right Join
                      Hash Cond: (t1.a = t2.b)
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                      ->  Hash
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                                  Filter: (a = 0)
                ->  Hash Right Join
                      Hash Cond: (t1_1.a = t2_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                      ->  Hash
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
+                                 Filter: (a = 0)
+               ->  Hash Right Join
+                     Hash Cond: (t1_2.a = t2_2.b)
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Hash
+                           ->  Seq Scan on prt2_p2 t2_2
                                  Filter: (a = 0)
                ->  Nested Loop Left Join
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
                            Filter: (a = 0)
-                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                           Index Cond: (a = t2_2.b)
-(21 rows)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                           Index Cond: (a = t2_3.b)
+               ->  Hash Right Join
+                     Hash Cond: (t1_4.a = t2_4.b)
+                     ->  Seq Scan on prt1_p4 t1_4
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4 t2_4
+                                 Filter: (a = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -250 | -0250 | -250 | -0250
+ -100 | -0100 | -100 | -0100
+    0 | 0000  |    0 | 0000
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+      |       | -175 | -0175
+      |       |  -25 | -0025
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(16 rows)
 
 -- full outer join
 EXPLAIN (COSTS OFF)
@@ -155,9 +229,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.b = 0) t1 FULL
                     QUERY PLAN                    
 --------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
          ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
+         ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                ->  Seq Scan on prt1_p1
                      Filter: (b = 0)
@@ -178,28 +259,48 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.b = 0) t1 FULL
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(24 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(38 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 | 150 | 0150
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
- 450 | 0450 | 450 | 0450
- 500 | 0500 |     | 
- 550 | 0550 |     | 
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(16 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -250 | -0250 | -250 | -0250
+ -200 | -0200 |      | 
+ -150 | -0150 |      | 
+ -100 | -0100 | -100 | -0100
+  -50 | -0050 |      | 
+    0 | 0000  |    0 | 0000
+    0 | 0000  |    0 | 0000
+   50 | 0050  |      | 
+  100 | 0100  |      | 
+  150 | 0150  |  150 | 0150
+  200 | 0200  |      | 
+  250 | 0250  |      | 
+  300 | 0300  |  300 | 0300
+  350 | 0350  |      | 
+  400 | 0400  |      | 
+  450 | 0450  |  450 | 0450
+  500 | 0500  |      | 
+  550 | 0550  |      | 
+  600 | 0600  |  600 | 0600
+  650 | 0650  |      | 
+  700 | 0700  |      | 
+  750 | 0750  |  750 | 0750
+      |       | -175 | -0175
+      |       |  -25 | -0025
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(29 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -208,9 +309,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                             QUERY PLAN                            
 ------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
          ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               Filter: (((50) = prt1_p0.a) OR ((75) = prt2_p0.b))
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
+         ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
                ->  Seq Scan on prt1_p1
@@ -234,7 +343,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               Filter: (((50) = prt1_p4.a) OR ((75) = prt2_p4.b))
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
@@ -248,10 +365,17 @@ SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 W
                        QUERY PLAN                       
 --------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Result
          ->  Append
                ->  Hash Full Join
+                     Hash Cond: (prt1_p0.a = prt2_p0.b)
+                     ->  Seq Scan on prt1_p0
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p0
+                                 Filter: (a = 0)
+               ->  Hash Full Join
                      Hash Cond: (prt1_p1.a = prt2_p1.b)
                      ->  Seq Scan on prt1_p1
                            Filter: (b = 0)
@@ -272,28 +396,48 @@ SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 W
                      ->  Hash
                            ->  Seq Scan on prt2_p3
                                  Filter: (a = 0)
-(25 rows)
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = prt2_p4.b)
+                     ->  Seq Scan on prt1_p4
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4
+                                 Filter: (a = 0)
+(39 rows)
 
 SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
-  a  |  c   | phv |  b  |  c   | phv 
------+------+-----+-----+------+-----
-   0 | 0000 |  25 |   0 | 0000 |  50
-  50 | 0050 |  25 |     |      |    
- 100 | 0100 |  25 |     |      |    
- 150 | 0150 |  25 | 150 | 0150 |  50
- 200 | 0200 |  25 |     |      |    
- 250 | 0250 |  25 |     |      |    
- 300 | 0300 |  25 | 300 | 0300 |  50
- 350 | 0350 |  25 |     |      |    
- 400 | 0400 |  25 |     |      |    
- 450 | 0450 |  25 | 450 | 0450 |  50
- 500 | 0500 |  25 |     |      |    
- 550 | 0550 |  25 |     |      |    
-     |      |     |  75 | 0075 |  50
-     |      |     | 225 | 0225 |  50
-     |      |     | 375 | 0375 |  50
-     |      |     | 525 | 0525 |  50
-(16 rows)
+  a   |   c   | phv |  b   |   c   | phv 
+------+-------+-----+------+-------+-----
+ -250 | -0250 |  25 | -250 | -0250 |  50
+ -200 | -0200 |  25 |      |       |    
+ -150 | -0150 |  25 |      |       |    
+ -100 | -0100 |  25 | -100 | -0100 |  50
+  -50 | -0050 |  25 |      |       |    
+    0 | 0000  |  25 |    0 | 0000  |  50
+    0 | 0000  |  25 |    0 | 0000  |  50
+   50 | 0050  |  25 |      |       |    
+  100 | 0100  |  25 |      |       |    
+  150 | 0150  |  25 |  150 | 0150  |  50
+  200 | 0200  |  25 |      |       |    
+  250 | 0250  |  25 |      |       |    
+  300 | 0300  |  25 |  300 | 0300  |  50
+  350 | 0350  |  25 |      |       |    
+  400 | 0400  |  25 |      |       |    
+  450 | 0450  |  25 |  450 | 0450  |  50
+  500 | 0500  |  25 |      |       |    
+  550 | 0550  |  25 |      |       |    
+  600 | 0600  |  25 |  600 | 0600  |  50
+  650 | 0650  |  25 |      |       |    
+  700 | 0700  |  25 |      |       |    
+  750 | 0750  |  25 |  750 | 0750  |  50
+      |       |     | -175 | -0175 |  50
+      |       |     |  -25 | -0025 |  50
+      |       |     |   75 | 0075  |  50
+      |       |     |  225 | 0225  |  50
+      |       |     |  375 | 0375  |  50
+      |       |     |  525 | 0525  |  50
+      |       |     |  675 | 0675  |  50
+(29 rows)
 
 -- Join with pruned partitions from joining relations
 EXPLAIN (COSTS OFF)
@@ -323,9 +467,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
          ->  Hash Left Join
+               Hash Cond: (prt1_p0.a = b)
+               ->  Seq Scan on prt1_p0
+                     Filter: ((a < 450) AND (b = 0))
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
+         ->  Hash Left Join
                Hash Cond: (prt1_p1.a = b)
                ->  Seq Scan on prt1_p1
                      Filter: ((a < 450) AND (b = 0))
@@ -339,30 +490,44 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                ->  Hash
                      ->  Seq Scan on prt1_p2
                            Filter: ((a < 450) AND (b = 0))
-(17 rows)
+(24 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-(9 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+(15 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
                          QUERY PLAN                         
 ------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
          ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = b)
+               Filter: ((prt1_p0.b = 0) OR (a = 0))
+               ->  Seq Scan on prt1_p0
+                     Filter: (a < 450)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
+         ->  Hash Full Join
                Hash Cond: (prt1_p1.a = b)
                Filter: ((prt1_p1.b = 0) OR (a = 0))
                ->  Seq Scan on prt1_p1
@@ -386,64 +551,157 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                ->  Hash
                      ->  Result
                            One-Time Filter: false
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt2_p4.b = a)
+               Filter: ((b = 0) OR (prt2_p4.a = 0))
+               ->  Seq Scan on prt2_p4
+                     Filter: (b > 250)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
-(12 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+      |       | 375 | 0375
+      |       | 450 | 0450
+      |       | 525 | 0525
+      |       | 600 | 0600
+      |       | 675 | 0675
+      |       | 750 | 0750
+(21 rows)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Semi Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Semi Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                            Filter: (a = 0)
-         ->  Nested Loop Semi Join
-               Join Filter: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
-               ->  Materialize
-                     ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
-(24 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -100 | 0 | -0100
+    0 | 0 | 0000
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(9 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt1_p3 t2_3
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+                           Filter: (b = 0)
+(37 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+ a |  b   |   c   
+---+------+-------
+ 0 | -250 | -0250
+ 0 | -100 | -0100
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(8 rows)
 
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
@@ -454,25 +712,74 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
    ->  Append
          ->  Hash Anti Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Hash Anti Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Hash Anti Join
                Hash Cond: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
-(17 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(27 rows)
 
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
-  sum  |         avg          | sum  |         avg         
--------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+  sum  |         avg          | sum  |        avg         
+-------+----------------------+------+--------------------
+ 95718 | 274.2636103151862464 | 2168 | 6.2120343839541547
+(1 row)
+
+EXPLAIN (COSTS OFF)
+SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a);
+                    QUERY PLAN                    
+--------------------------------------------------
+ Aggregate
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt1_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+(27 rows)
+
+SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a);
+ sum  |        avg         |  sum  |         avg          
+------+--------------------+-------+----------------------
+ 1084 | 6.1942857142857143 | 47859 | 273.4800000000000000
 (1 row)
 
 -- lateral reference
@@ -487,49 +794,77 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
    ->  Result
          ->  Append
                ->  Nested Loop Left Join
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
                      ->  Nested Loop
-                           ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
+                           ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
                                  Index Cond: (a = t1.a)
-                           ->  Index Scan using iprt2_p1_b on prt2_p1 t3
+                           ->  Index Scan using iprt2_p0_b on prt2_p0 t3
                                  Index Cond: (b = t2.a)
                ->  Nested Loop Left Join
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
                      ->  Nested Loop
-                           ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_1
+                           ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
                                  Index Cond: (a = t1_1.a)
-                           ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
+                           ->  Index Scan using iprt2_p1_b on prt2_p1 t3_1
                                  Index Cond: (b = t2_1.a)
                ->  Nested Loop Left Join
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
                      ->  Nested Loop
-                           ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_2
+                           ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
                                  Index Cond: (a = t1_2.a)
-                           ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
+                           ->  Index Scan using iprt2_p2_b on prt2_p2 t3_2
                                  Index Cond: (b = t2_2.a)
-(28 rows)
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Nested Loop
+                           ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                                 Index Cond: (a = t1_3.a)
+                           ->  Index Scan using iprt2_p3_b on prt2_p3 t3_3
+                                 Index Cond: (b = t2_3.a)
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+                     ->  Nested Loop
+                           ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                                 Index Cond: (a = t1_4.a)
+                           ->  Index Scan using iprt2_p4_b on prt2_p4 t3_4
+                                 Index Cond: (b = t2_4.a)
+(44 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   | t2a | t3a | least 
------+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
- 100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
- 250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
- 400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
- 550 | 0 | 0550 |     |     |      
-(12 rows)
+  a   | b |   c   | t2a  | t3a | least 
+------+---+-------+------+-----+-------
+ -250 | 0 | -0250 | -250 |   0 |  -250
+ -200 | 0 | -0200 |      |     |      
+ -150 | 0 | -0150 |      |     |      
+ -100 | 0 | -0100 | -100 |   0 |  -100
+  -50 | 0 | -0050 |      |     |      
+    0 | 0 | 0000  |    0 |   0 |     0
+    0 | 0 | 0000  |    0 |   0 |     0
+    0 | 0 | 0000  |    0 |   0 |     0
+    0 | 0 | 0000  |    0 |   0 |     0
+   50 | 0 | 0050  |      |     |      
+  100 | 0 | 0100  |      |     |      
+  150 | 0 | 0150  |  150 |   0 |   150
+  200 | 0 | 0200  |      |     |      
+  250 | 0 | 0250  |      |     |      
+  300 | 0 | 0300  |  300 |   0 |   300
+  350 | 0 | 0350  |      |     |      
+  400 | 0 | 0400  |      |     |      
+  450 | 0 | 0450  |  450 |   0 |   450
+  500 | 0 | 0500  |      |     |      
+  550 | 0 | 0550  |      |     |      
+  600 | 0 | 0600  |  600 |   0 |   600
+  650 | 0 | 0650  |      |     |      
+  700 | 0 | 0700  |      |     |      
+  750 | 0 | 0750  |  750 |   0 |   750
+(24 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
@@ -543,64 +878,98 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
          Hash Cond: ((t1.c)::text = (t2.c)::text)
          Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Seq Scan on prt1_p1 t1
-               ->  Seq Scan on prt1_p2 t1_1
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
          ->  Hash
                ->  Append
                      ->  Hash Join
                            Hash Cond: (t2.a = t3.b)
-                           ->  Seq Scan on prt1_p1 t2
+                           ->  Seq Scan on prt1_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t3
+                                 ->  Seq Scan on prt2_p0 t3
                      ->  Hash Join
                            Hash Cond: (t2_1.a = t3_1.b)
-                           ->  Seq Scan on prt1_p2 t2_1
+                           ->  Seq Scan on prt1_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t3_1
+                                 ->  Seq Scan on prt2_p1 t3_1
                      ->  Hash Join
                            Hash Cond: (t2_2.a = t3_2.b)
-                           ->  Seq Scan on prt1_p3 t2_2
+                           ->  Seq Scan on prt1_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p3 t3_2
-(26 rows)
+                                 ->  Seq Scan on prt2_p2 t3_2
+                     ->  Hash Join
+                           Hash Cond: (t2_3.a = t3_3.b)
+                           ->  Seq Scan on prt1_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t3_3
+                     ->  Hash Join
+                           Hash Cond: (t2_4.a = t3_4.b)
+                           ->  Seq Scan on prt1_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t3_4
+(38 rows)
 
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, t2.b t2b, t2.c t2c, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
-  a  | t2a | t2c  
------+-----+------
-   0 |   0 | 0000
-  50 |     | 
- 100 |     | 
- 150 | 150 | 0150
- 200 |     | 
- 250 |     | 
- 300 | 300 | 0300
- 350 |     | 
- 400 |     | 
- 450 | 450 | 0450
- 500 |     | 
- 550 |     | 
-(12 rows)
+  a   | t2a  |  t2c  
+------+------+-------
+ -250 | -250 | -0250
+ -200 |      | 
+ -150 |      | 
+ -100 | -100 | -0100
+  -50 |      | 
+    0 |    0 | 0000
+    0 |    0 | 0000
+    0 |    0 | 0000
+    0 |    0 | 0000
+   50 |      | 
+  100 |      | 
+  150 |  150 | 0150
+  200 |      | 
+  250 |      | 
+  300 |  300 | 0300
+  350 |      | 
+  400 |      | 
+  450 |  450 | 0450
+  500 |      | 
+  550 |      | 
+  600 |  600 | 0600
+  650 |      | 
+  700 |      | 
+  750 |  750 | 0750
+(24 rows)
 
 --
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
@@ -611,32 +980,49 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
    ->  Append
          ->  Hash Join
                Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
-               ->  Seq Scan on prt2_e_p1 t2
+               ->  Seq Scan on prt2_e_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 t1
+                     ->  Seq Scan on prt1_e_p0 t1
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
-               ->  Seq Scan on prt2_e_p2 t2_1
+               Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+               ->  Seq Scan on prt1_e_p1 t1_1
+                     Filter: (c = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 t1_1
-                           Filter: (c = 0)
+                     ->  Seq Scan on prt2_e_p1 t2_1
          ->  Hash Join
                Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
-               ->  Seq Scan on prt2_e_p3 t2_2
+               ->  Seq Scan on prt2_e_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 t1_2
+                     ->  Seq Scan on prt1_e_p2 t1_2
                            Filter: (c = 0)
-(21 rows)
+         ->  Hash Join
+               Hash Cond: (((t2_3.b + t2_3.a) / 2) = ((t1_3.a + t1_3.b) / 2))
+               ->  Seq Scan on prt2_e_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p3 t1_3
+                           Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (((t2_4.b + t2_4.a) / 2) = ((t1_4.a + t1_4.b) / 2))
+               ->  Seq Scan on prt2_e_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4 t1_4
+                           Filter: (c = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
- 150 | 0 | 150 | 0
- 300 | 0 | 300 | 0
- 450 | 0 | 450 | 0
-(4 rows)
+  a   | c |  b   | c 
+------+---+------+---
+ -250 | 0 | -250 | 0
+ -100 | 0 | -100 | 0
+    0 | 0 |    0 | 0
+    0 | 0 |    0 | 0
+  150 | 0 |  150 | 0
+  300 | 0 |  300 | 0
+  450 | 0 |  450 | 0
+  600 | 0 |  600 | 0
+  750 | 0 |  750 | 0
+(9 rows)
 
 --
 -- N-way join
@@ -650,107 +1036,163 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t
    ->  Result
          ->  Append
                ->  Nested Loop
-                     Join Filter: (t1.a = ((t3.a + t3.b) / 2))
+                     Join Filter: (t1.a = t2.b)
                      ->  Hash Join
-                           Hash Cond: (t2.b = t1.a)
-                           ->  Seq Scan on prt2_p1 t2
+                           Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                           ->  Seq Scan on prt1_e_p0 t3
                            ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                                        Filter: (b = 0)
-                     ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                           Index Cond: (((a + b) / 2) = t2.b)
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t2
+                           Index Cond: (b = ((t3.a + t3.b) / 2))
                ->  Nested Loop
                      Join Filter: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
                      ->  Hash Join
                            Hash Cond: (t2_1.b = t1_1.a)
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                                        Filter: (b = 0)
-                     ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
+                     ->  Index Scan using iprt1_e_p4_ab2 on prt1_e_p1 t3_1
                            Index Cond: (((a + b) / 2) = t2_1.b)
                ->  Nested Loop
                      Join Filter: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
                      ->  Hash Join
                            Hash Cond: (t2_2.b = t1_2.a)
-                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                                        Filter: (b = 0)
-                     ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
+                     ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_2
                            Index Cond: (((a + b) / 2) = t2_2.b)
-(34 rows)
+               ->  Nested Loop
+                     Join Filter: (t1_3.a = ((t3_3.a + t3_3.b) / 2))
+                     ->  Nested Loop
+                           ->  Seq Scan on prt1_p3 t1_3
+                                 Filter: (b = 0)
+                           ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                                 Index Cond: (b = t1_3.a)
+                     ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                           Index Cond: (((a + b) / 2) = t2_3.b)
+               ->  Nested Loop
+                     Join Filter: (t1_4.a = t2_4.b)
+                     ->  Hash Join
+                           Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                           ->  Seq Scan on prt1_e_p4 t3_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_p4 t1_4
+                                       Filter: (b = 0)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                           Index Cond: (b = ((t3_4.a + t3_4.b) / 2))
+(53 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
-(4 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 | -250 | -0250 |     -500 | 0
+ -100 | -0100 | -100 | -0100 |     -200 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(11 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                             QUERY PLAN                             
---------------------------------------------------------------------
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Result
          ->  Append
                ->  Hash Right Join
                      Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
-                     ->  Seq Scan on prt1_e_p1 t3
+                     ->  Seq Scan on prt1_e_p0 t3
                      ->  Hash
                            ->  Hash Right Join
                                  Hash Cond: (t2.b = t1.a)
-                                 ->  Seq Scan on prt2_p1 t2
+                                 ->  Seq Scan on prt2_p0 t2
                                  ->  Hash
-                                       ->  Seq Scan on prt1_p1 t1
+                                       ->  Seq Scan on prt1_p0 t1
                                              Filter: (b = 0)
                ->  Hash Right Join
                      Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_1.a)
-                     ->  Seq Scan on prt1_e_p2 t3_1
+                     ->  Seq Scan on prt1_e_p1 t3_1
                      ->  Hash
                            ->  Hash Right Join
                                  Hash Cond: (t2_1.b = t1_1.a)
-                                 ->  Seq Scan on prt2_p2 t2_1
+                                 ->  Seq Scan on prt2_p1 t2_1
                                  ->  Hash
-                                       ->  Seq Scan on prt1_p2 t1_1
+                                       ->  Seq Scan on prt1_p1 t1_1
                                              Filter: (b = 0)
                ->  Hash Right Join
                      Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_2.a)
-                     ->  Seq Scan on prt1_e_p3 t3_2
+                     ->  Seq Scan on prt1_e_p2 t3_2
                      ->  Hash
                            ->  Hash Right Join
                                  Hash Cond: (t2_2.b = t1_2.a)
-                                 ->  Seq Scan on prt2_p3 t2_2
+                                 ->  Seq Scan on prt2_p2 t2_2
+                                 ->  Hash
+                                       ->  Seq Scan on prt1_p2 t1_2
+                                             Filter: (b = 0)
+               ->  Nested Loop Left Join
+                     ->  Nested Loop Left Join
+                           ->  Seq Scan on prt1_p3 t1_3
+                                 Filter: (b = 0)
+                           ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                                 Index Cond: (t1_3.a = b)
+                     ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                           Index Cond: (t1_3.a = ((a + b) / 2))
+               ->  Hash Right Join
+                     Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                     ->  Seq Scan on prt1_e_p4 t3_4
+                     ->  Hash
+                           ->  Hash Right Join
+                                 Hash Cond: (t2_4.b = t1_4.a)
+                                 ->  Seq Scan on prt2_p4 t2_4
                                  ->  Hash
-                                       ->  Seq Scan on prt1_p3 t1_2
+                                       ->  Seq Scan on prt1_p4 t1_4
                                              Filter: (b = 0)
-(34 rows)
+(52 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 | -250 | -0250 |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 |      |       |     -300 | 0
+ -100 | -0100 | -100 | -0100 |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(24 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                               QUERY PLAN                                
--------------------------------------------------------------------------
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Result
@@ -758,48 +1200,77 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                ->  Nested Loop Left Join
                      ->  Hash Right Join
                            Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t3
+                                 ->  Seq Scan on prt1_e_p0 t3
                                        Filter: (c = 0)
-                     ->  Index Scan using iprt2_p1_b on prt2_p1 t2
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t2
                            Index Cond: (t1.a = b)
                ->  Nested Loop Left Join
                      ->  Hash Right Join
                            Hash Cond: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                 ->  Seq Scan on prt1_e_p1 t3_1
                                        Filter: (c = 0)
-                     ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
                            Index Cond: (t1_1.a = b)
                ->  Nested Loop Left Join
                      ->  Hash Right Join
                            Hash Cond: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                 ->  Seq Scan on prt1_e_p2 t3_2
                                        Filter: (c = 0)
-                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
                            Index Cond: (t1_2.a = b)
-(31 rows)
+               ->  Nested Loop Left Join
+                     ->  Nested Loop Left Join
+                           ->  Seq Scan on prt1_e_p3 t3_3
+                                 Filter: (c = 0)
+                           ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                                 Index Cond: (a = ((t3_3.a + t3_3.b) / 2))
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (t1_3.a = b)
+               ->  Nested Loop Left Join
+                     ->  Hash Right Join
+                           Hash Cond: (t1_4.a = ((t3_4.a + t3_4.b) / 2))
+                           ->  Seq Scan on prt1_p4 t1_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t3_4
+                                       Filter: (c = 0)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                           Index Cond: (t1_4.a = b)
+(48 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 | -250 | -0250 |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 |      |       |     -300 | 0
+ -100 | -0100 | -100 | -0100 |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(24 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -808,10 +1279,23 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                                                       QUERY PLAN                                                      
 ----------------------------------------------------------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   Sort Key: prt1_p0.a, prt2_p0.b, ((prt1_e_p0.a + prt1_e_p0.b))
    ->  Result
          ->  Append
                ->  Hash Full Join
+                     Hash Cond: (prt1_p0.a = ((prt1_e_p0.a + prt1_e_p0.b) / 2))
+                     Filter: ((prt1_p0.a = (50)) OR (prt2_p0.b = (75)) OR (((prt1_e_p0.a + prt1_e_p0.b) / 2) = (50)))
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_p0.a = prt2_p0.b)
+                           ->  Seq Scan on prt1_p0
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p0
+                                       Filter: (a = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p0
+                                 Filter: (c = 0)
+               ->  Hash Full Join
                      Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
                      Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
                      ->  Hash Full Join
@@ -850,7 +1334,20 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                      ->  Hash
                            ->  Seq Scan on prt1_e_p3
                                  Filter: (c = 0)
-(43 rows)
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = ((prt1_e_p4.a + prt1_e_p4.b) / 2))
+                     Filter: ((prt1_p4.a = (50)) OR (prt2_p4.b = (75)) OR (((prt1_e_p4.a + prt1_e_p4.b) / 2) = (50)))
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_p4.a = prt2_p4.b)
+                           ->  Seq Scan on prt1_p4
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4
+                                       Filter: (a = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p4
+                                 Filter: (c = 0)
+(69 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
@@ -868,173 +1365,263 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = t1_3.b)
+               Join Filter: (t1.a = t1_5.b)
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Join
-                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
-                           ->  Seq Scan on prt1_e_p1 t2
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t1_3
+                                 ->  Seq Scan on prt2_p0 t1_5
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
                      Index Cond: (a = ((t2.a + t2.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_1.a = t1_4.b)
+               Join Filter: (t1_1.a = t1_6.b)
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Join
-                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
-                           ->  Seq Scan on prt1_e_p2 t2_1
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_6.b)
+                           ->  Seq Scan on prt1_e_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t1_4
+                                 ->  Seq Scan on prt2_p1 t1_6
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
                      Index Cond: (a = ((t2_1.a + t2_1.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_2.a = t1_5.b)
+               Join Filter: (t1_2.a = t1_7.b)
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
+                           ->  Seq Scan on prt2_p2 t1_7
                                  Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
+                           ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t2_2
+                                 Index Cond: (((a + b) / 2) = t1_7.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t1_8.b)
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Nested Loop
+                           ->  Seq Scan on prt2_p3 t1_8
+                                 Filter: (a = 0)
+                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_3
+                                 Index Cond: (((a + b) / 2) = t1_8.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = ((t2_3.a + t2_3.b) / 2))
+                     Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t1_9.b)
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Join
+                           Hash Cond: (((t2_4.a + t2_4.b) / 2) = t1_9.b)
+                           ->  Seq Scan on prt1_e_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t1_9
+                                       Filter: (a = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = ((t2_4.a + t2_4.b) / 2))
+                     Filter: (b = 0)
+(66 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -100 | 0 | -0100
+    0 | 0 | 0000
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(9 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                                  QUERY PLAN                                   
--------------------------------------------------------------------------------
+                                 QUERY PLAN                                 
+----------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
-                           ->  Seq Scan on prt2_p1 t1_3
+                           Hash Cond: (t1_5.b = ((t1_10.a + t1_10.b) / 2))
+                           ->  Seq Scan on prt2_p0 t1_5
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t1_6
+                                 ->  Seq Scan on prt1_e_p0 t1_10
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
-                     Index Cond: (a = t1_3.b)
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t1_5.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
-                           ->  Seq Scan on prt2_p2 t1_4
+                           Hash Cond: (t1_6.b = ((t1_11.a + t1_11.b) / 2))
+                           ->  Seq Scan on prt2_p1 t1_6
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t1_7
+                                 ->  Seq Scan on prt1_e_p1 t1_11
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
-                     Index Cond: (a = t1_4.b)
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
+                     Index Cond: (a = t1_6.b)
                      Filter: (b = 0)
          ->  Nested Loop
-               ->  Unique
-                     ->  Sort
-                           Sort Key: t1_5.b
-                           ->  Hash Semi Join
-                                 Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
-                                 ->  Seq Scan on prt2_p3 t1_5
-                                 ->  Hash
-                                       ->  Seq Scan on prt1_e_p3 t1_8
-                                             Filter: (c = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t1_5.b)
+               ->  HashAggregate
+                     Group Key: t1_7.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_7.b = ((t1_12.a + t1_12.b) / 2))
+                           ->  Seq Scan on prt2_p2 t1_7
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p2 t1_12
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t1_7.b)
+                     Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Nested Loop Semi Join
+                     ->  Index Only Scan using iprt2_p3_b on prt2_p3 t1_8
+                           Index Cond: (b = t1_3.a)
+                     ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t1_13
+                           Index Cond: (((a + b) / 2) = t1_8.b)
+                           Filter: (c = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_9.b = ((t1_14.a + t1_14.b) / 2))
+                           ->  Seq Scan on prt2_p4 t1_9
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t1_14
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = t1_9.b)
                      Filter: (b = 0)
-(40 rows)
+(60 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -100 | 0 | -0100
+    0 | 0 | 0000
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(9 rows)
 
 -- test merge joins
 SET enable_hashjoin TO off;
 SET enable_nestloop TO off;
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                           QUERY PLAN                           
-----------------------------------------------------------------
+                            QUERY PLAN                            
+------------------------------------------------------------------
  Merge Append
    Sort Key: t1.a
    ->  Merge Semi Join
-         Merge Cond: (t1.a = t1_3.b)
+         Merge Cond: (t1.a = t1_5.b)
          ->  Sort
                Sort Key: t1.a
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+               Merge Cond: (t1_5.b = (((t1_10.a + t1_10.b) / 2)))
                ->  Sort
-                     Sort Key: t1_3.b
-                     ->  Seq Scan on prt2_p1 t1_3
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on prt2_p0 t1_5
                ->  Sort
-                     Sort Key: (((t1_6.a + t1_6.b) / 2))
-                     ->  Seq Scan on prt1_e_p1 t1_6
+                     Sort Key: (((t1_10.a + t1_10.b) / 2))
+                     ->  Seq Scan on prt1_e_p0 t1_10
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_1.a = t1_4.b)
+         Merge Cond: (t1_1.a = t1_6.b)
          ->  Sort
                Sort Key: t1_1.a
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+               Merge Cond: (t1_6.b = (((t1_11.a + t1_11.b) / 2)))
                ->  Sort
-                     Sort Key: t1_4.b
-                     ->  Seq Scan on prt2_p2 t1_4
+                     Sort Key: t1_6.b
+                     ->  Seq Scan on prt2_p1 t1_6
                ->  Sort
-                     Sort Key: (((t1_7.a + t1_7.b) / 2))
-                     ->  Seq Scan on prt1_e_p2 t1_7
+                     Sort Key: (((t1_11.a + t1_11.b) / 2))
+                     ->  Seq Scan on prt1_e_p1 t1_11
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_2.a = t1_5.b)
+         Merge Cond: (t1_2.a = t1_7.b)
          ->  Sort
                Sort Key: t1_2.a
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+               Merge Cond: (t1_7.b = (((t1_12.a + t1_12.b) / 2)))
                ->  Sort
-                     Sort Key: t1_5.b
-                     ->  Seq Scan on prt2_p3 t1_5
+                     Sort Key: t1_7.b
+                     ->  Seq Scan on prt2_p2 t1_7
+               ->  Sort
+                     Sort Key: (((t1_12.a + t1_12.b) / 2))
+                     ->  Seq Scan on prt1_e_p2 t1_12
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_3.a = t1_8.b)
+         ->  Sort
+               Sort Key: t1_3.a
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_8.b = (((t1_13.a + t1_13.b) / 2)))
                ->  Sort
-                     Sort Key: (((t1_8.a + t1_8.b) / 2))
-                     ->  Seq Scan on prt1_e_p3 t1_8
+                     Sort Key: t1_8.b
+                     ->  Seq Scan on prt2_p3 t1_8
+               ->  Sort
+                     Sort Key: (((t1_13.a + t1_13.b) / 2))
+                     ->  Seq Scan on prt1_e_p3 t1_13
                            Filter: (c = 0)
-(47 rows)
+   ->  Merge Semi Join
+         Merge Cond: (t1_4.a = t1_9.b)
+         ->  Sort
+               Sort Key: t1_4.a
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_9.b = (((t1_14.a + t1_14.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_9.b
+                     ->  Seq Scan on prt2_p4 t1_9
+               ->  Sort
+                     Sort Key: (((t1_14.a + t1_14.b) / 2))
+                     ->  Seq Scan on prt1_e_p4 t1_14
+                           Filter: (c = 0)
+(77 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -100 | 0 | -0100
+    0 | 0 | 0000
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(9 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
@@ -1052,14 +1639,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                                  Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
                                  ->  Sort
                                        Sort Key: (((t3.a + t3.b) / 2))
-                                       ->  Seq Scan on prt1_e_p1 t3
+                                       ->  Seq Scan on prt1_e_p0 t3
                                              Filter: (c = 0)
                                  ->  Sort
                                        Sort Key: t1.a
-                                       ->  Seq Scan on prt1_p1 t1
+                                       ->  Seq Scan on prt1_p0 t1
                      ->  Sort
                            Sort Key: t2.b
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                ->  Merge Left Join
                      Merge Cond: (t1_1.a = t2_1.b)
                      ->  Sort
@@ -1068,14 +1655,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                                  Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
                                  ->  Sort
                                        Sort Key: (((t3_1.a + t3_1.b) / 2))
-                                       ->  Seq Scan on prt1_e_p2 t3_1
+                                       ->  Seq Scan on prt1_e_p1 t3_1
                                              Filter: (c = 0)
                                  ->  Sort
                                        Sort Key: t1_1.a
-                                       ->  Seq Scan on prt1_p2 t1_1
+                                       ->  Seq Scan on prt1_p1 t1_1
                      ->  Sort
                            Sort Key: t2_1.b
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                ->  Merge Left Join
                      Merge Cond: (t1_2.a = t2_2.b)
                      ->  Sort
@@ -1084,35 +1671,805 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                                  Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
                                  ->  Sort
                                        Sort Key: (((t3_2.a + t3_2.b) / 2))
-                                       ->  Seq Scan on prt1_e_p3 t3_2
+                                       ->  Seq Scan on prt1_e_p2 t3_2
                                              Filter: (c = 0)
                                  ->  Sort
                                        Sort Key: t1_2.a
-                                       ->  Seq Scan on prt1_p3 t1_2
+                                       ->  Seq Scan on prt1_p2 t1_2
                      ->  Sort
                            Sort Key: t2_2.b
-                           ->  Seq Scan on prt2_p3 t2_2
-(52 rows)
+                           ->  Seq Scan on prt2_p2 t2_2
+               ->  Merge Left Join
+                     Merge Cond: (t1_3.a = t2_3.b)
+                     ->  Sort
+                           Sort Key: t1_3.a
+                           ->  Merge Left Join
+                                 Merge Cond: ((((t3_3.a + t3_3.b) / 2)) = t1_3.a)
+                                 ->  Sort
+                                       Sort Key: (((t3_3.a + t3_3.b) / 2))
+                                       ->  Seq Scan on prt1_e_p3 t3_3
+                                             Filter: (c = 0)
+                                 ->  Sort
+                                       Sort Key: t1_3.a
+                                       ->  Seq Scan on prt1_p3 t1_3
+                     ->  Sort
+                           Sort Key: t2_3.b
+                           ->  Seq Scan on prt2_p3 t2_3
+               ->  Merge Left Join
+                     Merge Cond: (t1_4.a = t2_4.b)
+                     ->  Sort
+                           Sort Key: t1_4.a
+                           ->  Merge Left Join
+                                 Merge Cond: ((((t3_4.a + t3_4.b) / 2)) = t1_4.a)
+                                 ->  Sort
+                                       Sort Key: (((t3_4.a + t3_4.b) / 2))
+                                       ->  Seq Scan on prt1_e_p4 t3_4
+                                             Filter: (c = 0)
+                                 ->  Sort
+                                       Sort Key: t1_4.a
+                                       ->  Seq Scan on prt1_p4 t1_4
+                     ->  Sort
+                           Sort Key: t2_4.b
+                           ->  Seq Scan on prt2_p4 t2_4
+(84 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 | -250 | -0250 |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 |      |       |     -300 | 0
+ -100 | -0100 | -100 | -0100 |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(24 rows)
 
 RESET enable_hashjoin;
 RESET enable_nestloop;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 | 0 | -0250
+ -100 | -0100 | 0 | -0100
+    0 | 0000  | 0 | 0000
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+(9 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (t1_3.a = b)
+         ->  Hash Right Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 | 0 | -0250
+ -200 | -0200 |   | 
+ -150 | -0150 |   | 
+ -100 | -0100 | 0 | -0100
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+(22 rows)
+
+-- TODO: below query is picking partition-wise-join which is not expected
+-- also generating wrong output, need to remove comment after fix
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Result
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(30 rows)
+
+SET enable_partition_wise_join to false;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 | 0 | -0250
+ -100 | -0100 | 0 | -0100
+    0 | 0000  | 0 | 0000
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+      |       | 0 | 0075
+      |       | 0 | 0375
+      |       | 0 | 0825
+      |       | 0 | -0175
+      |       | 0 | 0900
+      |       | 0 | 0675
+      |       | 0 | -0025
+      |       | 0 | 0525
+      |       | 0 | 0225
+      |       | 0 | 0975
+(19 rows)
+
+SET enable_partition_wise_join to true;
+-- TODO: below query is picking partition-wise-join which is not expected
+-- also generating wrong output, need to remove comment after fix
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.a = t2.b)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.a, 0)) = 0)
+               ->  Seq Scan on prt1_p0 t1
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Full Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.a, 0)) = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.a, 0)) = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.a, 0)) = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Full Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               Filter: ((COALESCE(t1_4.b, 0) + COALESCE(t2_4.a, 0)) = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(33 rows)
+
+SET enable_partition_wise_join to false;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 | 0 | -0250
+ -200 | -0200 |   | 
+ -150 | -0150 |   | 
+ -100 | -0100 | 0 | -0100
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+      |       | 0 | 0825
+      |       | 0 | -0175
+      |       | 0 | 0900
+      |       | 0 | 0675
+      |       | 0 | -0025
+      |       | 0 | 0525
+      |       | 0 | 0225
+      |       | 0 | 0975
+      |       | 0 | 0075
+      |       | 0 | 0375
+(32 rows)
+
+SET enable_partition_wise_join to true;
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -100 | 0 | -0100
+    0 | 0 | 0000
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(9 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -250 | -0250
+ 0 | -100 | -0100
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(8 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -200 | 0 | -0200
+ -150 | 0 | -0150
+  -50 | 0 | -0050
+   50 | 0 | 0050
+  100 | 0 | 0100
+  200 | 0 | 0200
+  250 | 0 | 0250
+  350 | 0 | 0350
+  400 | 0 | 0400
+  500 | 0 | 0500
+  550 | 0 | 0550
+  650 | 0 | 0650
+  700 | 0 | 0700
+(13 rows)
+
+-- TODO: below query is generating wrong output, need to remove comment after fix
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+SET enable_partition_wise_join to false;
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -175 | -0175
+ 0 |  -25 | -0025
+ 0 |   75 | 0075
+ 0 |  225 | 0225
+ 0 |  375 | 0375
+ 0 |  525 | 0525
+ 0 |  675 | 0675
+ 0 |  825 | 0825
+ 0 |  900 | 0900
+ 0 |  975 | 0975
+(10 rows)
+
+SET enable_partition_wise_join to true;
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
+                     Filter: (a = 0)
+(22 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
 --
 -- partitioned by multiple columns
 --
@@ -1182,32 +2539,388 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Result
+         ->  Append
+               ->  Hash Right Join
+                     Hash Cond: ((t1.c)::text = (t2.c)::text)
+                     Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Hash
+                           ->  Seq Scan on plt2_p4 t2
+               ->  Hash Left Join
+                     Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+                     Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Hash
+                           ->  Seq Scan on plt1_p1 t1_1
+               ->  Hash Left Join
+                     Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+                     Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Hash
+                           ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash Right Join
+                     Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+                     Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+                     ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt2_p3 t2_3
+(28 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  HashAggregate
+                           Group Key: (t2.c)::text
+                           ->  Seq Scan on plt2_p4 t2
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 -- test partition matching with N-way join
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                      QUERY PLAN                                      
---------------------------------------------------------------------------------------
+                                       QUERY PLAN                                       
+----------------------------------------------------------------------------------------
  Sort
    Sort Key: t1.c, t3.c
    ->  HashAggregate
@@ -1215,50 +2928,1166 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, pl
          ->  Result
                ->  Append
                      ->  Hash Join
-                           Hash Cond: (t1.c = t2.c)
-                           ->  Seq Scan on plt1_p1 t1
+                           Hash Cond: ((t1.c)::text = (t2.c)::text)
+                           ->  Hash Join
+                                 Hash Cond: ((t1.c)::text = ltrim(t3.c, 'A'::text))
+                                 ->  Seq Scan on plt1_p4 t1
+                                 ->  Hash
+                                       ->  Seq Scan on plt1_e_p4 t3
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p4 t2
+                     ->  Hash Join
+                           Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+                           ->  Hash Join
+                                 Hash Cond: ((t2_1.c)::text = ltrim(t3_1.c, 'A'::text))
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Hash
+                                       ->  Seq Scan on plt1_e_p1 t3_1
                            ->  Hash
-                                 ->  Hash Join
-                                       Hash Cond: (t2.c = ltrim(t3.c, 'A'::text))
-                                       ->  Seq Scan on plt2_p1 t2
-                                       ->  Hash
-                                             ->  Seq Scan on plt1_e_p1 t3
+                                 ->  Seq Scan on plt1_p1 t1_1
                      ->  Hash Join
-                           Hash Cond: (t1_1.c = t2_1.c)
-                           ->  Seq Scan on plt1_p2 t1_1
+                           Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+                           ->  Hash Join
+                                 Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
+                                 ->  Seq Scan on plt1_p2 t1_2
+                                 ->  Hash
+                                       ->  Seq Scan on plt1_e_p2 t3_2
                            ->  Hash
-                                 ->  Hash Join
-                                       Hash Cond: (t2_1.c = ltrim(t3_1.c, 'A'::text))
-                                       ->  Seq Scan on plt2_p2 t2_1
-                                       ->  Hash
-                                             ->  Seq Scan on plt1_e_p2 t3_1
+                                 ->  Seq Scan on plt2_p2 t2_2
                      ->  Hash Join
-                           Hash Cond: (t1_2.c = t2_2.c)
-                           ->  Seq Scan on plt1_p3 t1_2
+                           Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+                           ->  Hash Join
+                                 Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                                 ->  Seq Scan on plt1_p3 t1_3
+                                 ->  Hash
+                                       ->  Seq Scan on plt1_e_p3 t3_3
                            ->  Hash
-                                 ->  Hash Join
-                                       Hash Cond: (t2_2.c = ltrim(t3_2.c, 'A'::text))
-                                       ->  Seq Scan on plt2_p3 t2_2
-                                       ->  Hash
-                                             ->  Seq Scan on plt1_e_p3 t3_2
-(33 rows)
+                                 ->  Seq Scan on plt2_p3 t2_3
+(42 rows)
 
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  74.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 124.0000000000000000 | 124.5000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 224.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 274.0000000000000000 | 274.5000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 374.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 424.0000000000000000 | 424.5000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 524.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 574.0000000000000000 | 574.5000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  HashAggregate
+                           Group Key: (t2.c)::text
+                           ->  Seq Scan on plt2_p4 t2
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p5 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_3
+                                 ->  Seq Scan on plt2_p3 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Result
+         ->  Append
+               ->  Hash Left Join
+                     Hash Cond: ((t2.c)::text = (t1.c)::text)
+                     Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Hash
+                           ->  Seq Scan on plt1_p4 t1
+               ->  Hash Left Join
+                     Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+                     Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Hash
+                           ->  Seq Scan on plt1_p1 t1_1
+               ->  Hash Left Join
+                     Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+                     Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Hash
+                           ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash Right Join
+                     Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+                     Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+                     ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt2_p3 t2_3
+(28 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Seq Scan on plt2_p2 t2_2
+                                 ->  Seq Scan on plt2_p3 t2_3
+(22 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1286,16 +4115,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
@@ -1306,16 +4141,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 --
 -- multi-leveled partitions
@@ -1341,7 +4182,7 @@ CREATE TABLE prt2_l_p3_p2 PARTITION OF prt2_l_p3 FOR VALUES FROM (13) TO (25);
 INSERT INTO prt2_l SELECT i % 25, i, to_char(i % 4, 'FM0000') FROM generate_series(0, 599, 3) i;
 ANALYZE prt2_l;
 -- inner join, qual covering only top-level partitions
-EXPLAIN(COSTS OFF)
+EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1, prt2_l t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
                          QUERY PLAN                          
 -------------------------------------------------------------
@@ -1386,7 +4227,7 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1, prt2_l t2 WHERE t1.a = t2.b AND t1
 (4 rows)
 
 -- left join
-EXPLAIN(COSTS OFF)
+EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1 LEFT JOIN prt2_l t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t1.b = 0 ORDER BY t1.a, t2.b;
                                      QUERY PLAN                                     
 ------------------------------------------------------------------------------------
@@ -1440,7 +4281,7 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1 LEFT JOIN prt2_l t2 ON t1.a = t2.b
 (12 rows)
 
 -- right join
-EXPLAIN(COSTS OFF)
+EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1 RIGHT JOIN prt2_l t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.a = 0 ORDER BY t1.a, t2.b;
                                         QUERY PLAN                                        
 ------------------------------------------------------------------------------------------
@@ -1491,7 +4332,7 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1 RIGHT JOIN prt2_l t2 ON t1.a = t2.b
 (8 rows)
 
 -- full join
-EXPLAIN(COSTS OFF)
+EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.b = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.a = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
                                                      QUERY PLAN                                                     
 --------------------------------------------------------------------------------------------------------------------
@@ -1552,7 +4393,7 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.b = 0) t1
 (16 rows)
 
 -- lateral partition-wise join
-EXPLAIN(COSTS OFF)
+EXPLAIN (COSTS OFF)
 SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.b AS t3b, least(t1.a,t2.a,t3.b) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.c = t3.c)) ss
 			  ON t1.a = ss.t2a AND t1.c = ss.t2c WHERE t1.b = 0 ORDER BY t1.a;
@@ -1626,7 +4467,7 @@ SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
 (12 rows)
 
 -- join with one side empty
-EXPLAIN(COSTS OFF)
+EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a = 1 AND a = 2) t1 RIGHT JOIN prt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c;
                                QUERY PLAN                                
 -------------------------------------------------------------------------
@@ -1676,64 +4517,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
  Hash Join
    Hash Cond: (t1.a = t2.a)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt4_n_p1 t2
                ->  Seq Scan on prt4_n_p2 t2_1
                ->  Seq Scan on prt4_n_p3 t2_2
-(11 rows)
+(13 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a and t1.a = t3.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                        QUERY PLAN                        
+----------------------------------------------------------
  Hash Join
-   Hash Cond: (t2.a = t1.a)
+   Hash Cond: (t3.b = t1.a)
    ->  Append
-         ->  Seq Scan on prt4_n_p1 t2
-         ->  Seq Scan on prt4_n_p2 t2_1
-         ->  Seq Scan on prt4_n_p3 t2_2
+         ->  Seq Scan on prt2_p0 t3
+         ->  Seq Scan on prt2_p1 t3_1
+         ->  Seq Scan on prt2_p2 t3_2
+         ->  Seq Scan on prt2_p3 t3_3
+         ->  Seq Scan on prt2_p4 t3_4
+         ->  Seq Scan on prt2_p5 t3_5
    ->  Hash
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.a = t3_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_2.b)
-                     ->  Seq Scan on prt1_p3 t1_2
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3 t3_2
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt4_n_p1 t2
+                           ->  Seq Scan on prt4_n_p2 t2_1
+                           ->  Seq Scan on prt4_n_p3 t2_2
 (23 rows)
 
 -- partition-wise join can not be applied if there are no equi-join conditions
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (t1.a < b)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (t1.a < b)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (t1.a < b)
-(12 rows)
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+(16 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partition-wise join
@@ -1819,16 +4666,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
-                     ->  Seq Scan on plt1_p1 t3
-                     ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3
+                     ->  Seq Scan on plt1_p1 t3_1
+                     ->  Seq Scan on plt1_p2 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partition-wise join can not be applied for a join between list and range
 -- partitioned table
@@ -1839,12 +4687,14 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
  Hash Full Join
    Hash Cond: ((t2.c)::text = (t1.c)::text)
    ->  Append
-         ->  Seq Scan on prt1_p1 t2
-         ->  Seq Scan on prt1_p2 t2_1
-         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p0 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p2 t2_2
+         ->  Seq Scan on prt1_p3 t2_3
+         ->  Seq Scan on prt1_p4 t2_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt1_n_p1 t1
                ->  Seq Scan on prt1_n_p2 t1_1
-(10 rows)
+(12 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index cd54ea0..8f099a8 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -10,25 +10,43 @@ SET enable_partition_wise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
 INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 799, 3) i;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -77,11 +95,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a);
+SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a);
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -103,20 +129,30 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 
 EXPLAIN (COSTS OFF)
@@ -168,6 +204,104 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- TODO: below query is picking partition-wise-join which is not expected
+-- also generating wrong output, need to remove comment after fix
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+SET enable_partition_wise_join to false;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a;
+SET enable_partition_wise_join to true;
+
+-- TODO: below query is picking partition-wise-join which is not expected
+-- also generating wrong output, need to remove comment after fix
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+SET enable_partition_wise_join to false;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+SET enable_partition_wise_join to true;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- TODO: below query is generating wrong output, need to remove comment after fix
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+SET enable_partition_wise_join to false;
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+SET enable_partition_wise_join to true;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -192,28 +326,79 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
 -- test partition matching with N-way join
@@ -221,6 +406,175 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
@@ -260,27 +614,27 @@ INSERT INTO prt2_l SELECT i % 25, i, to_char(i % 4, 'FM0000') FROM generate_seri
 ANALYZE prt2_l;
 
 -- inner join, qual covering only top-level partitions
-EXPLAIN(COSTS OFF)
+EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1, prt2_l t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1, prt2_l t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
 
 -- left join
-EXPLAIN(COSTS OFF)
+EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1 LEFT JOIN prt2_l t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t1.b = 0 ORDER BY t1.a, t2.b;
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1 LEFT JOIN prt2_l t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t1.b = 0 ORDER BY t1.a, t2.b;
 
 -- right join
-EXPLAIN(COSTS OFF)
+EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1 RIGHT JOIN prt2_l t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.a = 0 ORDER BY t1.a, t2.b;
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1 RIGHT JOIN prt2_l t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.a = 0 ORDER BY t1.a, t2.b;
 
 -- full join
-EXPLAIN(COSTS OFF)
+EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.b = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.a = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.b = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.a = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
 
 -- lateral partition-wise join
-EXPLAIN(COSTS OFF)
+EXPLAIN (COSTS OFF)
 SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.b AS t3b, least(t1.a,t2.a,t3.b) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.c = t3.c)) ss
 			  ON t1.a = ss.t2a AND t1.c = ss.t2c WHERE t1.b = 0 ORDER BY t1.a;
@@ -289,7 +643,7 @@ SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
 			  ON t1.a = ss.t2a AND t1.c = ss.t2c WHERE t1.b = 0 ORDER BY t1.a;
 
 -- join with one side empty
-EXPLAIN(COSTS OFF)
+EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a = 1 AND a = 2) t1 RIGHT JOIN prt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c;
 
 --
#5Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Rajkumar Raghuwanshi (#4)
3 attachment(s)
Re: advanced partition matching algorithm for partition-wise join

I have applied v27 patches and tested feature. Also tried to reduce
regression diff with the
existing partition_join.sql by adding new partition instead of changing
original partition bounds.

Attached WIP patch have a server crash and some wrong output which need to
be fixed. I have
commented these issue in patch itself, Please take a look and let me know if
it need more
changes.

I have fixed the issues which were marked as TODOs in the attached
patches. Also, I have included your test change patch in my series of
patches. Are there any other issues you have commented out?

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

Attachments:

0011-Modify-bound-comparision-functions-to-accept-members.patchtext/x-patch; charset=US-ASCII; name=0011-Modify-bound-comparision-functions-to-accept-members.patchDownload
From b1888b037362acbc83fa00feba58624ecf62a6b9 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Thu, 6 Jul 2017 14:15:22 +0530
Subject: [PATCH 11/13] Modify bound comparision functions to accept members
 of PartitionKey

Functions partition_bound_cmp(), partition_rbound_cmp() and
partition_rbound_datum_cmp() are required to merge partition bounds
from joining relations. While doing so, we do not have access to the
PartitionKey of either relations. So, modify these functions to accept
only required members of PartitionKey so that the functions can be
reused for merging bounds.

Ashutosh Bapat.
---
 src/backend/catalog/partition.c |   76 ++++++++++++++++++++++-----------------
 1 file changed, 44 insertions(+), 32 deletions(-)

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 96a64ce..d42e1b5 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -126,15 +126,17 @@ static List *generate_partition_qual(Relation rel);
 
 static PartitionRangeBound *make_one_range_bound(PartitionKey key, int index,
 					 List *datums, bool lower);
-static int32 partition_rbound_cmp(PartitionKey key,
-					 Datum *datums1, PartitionRangeDatumKind *kind1,
-					 bool lower1, PartitionRangeBound *b2);
-static int32 partition_rbound_datum_cmp(PartitionKey key,
-						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
+static int32 partition_rbound_cmp(int partnatts, FmgrInfo *partsupfunc,
+					 Oid *partcollation, Datum *datums1,
+					 PartitionRangeDatumKind *kind1, bool lower1,
+					 PartitionRangeBound *b2);
+static int32 partition_rbound_datum_cmp(int partnatts, FmgrInfo *partsupfunc,
+						   Oid *partcollation, Datum *rb_datums,
+						   PartitionRangeDatumKind *rb_kind,
 						   Datum *tuple_datums);
 
-static int32 partition_bound_cmp(PartitionKey key,
-					PartitionBoundInfo boundinfo,
+static int32 partition_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+					Oid *partcollation, PartitionBoundInfo boundinfo,
 					int offset, void *probe, bool probe_is_bound);
 static int partition_bound_bsearch(PartitionKey key,
 						PartitionBoundInfo boundinfo,
@@ -719,8 +721,9 @@ check_new_partition_bound(char *relname, Relation parent,
 				 * First check if the resulting range would be empty with
 				 * specified lower and upper bounds
 				 */
-				if (partition_rbound_cmp(key, lower->datums, lower->kind, true,
-										 upper) >= 0)
+				if (partition_rbound_cmp(key->partnatts, key->partsupfunc,
+										 key->partcollation, lower->datums,
+										 lower->kind, true, upper) >= 0)
 				{
 					ereport(ERROR,
 							(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
@@ -771,9 +774,11 @@ check_new_partition_bound(char *relname, Relation parent,
 						{
 							int32		cmpval;
 
-							cmpval = partition_bound_cmp(key, boundinfo,
-														 offset + 1, upper,
-														 true);
+							cmpval = partition_bound_cmp(key->partnatts,
+														 key->partsupfunc,
+														 key->partcollation,
+														 boundinfo, offset + 1,
+														 upper, true);
 							if (cmpval < 0)
 							{
 								/*
@@ -2138,7 +2143,9 @@ qsort_partition_rbound_cmp(const void *a, const void *b, void *arg)
 	PartitionRangeBound *b2 = (*(PartitionRangeBound *const *) b);
 	PartitionKey key = (PartitionKey) arg;
 
-	return partition_rbound_cmp(key, b1->datums, b1->kind, b1->lower, b2);
+	return partition_rbound_cmp(key->partnatts, key->partsupfunc,
+								key->partcollation, b1->datums, b1->kind,
+								b1->lower, b2);
 }
 
 /*
@@ -2155,7 +2162,7 @@ qsort_partition_rbound_cmp(const void *a, const void *b, void *arg)
  * two contiguous partitions.
  */
 static int32
-partition_rbound_cmp(PartitionKey key,
+partition_rbound_cmp(int partnatts, FmgrInfo *partsupfunc, Oid *partcollation,
 					 Datum *datums1, PartitionRangeDatumKind *kind1,
 					 bool lower1, PartitionRangeBound *b2)
 {
@@ -2165,7 +2172,7 @@ partition_rbound_cmp(PartitionKey key,
 	PartitionRangeDatumKind *kind2 = b2->kind;
 	bool		lower2 = b2->lower;
 
-	for (i = 0; i < key->partnatts; i++)
+	for (i = 0; i < partnatts; i++)
 	{
 		/*
 		 * First, handle cases where the column is unbounded, which should not
@@ -2186,8 +2193,8 @@ partition_rbound_cmp(PartitionKey key,
 			 */
 			break;
 
-		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[i],
-												 key->partcollation[i],
+		cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[i],
+												 partcollation[i],
 												 datums1[i],
 												 datums2[i]));
 		if (cmpval != 0)
@@ -2213,22 +2220,23 @@ partition_rbound_cmp(PartitionKey key,
  * is <, =, or > partition key of tuple (tuple_datums)
  */
 static int32
-partition_rbound_datum_cmp(PartitionKey key,
-						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
+partition_rbound_datum_cmp(int partnatts, FmgrInfo *partsupfunc,
+						   Oid *partcollation, Datum *rb_datums,
+						   PartitionRangeDatumKind *rb_kind,
 						   Datum *tuple_datums)
 {
 	int			i;
 	int32		cmpval = -1;
 
-	for (i = 0; i < key->partnatts; i++)
+	for (i = 0; i < partnatts; i++)
 	{
 		if (rb_kind[i] == PARTITION_RANGE_DATUM_MINVALUE)
 			return -1;
 		else if (rb_kind[i] == PARTITION_RANGE_DATUM_MAXVALUE)
 			return 1;
 
-		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[i],
-												 key->partcollation[i],
+		cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[i],
+												 partcollation[i],
 												 rb_datums[i],
 												 tuple_datums[i]));
 		if (cmpval != 0)
@@ -2245,17 +2253,18 @@ partition_rbound_datum_cmp(PartitionKey key,
  * specified in *probe.
  */
 static int32
-partition_bound_cmp(PartitionKey key, PartitionBoundInfo boundinfo,
-					int offset, void *probe, bool probe_is_bound)
+partition_bound_cmp(int partnatts, FmgrInfo *partsupfunc, Oid *partcollation,
+					PartitionBoundInfo boundinfo, int offset, void *probe,
+					bool probe_is_bound)
 {
 	Datum	   *bound_datums = boundinfo->datums[offset];
 	int32		cmpval = -1;
 
-	switch (key->strategy)
+	switch (boundinfo->strategy)
 	{
 		case PARTITION_STRATEGY_LIST:
-			cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
-													 key->partcollation[0],
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
 													 bound_datums[0],
 													 *(Datum *) probe));
 			break;
@@ -2273,12 +2282,14 @@ partition_bound_cmp(PartitionKey key, PartitionBoundInfo boundinfo,
 					 */
 					bool		lower = boundinfo->indexes[offset] < 0;
 
-					cmpval = partition_rbound_cmp(key,
-												  bound_datums, kind, lower,
+					cmpval = partition_rbound_cmp(partnatts, partsupfunc,
+												  partcollation, bound_datums,
+												  kind, lower,
 												  (PartitionRangeBound *) probe);
 				}
 				else
-					cmpval = partition_rbound_datum_cmp(key,
+					cmpval = partition_rbound_datum_cmp(partnatts, partsupfunc,
+														partcollation,
 														bound_datums, kind,
 														(Datum *) probe);
 				break;
@@ -2286,7 +2297,7 @@ partition_bound_cmp(PartitionKey key, PartitionBoundInfo boundinfo,
 
 		default:
 			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) key->strategy);
+				 (int) boundinfo->strategy);
 	}
 
 	return cmpval;
@@ -2320,7 +2331,8 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
 		int32		cmpval;
 
 		mid = (lo + hi + 1) / 2;
-		cmpval = partition_bound_cmp(key, boundinfo, mid, probe,
+		cmpval = partition_bound_cmp(key->partnatts, key->partsupfunc,
+									 key->partcollation, boundinfo, mid, probe,
 									 probe_is_bound);
 		if (cmpval <= 0)
 		{
-- 
1.7.9.5

0012-WIP-Partition-wise-join-for-1-1-1-0-0-1-partition-ma.patchtext/x-patch; charset=US-ASCII; name=0012-WIP-Partition-wise-join-for-1-1-1-0-0-1-partition-ma.patchDownload
From f7c7b3817d733a78dad03ee89f86179f686d7c06 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Wed, 9 Aug 2017 12:30:34 +0530
Subject: [PATCH 12/13] WIP Partition-wise join for 1:1, 1:0, 0:1 partition
 matching.

Earlier version of partition-wise join implementation allowed
partition-wise join between two relations with exactly same partition
bounds. This commit allows partition-wise join to be applied under
following conditions

1. the partition bounds of joining relations are such that rows from
given partition on one side join can join with rows from maximum one
partition on the other side i.e. bounds of a given partition on one
side match/overlap with those of maximum one partition on the other
side. If the mapping happens to be m:n where m > 1 or n > 1, we have
to gang multiple partition relations together into a single relation.
This means that we have to add simple relations during join
processing, something which is not supported right now.  ALso, in such
a case, different pairs of joining relations can produce different
partition bounds for the same join relation, which again is not
supported right now.

2. For every partition on outer side that can contribute to the result
of an OUTER side, there exists at least one (taken along with item 1,
it means exactly one)  matching partition on the inner side. To
support partition-wise join when the inner matching partition doesn't
exist, we have to add a dummy base relation corresponding to the
non-existent inner partition. We don't have support add base relations
during join processing.

This commit is not complete yet.

Ashutosh Bapat.
---
 src/backend/catalog/partition.c       | 1258 +++++++++++++++++++++++++++++++++
 src/backend/optimizer/path/joinrels.c |   77 +-
 src/backend/optimizer/util/relnode.c  |   42 +-
 src/include/catalog/partition.h       |    6 +
 4 files changed, 1352 insertions(+), 31 deletions(-)

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index d42e1b5..94e48bd 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -141,6 +141,38 @@ static int32 partition_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
 static int partition_bound_bsearch(PartitionKey key,
 						PartitionBoundInfo boundinfo,
 						void *probe, bool probe_is_bound, bool *is_equal);
+static PartitionBoundInfo partition_range_bounds_merge(int partnatts,
+							 FmgrInfo *supfuncs, Oid *collations,
+							 PartitionBoundInfo boundinfo1, int nparts1,
+							 PartitionBoundInfo boundinfo2, int nparts2,
+							 JoinType jointype, List **parts1, List **parts2);
+static PartitionBoundInfo partition_list_bounds_merge(int partnatts,
+							FmgrInfo *partsupfunc, Oid *collations,
+							PartitionBoundInfo boundinfo1, int nparts1,
+							PartitionBoundInfo boundinfo2, int nparts2,
+							JoinType jointype, List **parts1, List **parts2);
+static void generate_matching_part_pairs(int *mergemap1, int npart1,
+							 int *mergemap2, int nparts2, JoinType jointype,
+							 int nparts, List **parts1, List **parts2);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index);
+static int map_and_merge_partitions(int *partmap1, int *mergemap1, int index1,
+						 int *partmap2, int *mergemap2, int index2,
+						 int *next_index);
+static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *collations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2);
+static int partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
+						   Oid *collations, PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, bool *overlap);
+static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes);
 
 /*
  * RelationBuildPartitionDesc
@@ -2348,3 +2380,1229 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
 
 	return lo;
 }
+
+/*
+ * Merge the given partition bounds.
+ *
+ * If given partition bounds can not be merged, return NULL.
+ *
+ * The function also returns two lists of partition indexes one for each of the
+ * joining relations. Both the lists contain the same number of elements. The
+ * partition indexes at the same positions in the list indicate partitions from
+ * each side to be joined and their position corresponds to the index of
+ * partition to which the results of the child-join belong in the partitioned
+ * join.
+ */
+extern PartitionBoundInfo
+partition_bounds_merge(int partnatts, FmgrInfo *partsupfunc, Oid *partcollation,
+					   PartitionBoundInfo boundinfo1, int nparts1,
+					   PartitionBoundInfo boundinfo2, int nparts2,
+					   JoinType jointype, List **parts1, List **parts2)
+{
+	PartitionBoundInfo merged_bounds;
+	char		strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (boundinfo1->strategy != boundinfo2->strategy)
+		return NULL;
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+	strategy = boundinfo1->strategy;
+	if (strategy == PARTITION_STRATEGY_LIST)
+		merged_bounds = partition_list_bounds_merge(partnatts, partsupfunc,
+													partcollation, boundinfo1,
+													nparts1, boundinfo2,
+													nparts2, jointype, parts1,
+													parts2);
+	else if (strategy == PARTITION_STRATEGY_RANGE)
+		merged_bounds = partition_range_bounds_merge(partnatts, partsupfunc,
+													 partcollation, boundinfo1,
+													 nparts1, boundinfo2,
+													 nparts2, jointype, parts1,
+													 parts2);
+	else
+		elog(ERROR, "unexpected partition strategy: %d", strategy);
+
+	Assert(merged_bounds || (*parts1 == NIL && *parts2 == NIL));
+	return merged_bounds;
+}
+
+/*
+ * partition_get_range_bounds
+ *
+ * Given the index of lower bound in datums array, return lower and upper
+ * bounds and the index of the partition with that lower bound.
+ */
+static int
+partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
+						   PartitionRangeBound *lower,
+						   PartitionRangeBound *upper)
+{
+	int			part_index;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The lower bound should correspond to a valid partition. */
+	part_index = bi->indexes[lb_index + 1];
+	Assert(part_index >= 0);
+
+	lower->kind = bi->kind[lb_index];
+	lower->datums = bi->datums[lb_index];
+	lower->lower = true;
+	upper->kind = bi->kind[lb_index + 1];
+	upper->datums = bi->datums[lb_index + 1];
+	upper->lower = false;
+
+	return part_index;
+}
+
+/*
+ * partition_range_get_next_lb_index
+ *
+ * Given the index of lower bound in datums array return the
+ * index of lower bound of the next partition. When the given index corresponds
+ * to the last partition, return number of datums (ndatums).
+ */
+static int
+partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
+{
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The partition index corresponding to the upper bound should be valid. */
+	Assert(bi->indexes[lb_index + 1] >= 0);
+
+	/*
+	 * If there are no bounds left beyond the upper bound, we have reached the
+	 * last partition.
+	 */
+	if (lb_index + 2 < bi->ndatums)
+	{
+		/*
+		 * If the bound next to the upper bound corresponds to no partition,
+		 * that's the next lower bound of the next partition. Otherwise, the
+		 * current upper bound is the lower bound of the next partition.
+		 */
+		if (bi->indexes[lb_index + 2] < 0)
+			return lb_index + 2;
+		else
+			return lb_index + 1;
+	}
+	else
+		return bi->ndatums;
+}
+
+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc, Oid *collations,
+						  PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2)
+{
+	return partition_rbound_cmp(partnatts, partsupfunc, collations,
+								bound1->datums, bound1->kind, bound1->lower,
+								bound2);
+}
+
+/*
+ * partition_range_cmp
+ *
+ * Compare the bounds of two range partitions and return <, = or > 0, if the
+ * first partition's upper bound is lower than, equal to or higher than the
+ * second partition's upper bound resp.
+ *
+ * Also, set overlaps to true, if the ranges overlap, otherwise set it to
+ * false.
+ */
+static int
+partition_range_cmp(int partnatts, FmgrInfo *supfuncs, Oid *collations,
+						   PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, bool *overlap)
+{
+	/*
+	 * Compare upper bound of the first partition with the lower bound of the
+	 * second and vice-versa. If lower bound is higher than the upper bound,
+	 * the partitions are not overlapping. All other cases indicate overlapping
+	 * partitions.
+	 * TODO: Add a testcase which has lower and upper bound matching exactly.
+	 * Lower bound is inclusive and upper bound is exclusive, so even if the
+	 * datums match, the bounds do not match exactly.
+	 */
+	if (partition_range_bound_cmp(partnatts, supfuncs, collations,
+								  lower_bound1, upper_bound2) > 0)
+	{
+		*overlap = false;
+		return 1;
+	}
+	else if (partition_range_bound_cmp(partnatts, supfuncs, collations,
+									   lower_bound2, upper_bound1) > 0)
+	{
+		*overlap = false;
+		return -1;
+	}
+	else
+	{
+		*overlap = true;
+		return partition_range_bound_cmp(partnatts, supfuncs, collations,
+										 upper_bound1, upper_bound2);
+	}
+}
+
+/*
+ * partition_range_merge
+ *
+ * Merge the partition bounds of given two partitions such that the join
+ * between the given two partitions fits merged bounds.
+ *
+ * "merged_upper" will be set to one of the given upper bounds and
+ * "merged_lower" will be set to one of the given lower bounds.
+ */
+static void
+partition_range_merge(int partnatts, FmgrInfo *supfuncs,
+							 Oid *collations, JoinType jointype,
+							 PartitionRangeBound *left_lb,
+							 PartitionRangeBound *left_ub,
+							 PartitionRangeBound *right_lb,
+							 PartitionRangeBound *right_ub,
+							 PartitionRangeBound **merged_lb,
+							 PartitionRangeBound **merged_ub)
+{
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = left_ub;
+			*merged_lb = left_lb;
+			break;
+
+		case JOIN_RIGHT:
+			*merged_ub = right_ub;
+			*merged_lb = right_lb;
+			break;
+
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			if (partition_range_bound_cmp(partnatts, supfuncs, collations,
+										  left_ub, right_ub) < 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, supfuncs, collations,
+										  left_lb, right_lb) > 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		case JOIN_FULL:
+			if (partition_range_bound_cmp(partnatts, supfuncs, collations,
+										  left_ub, right_ub) > 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, supfuncs, collations,
+										  left_lb, right_lb) < 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		default:
+			elog(ERROR, "Unexpected join type %d", jointype);
+	}
+
+	return;
+}
+
+/*
+ * Add the lower bound of the next range to the list of bounds, if the lower
+ * bound is higher or equal to the previous upper bound. If successful return
+ * true, otherwise false.
+ */
+static bool
+partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		prev_ub.datums = llast(*merged_datums);
+		prev_ub.kind = llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		/*
+		 * TODO: explain why do we pass lower to be false for the next lower
+		 * bound.
+		 */
+		cmpval = partition_rbound_cmp(partnatts, supfuncs, collations,
+									  next_lb_datums, next_lb_kind, false,
+									  &prev_ub);
+	}
+
+	/*
+	 * The lower bound is lower than the last upper bound, thus does not fit
+	 * the bounds created so far and hence can not be merged with the existing
+	 * bounds.
+	 */
+	if (cmpval < 0)
+		return false;
+
+	/*
+	 * Add bounds of the new merged partition. If the next lower bound is
+	 * higher than the last upper bound, add new range with index
+	 * corresponding to the lower bound as -1. If the merged lower bound
+	 * is same as the last merged upper bound, the last upper bound will be
+	 * reused as the lower bound of the next range.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, next_lb_datums);
+		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	return true;
+}
+
+/*
+ * Merge given two range partition bounds.
+ *
+ * Work horse function for partition_bounds_merge() for range partitioned
+ * tables.
+ *
+ * TODO: for an anti-join, the caller is supposed to send the outer relation as
+ * left relation. May be we should rename left and right as inner and outer. We
+ * don't need to handle RIGHT joins in this function, so renaming them as outer
+ * and inner is fine.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(int partnatts, FmgrInfo *supfuncs, Oid *collations,
+							 PartitionBoundInfo left_bi, int left_nparts,
+							 PartitionBoundInfo right_bi, int right_nparts,
+							 JoinType jointype, List **left_parts, List **right_parts)
+{
+	int		   *left_pmap;
+	int		   *left_mmap;
+	int		   *right_pmap;
+	int		   *right_mmap;
+	int			cnt1;
+	int			cnt2;
+	int			left_part;
+	int			right_part;
+	PartitionBoundInfo merged_bounds = NULL;
+	bool		merged = true;	/* By default we ranges are merge-able. */
+	int			left_lb_index;
+	int			right_lb_index;
+	int			next_index;
+	int			cmpval;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	List	   *merged_kinds = NIL;
+
+	Assert(left_bi->strategy == right_bi->strategy &&
+		   left_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	*left_parts = NIL;
+	*right_parts = NIL;
+
+	/* Allocate and initialize partition maps. */
+	left_pmap = (int *) palloc(sizeof(int) * left_nparts);
+	left_mmap = (int *) palloc(sizeof(int) * left_nparts);
+	right_pmap = (int *) palloc(sizeof(int) * right_nparts);
+	right_mmap = (int *) palloc(sizeof(int) * right_nparts);
+
+	for (cnt1 = 0; cnt1 < left_nparts; cnt1++)
+	{
+		left_pmap[cnt1] = -1;
+		left_mmap[cnt1] = -1;
+	}
+	for (cnt2 = 0; cnt2 < right_nparts; cnt2++)
+	{
+		right_pmap[cnt2] = -1;
+		right_mmap[cnt2] = -1;
+	}
+
+	left_lb_index = 0;
+	right_lb_index = 0;
+	next_index = 0;
+	while (left_lb_index < left_bi->ndatums &&
+		   right_lb_index < right_bi->ndatums)
+	{
+		PartitionRangeBound left_lb;
+		PartitionRangeBound left_ub;
+		PartitionRangeBound right_lb;
+		PartitionRangeBound right_ub;
+		PartitionRangeBound *merged_lb = NULL;
+		PartitionRangeBound *merged_ub = NULL;
+		int			merged_index = -1;
+		bool		overlap;
+
+		/* Get the range bounds of the next partition. */
+		left_part = partition_get_range_bounds(left_bi, left_lb_index,
+											   &left_lb, &left_ub);
+		right_part = partition_get_range_bounds(right_bi, right_lb_index,
+												&right_lb, &right_ub);
+
+		cmpval = partition_range_cmp(partnatts, supfuncs, collations,
+									 &left_lb, &left_ub, &right_lb, &right_ub,
+									 &overlap);
+
+		if (overlap)
+		{
+			/* Overlapping ranges, try merging. */
+			partition_range_merge(partnatts, supfuncs, collations, jointype,
+								  &left_lb, &left_ub, &right_lb, &right_ub,
+								  &merged_lb, &merged_ub);
+			merged_index = map_and_merge_partitions(left_pmap, left_mmap,
+													left_part, right_pmap,
+													right_mmap, right_part,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				merged = false;
+				break;
+			}
+		}
+
+		if (cmpval == 0)
+		{
+			Assert(overlap);
+
+			/* Move to the next pair of partitions. */
+			left_lb_index = partition_range_get_next_lb_index(left_bi,
+															  left_lb_index);
+			right_lb_index = partition_range_get_next_lb_index(right_bi,
+															   right_lb_index);
+		}
+		else if (cmpval < 0)
+		{
+			/*
+			 * If the partition on the left was not mapped to any partition on
+			 * the right. Any row from that partition will not have a matching
+			 * row from the other relation. So the partition will be part of
+			 * the join if it's an anti-join or the left side is the outer
+			 * side.
+			 */
+			if (jointype == JOIN_INNER || jointype == JOIN_SEMI ||
+				jointype == JOIN_RIGHT)
+			{
+				/* Nothing to do. */
+			}
+			else if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+					 jointype == JOIN_ANTI)
+			{
+				if (left_mmap[left_part] < 0)
+				{
+					left_mmap[left_part] = next_index++;
+					merged_index = left_mmap[left_part];
+					merged_lb = &left_lb;
+					merged_ub = &left_ub;
+				}
+			}
+			else
+			{
+				/* Don't know what to do with other join types. Bail out. */
+				merged = false;
+			}
+
+			/* Move to the next partition on the left side. */
+			left_lb_index = partition_range_get_next_lb_index(left_bi,
+															  left_lb_index);
+		}
+		else
+		{
+			Assert(cmpval > 0);
+
+			/*
+			 * If the partition on the right was not mapped to any partition on
+			 * the left. Any row from that partition will not have a matching
+			 * row from the other relation. So the partition will be part of
+			 * the join if the right side is the outer side.
+			 */
+			if (jointype == JOIN_INNER || jointype == JOIN_SEMI ||
+				jointype == JOIN_LEFT || jointype == JOIN_ANTI)
+			{
+				/* Nothing to do. */
+			}
+			else if (jointype == JOIN_RIGHT || jointype == JOIN_FULL)
+			{
+				if (right_mmap[right_part] < 0)
+				{
+					right_mmap[right_part] = next_index++;
+					merged_index = right_mmap[right_part];
+					merged_lb = &right_lb;
+					merged_ub = &right_ub;
+				}
+			}
+			else
+			{
+				/* Don't know what to do with other join types. Bail out. */
+				merged = false;
+			}
+
+			/* Move to the next partition on the right side. */
+			right_lb_index = partition_range_get_next_lb_index(right_bi,
+															   right_lb_index);
+		}
+
+		if (!merged)
+			break;
+
+		/* A skipped partition is not added to merged bounds. */
+		if (merged_index < 0)
+			continue;
+
+		/*
+		 * We have a valid partition index for the next partition of join. The
+		 * partition should have valid range.
+		 */
+		Assert(merged_lb && merged_ub);
+
+		/* Try merging merged lower bound with the last upper bound. */
+		merged = partition_range_merge_next_lb(partnatts, supfuncs,
+											   collations, merged_lb->datums,
+											   merged_lb->kind, &merged_datums,
+											   &merged_kinds, &merged_indexes);
+		if (merged)
+		{
+			/* Add upper bound with the merged partition index. */
+			merged_datums = lappend(merged_datums, merged_ub->datums);
+			merged_kinds = lappend(merged_kinds, merged_ub->kind);
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+		}
+		else
+			break;
+	}
+
+	/*
+	 * We will run the above loop till we exhaust ranges of at least one side
+	 * unless we failed to merge the ranges.
+	 */
+	Assert (!merged || (left_lb_index >= left_bi->ndatums ||
+						right_lb_index >= right_bi->ndatums));
+
+	/*
+	 * Handle any remaining partition bounds.  If remaining partitions fall on
+	 * the inner side of the join, none of the rows in those partition are
+	 * going to be joined with any row on the outer side and hence those
+	 * partitions will not be part of the join result. Hence only consider the
+	 * remaining partitions on the outer side of the join.
+	 */
+	if (merged &&
+		((left_lb_index < left_bi->ndatums &&
+		  (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+		   jointype == JOIN_ANTI)) ||
+		 (right_lb_index < right_bi->ndatums &&
+		  (jointype == JOIN_RIGHT || jointype == JOIN_FULL))))
+	{
+		int			bound_index = -1;
+		PartitionBoundInfo rem_bi = NULL;
+		int		   *mmap = NULL;
+		int			part_index;
+		PartitionRangeBound rem_lb;
+		PartitionRangeBound rem_ub;
+
+		if (left_lb_index < left_bi->ndatums)
+		{
+			Assert(jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+				   jointype == JOIN_ANTI);
+			bound_index = left_lb_index;
+			rem_bi = left_bi;
+			mmap = left_mmap;
+		}
+		else if (right_lb_index < right_bi->ndatums)
+		{
+			Assert(jointype == JOIN_RIGHT || jointype == JOIN_FULL);
+			bound_index = right_lb_index;
+			rem_bi = right_bi;
+			mmap = right_mmap;
+		}
+		Assert((bound_index >= 0 && bound_index < rem_bi->ndatums) &&
+			   rem_bi && mmap);
+
+		/*
+		 * If the partition corresponding to this lower bound has been already
+		 * mapped to a merged partition, don't need to add it again. This may
+		 * happen if the range of the last partition on the inner side overlaps
+		 * with this partition's range and has upper bound lesser than upper
+		 * bound of this partition's range.
+		 */
+		part_index = partition_get_range_bounds(rem_bi, bound_index, &rem_lb,
+												&rem_ub);
+		Assert(part_index >= 0);
+		if (mmap[part_index] >= 0)
+			bound_index = partition_range_get_next_lb_index(rem_bi, bound_index);
+
+		/*
+		 * Merge lower bound of the next range with the upper bound of last
+		 * range.
+		 */
+		if (bound_index < rem_bi->ndatums)
+			merged = partition_range_merge_next_lb(partnatts, supfuncs,
+												   collations,
+												   rem_bi->datums[bound_index],
+												   rem_bi->kind[bound_index],
+												   &merged_datums,
+												   &merged_kinds,
+												   &merged_indexes);
+
+		/*
+		 * Rest of the bounds correspond to valid ranges so add them after
+		 * remapping their partitions as required.
+		 */
+		for (bound_index++; merged && bound_index < rem_bi->ndatums;
+			 bound_index++)
+		{
+			Datum	   *datums = rem_bi->datums[bound_index];
+			int			index = rem_bi->indexes[bound_index];
+			int			part_index;
+
+			/*
+			 * Add lower bounds with partition index -1 and assign a new
+			 * partition index to the upper bounds.
+			 */
+			if (index < 0)
+				part_index = index;
+			else
+			{
+				if (mmap[index] < 0)
+					mmap[index] = next_index++;
+				part_index = mmap[index];
+			}
+
+			merged_indexes = lappend_int(merged_indexes, part_index);
+			merged_datums = lappend(merged_datums, datums);
+			merged_kinds = lappend(merged_kinds,
+								   rem_bi->kind[bound_index]);
+		}
+	}
+
+	/* Create PartitionBoundInfo */
+	if (merged)
+	{
+		/* Use maps to match partition from the joining relations. */
+		generate_matching_part_pairs(left_mmap, left_nparts, right_mmap,
+									 right_nparts, jointype, next_index,
+									 left_parts, right_parts);
+
+		/* Craft a PartitionBoundInfo to return. */
+		if (*left_parts && *right_parts)
+		{
+			Assert(list_length(*left_parts) == list_length(*right_parts));
+			Assert(list_length(*left_parts) == next_index);
+			merged_bounds = build_merged_partition_bounds(left_bi->strategy,
+														  merged_datums,
+														  merged_indexes,
+														  merged_kinds,
+														  -1);
+		}
+	}
+
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	pfree(left_pmap);
+	pfree(right_pmap);
+	pfree(left_mmap);
+	pfree(right_mmap);
+
+	/* Free any memory we used in this function. */
+	return merged_bounds;
+}
+
+/*
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ * The function builds the maps of matching partitions from either relation. It
+ * builds the list of partition key values that may appear in the join result
+ * alongwith the list of indexes of partitions of join to which those values
+ * belong. It then crafts a PartitionBoundInfo structure representing the
+ * partition bounds of the join result.
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(int partnatts, FmgrInfo *partsupfunc,
+							Oid *partcollation, PartitionBoundInfo left_bi,
+							int left_nparts, PartitionBoundInfo right_bi,
+							int right_nparts, JoinType jointype, List **left_parts,
+							List **right_parts)
+{
+	int		   *left_pmap;	/* left to right partition map */
+	int		   *left_mmap;	/* left to merged partition map */
+	int		   *right_pmap;	/* right to left partition map */
+	int		   *right_mmap;	/* right to merged partition map */
+	int			cntl;
+	int			cntr;
+	bool		merged = true;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			next_index = 0;
+	int			null_index;
+	PartitionBoundInfo merged_bounds = NULL;
+	int		   *left_indexes = left_bi->indexes;
+	int		   *right_indexes = right_bi->indexes;
+	int			left_ni = left_bi->null_index;
+	int			right_ni = right_bi->null_index;
+
+	Assert(left_bi->strategy == right_bi->strategy &&
+		   left_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!left_bi->kind && !right_bi->kind);
+
+	/* Allocate and initialize partition maps. */
+	left_pmap = (int *) palloc(sizeof(int) * left_nparts);
+	left_mmap = (int *) palloc(sizeof(int) * left_nparts);
+	right_pmap = (int *) palloc(sizeof(int) * right_nparts);
+	right_mmap = (int *) palloc(sizeof(int) * right_nparts);
+
+	/* Initialize partition maps. */
+	for (cntl = 0; cntl < left_nparts; cntl++)
+	{
+		left_pmap[cntl] = -1;
+		left_mmap[cntl] = -1;
+	}
+	for (cntr = 0; cntr < right_nparts; cntr++)
+	{
+		right_pmap[cntr] = -1;
+		right_mmap[cntr] = -1;
+	}
+
+	cntl = cntr = 0;
+	while (cntl < left_bi->ndatums && cntr < right_bi->ndatums)
+	{
+		Datum	   *ldatums = left_bi->datums[cntl];
+		Datum	   *rdatums = right_bi->datums[cntr];
+		int			l_index = left_indexes[cntl];
+		int			r_index = right_indexes[cntr];
+		int			cmpval;
+		int			merged_index;
+		Datum	   *merged_datum;
+
+		/* Every list datum should map to a valid partition index. */
+		Assert(l_index >= 0 && r_index >= 0);
+
+		cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+												 partcollation[0], ldatums[0],
+												 rdatums[0]));
+		if (cmpval == 0)
+		{
+			/*
+			 * Try matching partitions containing the matching datums. If
+			 * successful, add the datum to the merged bounds with index of
+			 * merged partition containing it.
+			 */
+			merged_datum = ldatums;
+			merged_index = map_and_merge_partitions(left_pmap, left_mmap, l_index,
+													right_pmap, right_mmap, r_index,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				merged = false;
+				break;
+			}
+
+			/* Move to the next pair of bounds. */
+			cntl++;
+			cntr++;
+		}
+		else if (cmpval < 0)
+		{
+			/*
+			 * This list datum is present in the left side but not the right
+			 * side. So it will appear in the join when the left side is outer
+			 * side.
+			 */
+			if (jointype == JOIN_INNER || jointype == JOIN_RIGHT ||
+				jointype == JOIN_SEMI)
+				merged_index = -1;
+			else if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+					 jointype == JOIN_ANTI)
+			{
+				if (left_mmap[l_index] < 0)
+					left_mmap[l_index] = next_index++;
+				merged_index = left_mmap[l_index];
+				merged_datum = ldatums;
+			}
+			else
+			{
+				/* Don't know what to do with other join types. */
+				merged = false;
+				break;
+			}
+
+			/* Move to the next datum on the left side. */
+			cntl++;
+		}
+		else
+		{
+			Assert(cmpval > 0);
+
+			/*
+			 * This list datum is present in the right side but not the left
+			 * side. So it will appear in the join when the right side is outer
+			 * side.
+			 */
+			if (jointype == JOIN_INNER || jointype == JOIN_LEFT ||
+				jointype == JOIN_SEMI || jointype == JOIN_ANTI)
+				merged_index = -1;
+			else if (jointype == JOIN_RIGHT || jointype == JOIN_FULL)
+			{
+				/*
+				 * Every list value on the outer side will appear in the
+				 * join.  Find the merged partition to which this value
+				 * belongs.
+				 */
+				if (right_mmap[r_index] < 0)
+					right_mmap[r_index] = next_index++;
+				merged_index = right_mmap[r_index];
+				merged_datum = rdatums;
+			}
+			else
+			{
+				/* Don't know what to do with other join types. */
+				merged = false;
+				break;
+			}
+
+			/* Move to the next datum on the right side. */
+			cntr++;
+		}
+
+		/*
+		 * Add the datum with appropriate index in the list of datums, if the
+		 * rows containing that datum are deemed to be part of the join.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	/*
+	 * If merge is unsuccessful, bail out without any further processing.
+	 * That leaks the memory allocated in this function. So, try not to leak
+	 * memory.
+	 */
+	if (!merged)
+		goto merge_failed;
+
+	/* We should have exhausted datums on at least one side. */
+	Assert(cntr >= right_bi->ndatums || cntl >= left_bi->ndatums);
+
+	/*
+	 * Add any remaining list values on the outer side, assigning partition
+	 * indexes if required.
+	 */
+	if (jointype == JOIN_LEFT || jointype == JOIN_FULL || jointype == JOIN_ANTI)
+	{
+		for (;cntl < left_bi->ndatums; cntl++)
+		{
+			Datum	   *ldatums = left_bi->datums[cntl];
+			int			l_index = left_indexes[cntl];
+
+			if (left_mmap[l_index] < 0)
+				left_mmap[l_index] = next_index++;
+			merged_indexes = lappend_int(merged_indexes, left_mmap[l_index]);
+			merged_datums = lappend(merged_datums, ldatums);
+		}
+	}
+
+	if (jointype == JOIN_RIGHT || jointype == JOIN_FULL)
+	{
+		for (;cntr < right_bi->ndatums; cntr++)
+		{
+			Datum	   *rdatums = right_bi->datums[cntr];
+			int			r_index = right_indexes[cntr];
+
+			if (right_mmap[r_index] < 0)
+				right_mmap[r_index] = next_index++;
+			merged_indexes = lappend_int(merged_indexes, right_mmap[r_index]);
+			merged_datums = lappend(merged_datums, rdatums);
+		}
+	}
+
+	/*
+	 * Merge NULL partitions if any. Find the index of merged partition to
+	 * which the NULL values belong in the join result. We can eliminate a NULL
+	 * partition when it appears only in the inner relation.
+	 */
+	if (!partition_bound_accepts_nulls(left_bi) &&
+		!partition_bound_accepts_nulls(right_bi))
+		null_index = -1;
+	else if (partition_bound_accepts_nulls(left_bi) &&
+			 !partition_bound_accepts_nulls(right_bi))
+	{
+		if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+			jointype == JOIN_ANTI)
+		{
+			if (left_mmap[left_ni] < 0)
+				left_mmap[left_ni] = next_index++;
+			null_index = left_mmap[left_ni];
+		}
+		else
+			null_index = -1;
+	}
+	else if (!partition_bound_accepts_nulls(left_bi) &&
+			 partition_bound_accepts_nulls(right_bi))
+	{
+		if (jointype == JOIN_RIGHT || jointype == JOIN_FULL)
+		{
+			if (right_mmap[right_ni] < 0)
+				right_mmap[right_ni] = next_index++;
+			null_index = right_mmap[right_ni];
+		}
+		else
+			null_index = -1;
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		null_index = map_and_merge_partitions(left_pmap, left_mmap,
+											  left_ni, right_pmap,
+											  right_mmap, right_ni,
+											  &next_index);
+		if (null_index < 0)
+			merged = false;
+	}
+
+	/* If successful build the output structures. */
+	if (merged)
+	{
+
+		/* Use maps to match partition from the joining relations. */
+		generate_matching_part_pairs(left_mmap, left_nparts, right_mmap,
+									 right_nparts, jointype, next_index,
+									 left_parts, right_parts);
+		/* Craft a PartitionBoundInfo to return. */
+		if (*left_parts && *right_parts)
+		{
+			Assert(list_length(*left_parts) == list_length(*right_parts));
+			Assert(list_length(*left_parts) == next_index);
+			merged_bounds = build_merged_partition_bounds(left_bi->strategy,
+														  merged_datums,
+														  merged_indexes, NIL,
+														  null_index);
+		}
+	}
+
+merge_failed:
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	pfree(left_pmap);
+	pfree(right_pmap);
+	pfree(left_mmap);
+	pfree(right_mmap);
+
+	return merged_bounds;
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * If the two given partitions (given by index1 and index2 resp.) are already
+ * mapped to each other return the index of corresponding partition in the
+ * merged set of partitions.  If they do not have a merged partition associated
+ * with them, assign a new merged partition index.  If the partitions are
+ * already mapped and their mapped partitions are different from each other,
+ * they can not be merged, so return -1.
+ *
+ * partmap1[i] gives the partition of relation 2 which matches ith partition of
+ * relation 1. Similarly for partmap2.
+ *
+ * mergemap1[i] gives the partition in the merged set to which ith partition of
+ * relation 1 maps to. Similarly for mergemap2.
+ *
+ * index1 and index2 are the indexes of matching partition from respective
+ * relations.
+ *
+ * *next_index is used and incremented when the given partitions require a new
+ * merged partition.
+ */
+static int
+map_and_merge_partitions(int *partmap1, int *mergemap1, int index1,
+						 int *partmap2, int *mergemap2, int index2,
+						 int *next_index)
+{
+	int			merged_index;
+
+	/*
+	 * If both the partitions are not mapped to each other, update the
+	 * maps.
+	 */
+	if (partmap1[index1] < 0 && partmap2[index2] < 0)
+	{
+		partmap1[index1] = index2;
+		partmap2[index2] = index1;
+	}
+
+	/*
+	 * If the given to partitions map to each other, find the corresponding
+	 * merged partition index .
+	 */
+	if (partmap1[index1] == index2 && partmap2[index2] == index1)
+	{
+		/*
+		 * If both the partitions are mapped to the same merged partition, get
+		 * the index of merged partition.
+		 */
+		if (mergemap1[index1] == mergemap2[index2])
+		{
+			merged_index = mergemap1[index1];
+
+			/*
+			 * If the given two partitions do not have a merged partition
+			 * associated with them, allocate a new merged partition.
+			 */
+			if (merged_index < 0)
+			{
+				merged_index = *next_index;
+				*next_index = *next_index + 1;
+				mergemap1[index1] = merged_index;
+				mergemap2[index2] = merged_index;
+			}
+		}
+
+		/*
+		 * If partition from one relation was mapped to a merged partition but
+		 * not the partition from the other relation, map the same merged
+		 * partition to the partition from other relation, since matching
+		 * partitions map to the same merged partition.
+		 */
+		else if (mergemap1[index1] >= 0 && mergemap2[index2] < 0)
+		{
+			mergemap2[index2] = mergemap1[index1];
+			merged_index = mergemap1[index1];
+		}
+		else if (mergemap1[index1] < 0 && mergemap2[index2] >= 0)
+		{
+			mergemap1[index1] = mergemap2[index2];
+			merged_index = mergemap2[index2];
+		}
+		else
+		{
+			Assert(mergemap1[index1] != mergemap2[index2] &&
+				   mergemap1[index1] >= 0 && mergemap2[index2] >= 0);
+
+			/*
+			 * Both the partitions map to different merged partitions. This
+			 * means that multiple partitions from one relation matches to one
+			 * partition from the other relation. Partition-wise join does not
+			 * handle this case right now, since it requires ganging multiple
+			 * partitions together (into one RelOptInfo).
+			 */
+			merged_index = -1;
+		}
+	}
+	else
+	{
+		/*
+		 * Multiple partitions from one relation map to one partition from the
+		 * other relation. Partition-wise join does not handle this case right
+		 * now, since it requires ganging multiple partitions together (into
+		 * one RelOptInfo).
+		 */
+		merged_index = -1;
+	}
+
+	return merged_index;
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * Given the merged partition to which partition on either side of join map,
+ * produce the list pairs of partitions which when joined produce the merged
+ * partitions in the order of merged partition indexes.
+ *
+ * If successful, the pairs of partitions are returned as two separate lists
+ * one for each side. Otherwise, those lists will be set to NIL.
+ *
+ * TODO: rename the sides as outer and inner. You may not need to support
+ * JOIN_RIGHT, since we won't see that type here.
+ */
+static void
+generate_matching_part_pairs(int *mergemap1, int nparts1, int *mergemap2,
+							 int nparts2, JoinType jointype, int nparts,
+							 List **parts1, List **parts2)
+{
+	bool		merged = true;
+	int		  **matching_parts;
+	int			cnt1;
+	int			cnt2;
+
+	matching_parts = (int **) palloc(sizeof(int *) * 2);
+	matching_parts[0] = (int *) palloc(sizeof(int) * nparts);
+	matching_parts[1] = (int *) palloc(sizeof(int) * nparts);
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt1 = 0; cnt1 < nparts; cnt1++)
+	{
+		matching_parts[0][cnt1] = -1;
+		matching_parts[1][cnt1] = -1;
+	}
+
+	/* Set pairs of matching partitions. */
+	for (cnt1 = 0; cnt1 < nparts1; cnt1++)
+	{
+		if (mergemap1[cnt1] >= 0)
+		{
+			Assert(mergemap1[cnt1] < nparts);
+			matching_parts[0][mergemap1[cnt1]] = cnt1;
+		}
+	}
+	for (cnt2 = 0; cnt2 < nparts2; cnt2++)
+	{
+		if (mergemap2[cnt2] >= 0)
+		{
+			Assert(mergemap2[cnt2] < nparts);
+			matching_parts[1][mergemap2[cnt2]] = cnt2;
+		}
+	}
+
+	/*
+	 * If we have a partition missing on an inner side, we need to add a dummy
+	 * relation which joins with the outer partition. If the inner relation
+	 * happens to be a base relation, it will require adding a dummy child
+	 * base relation during join processing. Right now, we freeze the base
+	 * relation arrays like PlannerInfo::simple_rte_array after planning for
+	 * base relations. Adding a new (dummy) base relation would require some
+	 * changes to that. So, right now, we do not implement partition-wise join
+	 * in such cases.
+	 */
+	for (cnt1 = 0; cnt1 < nparts; cnt1++)
+	{
+		int			part1 = matching_parts[0][cnt1];
+		int			part2 = matching_parts[1][cnt1];
+
+		/* At least one of the partitions should exist. */
+		Assert(part1 >= 0 || part2 >= 0);
+
+		switch (jointype)
+		{
+			case JOIN_INNER:
+			case JOIN_SEMI:
+
+				/*
+				 * An inner or semi join can not return any row when the
+				 * matching partition on either side is missing. We should
+				 * have eliminated all such cases while merging the bounds.
+				 */
+				Assert(part1 >= 0 && part2 >= 0);
+				break;
+
+			case JOIN_LEFT:
+			case JOIN_ANTI:
+				Assert(part1 >= 0);
+				if (part2 < 0)
+					merged = false;
+				break;
+
+			case JOIN_RIGHT:
+				Assert(part2 >= 0);
+				if (part1 < 0)
+					merged = false;
+				break;
+
+			case JOIN_FULL:
+				if (part1 < 0 || part2 < 0)
+					merged = false;
+				break;
+
+			default:
+				/* We do not know what to do in this case. Bail out. */
+				merged = false;
+		}
+
+		if (!merged)
+			break;
+
+		*parts1 = lappend_int(*parts1, part1);
+		*parts2 = lappend_int(*parts2, part2);
+	}
+
+	pfree(matching_parts[0]);
+	pfree(matching_parts[1]);
+	pfree(matching_parts);
+
+	if (!merged)
+	{
+		list_free(*parts1);
+		list_free(*parts2);
+		*parts1 = NIL;
+		*parts2 = NIL;
+	}
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind = (PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+															   list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+
+	return merged_bounds;
+}
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index e3ac4de..2e39c55 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1308,8 +1308,13 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 						RelOptInfo *joinrel, SpecialJoinInfo *parent_sjinfo,
 						List *parent_restrictlist)
 {
-	int			nparts;
 	int			cnt_parts;
+	PartitionScheme part_scheme;
+	PartitionBoundInfo join_boundinfo;
+	List	   *parts1;
+	List	   *parts2;
+	ListCell   *lc1;
+	ListCell   *lc2;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
@@ -1359,39 +1364,54 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	 */
 	Assert(joinrel->part_scheme == rel1->part_scheme &&
 		   joinrel->part_scheme == rel2->part_scheme);
+	part_scheme = joinrel->part_scheme;
 
 	/*
-	 * Since we allow partition-wise join only when the partition bounds of
-	 * the joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * Get the list of matching partitions from both sides of the join. While
+	 * doing so, we also build the partition bounds of the join relation,
+	 * which should match the bounds calculated for other pairs. TODO: why
+	 * should every pair result in the same partition bounds?
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
-
-	nparts = joinrel->nparts;
-
+	join_boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											part_scheme->partsupfunc,
+											part_scheme->parttypcoll,
+											rel1->boundinfo, rel1->nparts,
+											rel2->boundinfo, rel2->nparts,
+											parent_sjinfo->jointype,
+											&parts1, &parts2);
+
+	Assert(join_boundinfo);
+	Assert(partition_bounds_equal(part_scheme->partnatts,
+								  part_scheme->parttyplen,
+								  part_scheme->parttypbyval, join_boundinfo,
+								  joinrel->boundinfo));
 	elog(DEBUG3, "join between relations %s and %s is considered for partition-wise join.",
 		 bmsToString(rel1->relids), bmsToString(rel2->relids));
 
-	/* Allocate space to hold child-joins RelOptInfos, if not already done. */
+	/*
+	 * Every pair of joining relations should result in the same number of
+	 * child-joins.
+	 */
+	Assert(joinrel->nparts == list_length(parts1));
+	Assert(joinrel->nparts == list_length(parts2));
+
+	/* Allocate space for hold child-joins RelOptInfos, if not already done. */
 	if (!joinrel->part_rels)
-		joinrel->part_rels = (RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+		joinrel->part_rels = (RelOptInfo **) palloc0(sizeof(RelOptInfo *) *
+													 joinrel->nparts);
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	cnt_parts = 0;
+	forboth(lc1, parts1, lc2, parts2)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
+		int			part1 = lfirst_int(lc1);
+		int			part2 = lfirst_int(lc2);
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1399,6 +1419,10 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		Assert(part1 >= 0 && part2 >= 0);
+		child_rel1 = rel1->part_rels[part1];
+		child_rel2 = rel2->part_rels[part2];
+
 		/* We should never try to join two overlapping sets of rels. */
 		Assert(!bms_overlap(child_rel1->relids, child_rel2->relids));
 		child_joinrelids = bms_union(child_rel1->relids, child_rel2->relids);
@@ -1431,6 +1455,15 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 			joinrel->part_rels[cnt_parts] = child_joinrel;
 		}
 
+		/*
+		 * For every pair of joining relations, the set of matching partitions
+		 * would change. However, the base relation partitions constituting
+		 * the given child should remain same for all the joining pairs. Since
+		 * the order in which children are stored in the array of child-joins,
+		 * depends upon partition bounds of the join, which are same for all
+		 * the joining pairs, every joining pair yields the child-joins in the
+		 * same order.
+		 */
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
 
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
@@ -1443,7 +1476,11 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		 */
 		try_partition_wise_join(root, child_rel1, child_rel2, child_joinrel,
 								child_sjinfo, child_restrictlist);
+
+		cnt_parts++;
 	}
+
+	Assert(cnt_parts == joinrel->nparts);
 }
 
 /*
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 81bc2b1..5d1992e 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1613,6 +1613,9 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	int			partnatts;
 	int			cnt;
 	PartitionScheme part_scheme;
+	PartitionBoundInfo join_boundinfo;
+	List	   *parts1;
+	List	   *parts2;
 
 	/* Nothing to do if partition-wise join technique is disabled. */
 	if (!enable_partition_wise_join)
@@ -1653,17 +1656,26 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 		   REL_HAS_ALL_PART_PROPS(inner_rel));
 
 	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
+	 * Every pair of joining relations would yield the same partition bounds
+	 * for a given join (TODO: why?) so we compute the bounds only the first
+	 * time. Then for every pair we find the pairs of matching partitions from
+	 * the joining relations and join those. TODO: Needs a better explanation
+	 * of why is this true.  TODO: Also there is no reason to have
+	 * part_indexes1 and part_indexes2 pulled here just to be freed up later.
+	 * So, we might want to do something better.
 	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
+	join_boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											part_scheme->partsupfunc,
+											part_scheme->parttypcoll,
+											outer_rel->boundinfo,
+											outer_rel->nparts,
+											inner_rel->boundinfo,
+											inner_rel->nparts,
+											jointype, &parts1, &parts2);
+	if (!join_boundinfo)
 	{
 		Assert(!IS_PARTITIONED_REL(joinrel));
+		Assert(!parts1 && !parts2);
 		return;
 	}
 
@@ -1676,13 +1688,16 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 		   !joinrel->nullable_partexprs && !joinrel->part_rels &&
 		   !joinrel->boundinfo);
 
+	Assert(list_length(parts1) == list_length(parts2));
+
 	/*
 	 * Join relation is partitioned using same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations. It will have as many partitions as the pairs of
+	 * matching partitions we found.
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
-	joinrel->nparts = outer_rel->nparts;
+	joinrel->nparts = list_length(parts1);
+	joinrel->boundinfo = join_boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 
 	/*
@@ -1803,4 +1818,9 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 			joinrel->nullable_partexprs[cnt] = nullable_partexprs;
 		}
 	}
+
+	/* TODO: OR we could actually create the child-join relations here.*/
+	list_free(parts1);
+	list_free(parts2);
+
 }
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 2283c67..056a4f9 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -99,4 +99,10 @@ extern int get_partition_for_tuple(PartitionDispatch *pd,
 						EState *estate,
 						PartitionDispatchData **failed_at,
 						TupleTableSlot **failed_slot);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   FmgrInfo *supfuncs, Oid *collations,
+					   PartitionBoundInfo boundinfo1, int nparts1,
+					   PartitionBoundInfo boundinfo2, int nparts2,
+					   JoinType jointype, List **parts1, List **parts2);
+
 #endif							/* PARTITION_H */
-- 
1.7.9.5

0013-Tests-for-0-1-1-1-and-1-0-partition-matching.patchtext/x-patch; charset=US-ASCII; name=0013-Tests-for-0-1-1-1-and-1-0-partition-matching.patchDownload
From 753c8981723c2ab5d9dce57257d5e1a7c85fa7b3 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Tue, 5 Sep 2017 09:51:41 +0530
Subject: [PATCH 13/13] Tests for 0:1, 1:1 and 1:0 partition matching

Rajkumar Raghuvanshi and Ashutosh Bapat.
---
 src/test/regress/expected/partition_join.out | 4102 +++++++++++++++++++++-----
 src/test/regress/sql/partition_join.sql      |  367 ++-
 2 files changed, 3752 insertions(+), 717 deletions(-)

diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index f9543c8..151d70e 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -8,105 +8,152 @@ SET enable_partition_wise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-(4 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+(7 rows)
 
 -- left outer join, with whole-row reference
 EXPLAIN (COSTS OFF)
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b
    ->  Result
          ->  Append
                ->  Hash Right Join
                      Hash Cond: (t2.b = t1.a)
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                      ->  Hash
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                                  Filter: (b = 0)
                ->  Hash Right Join
                      Hash Cond: (t2_1.b = t1_1.a)
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                                  Filter: (b = 0)
                ->  Hash Right Join
                      Hash Cond: (t2_2.b = t1_2.a)
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                                  Filter: (b = 0)
-(22 rows)
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (t1_3.a = b)
+               ->  Hash Right Join
+                     Hash Cond: (t2_4.b = t1_4.a)
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_p4 t1_4
+                                 Filter: (b = 0)
+(33 rows)
 
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-      t1      |      t2      
---------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
- (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
- (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
- (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
- (550,0,0550) | 
-(12 rows)
+       t1       |       t2       
+----------------+----------------
+ (-250,0,-0250) | 
+ (-200,0,-0200) | 
+ (-150,0,-0150) | (0,-150,-0150)
+ (-100,0,-0100) | 
+ (-50,0,-0050)  | 
+ (0,0,0000)     | (0,0,0000)
+ (50,0,0050)    | 
+ (100,0,0100)   | 
+ (150,0,0150)   | (0,150,0150)
+ (200,0,0200)   | 
+ (250,0,0250)   | 
+ (300,0,0300)   | (0,300,0300)
+ (350,0,0350)   | 
+ (400,0,0400)   | 
+ (450,0,0450)   | (0,450,0450)
+ (500,0,0500)   | 
+ (550,0,0550)   | 
+ (600,0,0600)   | (0,600,0600)
+ (650,0,0650)   | 
+ (700,0,0700)   | 
+ (750,0,0750)   | (0,750,0750)
+(21 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
@@ -119,35 +166,53 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
          ->  Append
                ->  Hash Right Join
                      Hash Cond: (t1.a = t2.b)
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                      ->  Hash
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                                  Filter: (a = 0)
                ->  Hash Right Join
                      Hash Cond: (t1_1.a = t2_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Hash
+                           ->  Seq Scan on prt2_p1 t2_1
+                                 Filter: (a = 0)
+               ->  Hash Right Join
+                     Hash Cond: (t1_2.a = t2_2.b)
+                     ->  Seq Scan on prt1_p2 t1_2
                      ->  Hash
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p2 t2_2
                                  Filter: (a = 0)
                ->  Nested Loop Left Join
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
                            Filter: (a = 0)
-                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                           Index Cond: (a = t2_2.b)
-(21 rows)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                           Index Cond: (a = t2_3.b)
+               ->  Hash Right Join
+                     Hash Cond: (t1_4.a = t2_4.b)
+                     ->  Seq Scan on prt1_p4 t1_4
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4 t2_4
+                                 Filter: (a = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+      |       | -225 | -0225
+      |       |  -75 | -0075
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(14 rows)
 
 -- full outer join
 EXPLAIN (COSTS OFF)
@@ -155,9 +220,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.b = 0) t1 FULL
                     QUERY PLAN                    
 --------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
          ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
+         ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                ->  Seq Scan on prt1_p1
                      Filter: (b = 0)
@@ -178,28 +250,47 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.b = 0) t1 FULL
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(24 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(38 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 | 150 | 0150
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
- 450 | 0450 | 450 | 0450
- 500 | 0500 |     | 
- 550 | 0550 |     | 
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(16 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -250 | -0250 |      | 
+ -200 | -0200 |      | 
+ -150 | -0150 | -150 | -0150
+ -100 | -0100 |      | 
+  -50 | -0050 |      | 
+    0 | 0000  |    0 | 0000
+   50 | 0050  |      | 
+  100 | 0100  |      | 
+  150 | 0150  |  150 | 0150
+  200 | 0200  |      | 
+  250 | 0250  |      | 
+  300 | 0300  |  300 | 0300
+  350 | 0350  |      | 
+  400 | 0400  |      | 
+  450 | 0450  |  450 | 0450
+  500 | 0500  |      | 
+  550 | 0550  |      | 
+  600 | 0600  |  600 | 0600
+  650 | 0650  |      | 
+  700 | 0700  |      | 
+  750 | 0750  |  750 | 0750
+      |       | -225 | -0225
+      |       |  -75 | -0075
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(28 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -208,9 +299,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                             QUERY PLAN                            
 ------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
          ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               Filter: (((50) = prt1_p0.a) OR ((75) = prt2_p0.b))
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
+         ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
                ->  Seq Scan on prt1_p1
@@ -234,7 +333,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               Filter: (((50) = prt1_p4.a) OR ((75) = prt2_p4.b))
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
@@ -248,10 +355,17 @@ SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 W
                        QUERY PLAN                       
 --------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Result
          ->  Append
                ->  Hash Full Join
+                     Hash Cond: (prt1_p0.a = prt2_p0.b)
+                     ->  Seq Scan on prt1_p0
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p0
+                                 Filter: (a = 0)
+               ->  Hash Full Join
                      Hash Cond: (prt1_p1.a = prt2_p1.b)
                      ->  Seq Scan on prt1_p1
                            Filter: (b = 0)
@@ -272,28 +386,47 @@ SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 W
                      ->  Hash
                            ->  Seq Scan on prt2_p3
                                  Filter: (a = 0)
-(25 rows)
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = prt2_p4.b)
+                     ->  Seq Scan on prt1_p4
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4
+                                 Filter: (a = 0)
+(39 rows)
 
 SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
-  a  |  c   | phv |  b  |  c   | phv 
------+------+-----+-----+------+-----
-   0 | 0000 |  25 |   0 | 0000 |  50
-  50 | 0050 |  25 |     |      |    
- 100 | 0100 |  25 |     |      |    
- 150 | 0150 |  25 | 150 | 0150 |  50
- 200 | 0200 |  25 |     |      |    
- 250 | 0250 |  25 |     |      |    
- 300 | 0300 |  25 | 300 | 0300 |  50
- 350 | 0350 |  25 |     |      |    
- 400 | 0400 |  25 |     |      |    
- 450 | 0450 |  25 | 450 | 0450 |  50
- 500 | 0500 |  25 |     |      |    
- 550 | 0550 |  25 |     |      |    
-     |      |     |  75 | 0075 |  50
-     |      |     | 225 | 0225 |  50
-     |      |     | 375 | 0375 |  50
-     |      |     | 525 | 0525 |  50
-(16 rows)
+  a   |   c   | phv |  b   |   c   | phv 
+------+-------+-----+------+-------+-----
+ -250 | -0250 |  25 |      |       |    
+ -200 | -0200 |  25 |      |       |    
+ -150 | -0150 |  25 | -150 | -0150 |  50
+ -100 | -0100 |  25 |      |       |    
+  -50 | -0050 |  25 |      |       |    
+    0 | 0000  |  25 |    0 | 0000  |  50
+   50 | 0050  |  25 |      |       |    
+  100 | 0100  |  25 |      |       |    
+  150 | 0150  |  25 |  150 | 0150  |  50
+  200 | 0200  |  25 |      |       |    
+  250 | 0250  |  25 |      |       |    
+  300 | 0300  |  25 |  300 | 0300  |  50
+  350 | 0350  |  25 |      |       |    
+  400 | 0400  |  25 |      |       |    
+  450 | 0450  |  25 |  450 | 0450  |  50
+  500 | 0500  |  25 |      |       |    
+  550 | 0550  |  25 |      |       |    
+  600 | 0600  |  25 |  600 | 0600  |  50
+  650 | 0650  |  25 |      |       |    
+  700 | 0700  |  25 |      |       |    
+  750 | 0750  |  25 |  750 | 0750  |  50
+      |       |     | -225 | -0225 |  50
+      |       |     |  -75 | -0075 |  50
+      |       |     |   75 | 0075  |  50
+      |       |     |  225 | 0225  |  50
+      |       |     |  375 | 0375  |  50
+      |       |     |  525 | 0525  |  50
+      |       |     |  675 | 0675  |  50
+(28 rows)
 
 -- Join with pruned partitions from joining relations
 EXPLAIN (COSTS OFF)
@@ -323,9 +456,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
          ->  Hash Left Join
+               Hash Cond: (prt1_p0.a = b)
+               ->  Seq Scan on prt1_p0
+                     Filter: ((a < 450) AND (b = 0))
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
+         ->  Hash Left Join
                Hash Cond: (prt1_p1.a = b)
                ->  Seq Scan on prt1_p1
                      Filter: ((a < 450) AND (b = 0))
@@ -339,30 +479,43 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                ->  Hash
                      ->  Seq Scan on prt1_p2
                            Filter: ((a < 450) AND (b = 0))
-(17 rows)
+(24 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-(9 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+(14 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
                          QUERY PLAN                         
 ------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
          ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = b)
+               Filter: ((prt1_p0.b = 0) OR (a = 0))
+               ->  Seq Scan on prt1_p0
+                     Filter: (a < 450)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
+         ->  Hash Full Join
                Hash Cond: (prt1_p1.a = b)
                Filter: ((prt1_p1.b = 0) OR (a = 0))
                ->  Seq Scan on prt1_p1
@@ -386,64 +539,153 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                ->  Hash
                      ->  Result
                            One-Time Filter: false
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt2_p4.b = a)
+               Filter: ((b = 0) OR (prt2_p4.a = 0))
+               ->  Seq Scan on prt2_p4
+                     Filter: (b > 250)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
-(12 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+      |       | 375 | 0375
+      |       | 450 | 0450
+      |       | 525 | 0525
+      |       | 600 | 0600
+      |       | 675 | 0675
+      |       | 750 | 0750
+(20 rows)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Semi Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Semi Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                            Filter: (a = 0)
-         ->  Nested Loop Semi Join
-               Join Filter: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
-               ->  Materialize
-                     ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
-(24 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt1_p3 t2_3
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+                           Filter: (b = 0)
+(37 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
 
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
@@ -454,27 +696,82 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
    ->  Append
          ->  Hash Anti Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Hash Anti Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Hash Anti Join
                Hash Cond: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
-(17 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(27 rows)
 
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
-  sum  |         avg          | sum  |         avg         
--------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+  sum  |         avg          | sum  |        avg         
+-------+----------------------+------+--------------------
+ 95550 | 273.0000000000000000 | 2200 | 6.2857142857142857
 (1 row)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Append
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p0 t1
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+               Index Cond: (a = t1.b)
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.b = t2_1.a)
+         ->  Seq Scan on prt2_p1 t1_1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p1 t2_1
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p2 t1_2
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+               Index Cond: (a = t1_2.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p3 t1_3
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+               Index Cond: (a = t1_3.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p4 t1_4
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+               Index Cond: (a = t1_4.b)
+(27 rows)
+
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+  b   |   c   
+------+-------
+ -225 | -0225
+  -75 | -0075
+   75 | 0075
+  225 | 0225
+  375 | 0375
+  525 | 0525
+  675 | 0675
+(7 rows)
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -487,49 +784,74 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
    ->  Result
          ->  Append
                ->  Nested Loop Left Join
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
                      ->  Nested Loop
-                           ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
+                           ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
                                  Index Cond: (a = t1.a)
-                           ->  Index Scan using iprt2_p1_b on prt2_p1 t3
+                           ->  Index Scan using iprt2_p0_b on prt2_p0 t3
                                  Index Cond: (b = t2.a)
                ->  Nested Loop Left Join
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
                      ->  Nested Loop
-                           ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_1
+                           ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
                                  Index Cond: (a = t1_1.a)
-                           ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
+                           ->  Index Scan using iprt2_p1_b on prt2_p1 t3_1
                                  Index Cond: (b = t2_1.a)
                ->  Nested Loop Left Join
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
                      ->  Nested Loop
-                           ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_2
+                           ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
                                  Index Cond: (a = t1_2.a)
-                           ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
+                           ->  Index Scan using iprt2_p2_b on prt2_p2 t3_2
                                  Index Cond: (b = t2_2.a)
-(28 rows)
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Nested Loop
+                           ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                                 Index Cond: (a = t1_3.a)
+                           ->  Index Scan using iprt2_p3_b on prt2_p3 t3_3
+                                 Index Cond: (b = t2_3.a)
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+                     ->  Nested Loop
+                           ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                                 Index Cond: (a = t1_4.a)
+                           ->  Index Scan using iprt2_p4_b on prt2_p4 t3_4
+                                 Index Cond: (b = t2_4.a)
+(44 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   | t2a | t3a | least 
------+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
- 100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
- 250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
- 400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
- 550 | 0 | 0550 |     |     |      
-(12 rows)
+  a   | b |   c   | t2a  | t3a | least 
+------+---+-------+------+-----+-------
+ -250 | 0 | -0250 |      |     |      
+ -200 | 0 | -0200 |      |     |      
+ -150 | 0 | -0150 | -150 |   0 |  -150
+ -100 | 0 | -0100 |      |     |      
+  -50 | 0 | -0050 |      |     |      
+    0 | 0 | 0000  |    0 |   0 |     0
+   50 | 0 | 0050  |      |     |      
+  100 | 0 | 0100  |      |     |      
+  150 | 0 | 0150  |  150 |   0 |   150
+  200 | 0 | 0200  |      |     |      
+  250 | 0 | 0250  |      |     |      
+  300 | 0 | 0300  |  300 |   0 |   300
+  350 | 0 | 0350  |      |     |      
+  400 | 0 | 0400  |      |     |      
+  450 | 0 | 0450  |  450 |   0 |   450
+  500 | 0 | 0500  |      |     |      
+  550 | 0 | 0550  |      |     |      
+  600 | 0 | 0600  |  600 |   0 |   600
+  650 | 0 | 0650  |      |     |      
+  700 | 0 | 0700  |      |     |      
+  750 | 0 | 0750  |  750 |   0 |   750
+(21 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
@@ -543,64 +865,95 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
          Hash Cond: ((t1.c)::text = (t2.c)::text)
          Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Seq Scan on prt1_p1 t1
-               ->  Seq Scan on prt1_p2 t1_1
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
          ->  Hash
                ->  Append
                      ->  Hash Join
                            Hash Cond: (t2.a = t3.b)
-                           ->  Seq Scan on prt1_p1 t2
+                           ->  Seq Scan on prt1_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t3
+                                 ->  Seq Scan on prt2_p0 t3
                      ->  Hash Join
                            Hash Cond: (t2_1.a = t3_1.b)
-                           ->  Seq Scan on prt1_p2 t2_1
+                           ->  Seq Scan on prt1_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t3_1
+                                 ->  Seq Scan on prt2_p1 t3_1
                      ->  Hash Join
                            Hash Cond: (t2_2.a = t3_2.b)
-                           ->  Seq Scan on prt1_p3 t2_2
+                           ->  Seq Scan on prt1_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p3 t3_2
-(26 rows)
+                                 ->  Seq Scan on prt2_p2 t3_2
+                     ->  Hash Join
+                           Hash Cond: (t2_3.a = t3_3.b)
+                           ->  Seq Scan on prt1_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t3_3
+                     ->  Hash Join
+                           Hash Cond: (t2_4.a = t3_4.b)
+                           ->  Seq Scan on prt1_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t3_4
+(38 rows)
 
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, t2.b t2b, t2.c t2c, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
-  a  | t2a | t2c  
------+-----+------
-   0 |   0 | 0000
-  50 |     | 
- 100 |     | 
- 150 | 150 | 0150
- 200 |     | 
- 250 |     | 
- 300 | 300 | 0300
- 350 |     | 
- 400 |     | 
- 450 | 450 | 0450
- 500 |     | 
- 550 |     | 
-(12 rows)
+  a   | t2a  |  t2c  
+------+------+-------
+ -250 |      | 
+ -200 |      | 
+ -150 | -150 | -0150
+ -100 |      | 
+  -50 |      | 
+    0 |    0 | 0000
+   50 |      | 
+  100 |      | 
+  150 |  150 | 0150
+  200 |      | 
+  250 |      | 
+  300 |  300 | 0300
+  350 |      | 
+  400 |      | 
+  450 |  450 | 0450
+  500 |      | 
+  550 |      | 
+  600 |  600 | 0600
+  650 |      | 
+  700 |      | 
+  750 |  750 | 0750
+(21 rows)
 
 --
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
@@ -611,32 +964,49 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
    ->  Append
          ->  Hash Join
                Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
-               ->  Seq Scan on prt2_e_p1 t2
+               ->  Seq Scan on prt2_e_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 t1
+                     ->  Seq Scan on prt1_e_p0 t1
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
-               ->  Seq Scan on prt2_e_p2 t2_1
+               Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+               ->  Seq Scan on prt1_e_p1 t1_1
+                     Filter: (c = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 t1_1
-                           Filter: (c = 0)
+                     ->  Seq Scan on prt2_e_p1 t2_1
          ->  Hash Join
                Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
-               ->  Seq Scan on prt2_e_p3 t2_2
+               ->  Seq Scan on prt2_e_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 t1_2
+                     ->  Seq Scan on prt1_e_p2 t1_2
                            Filter: (c = 0)
-(21 rows)
+         ->  Hash Join
+               Hash Cond: (((t2_3.b + t2_3.a) / 2) = ((t1_3.a + t1_3.b) / 2))
+               ->  Seq Scan on prt2_e_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p3 t1_3
+                           Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (((t2_4.b + t2_4.a) / 2) = ((t1_4.a + t1_4.b) / 2))
+               ->  Seq Scan on prt2_e_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4 t1_4
+                           Filter: (c = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
- 150 | 0 | 150 | 0
- 300 | 0 | 300 | 0
- 450 | 0 | 450 | 0
-(4 rows)
+  a   | c |  b   | c 
+------+---+------+---
+ -250 | 0 | -250 | 0
+ -100 | 0 | -100 | 0
+    0 | 0 |    0 | 0
+    0 | 0 |    0 | 0
+  150 | 0 |  150 | 0
+  300 | 0 |  300 | 0
+  450 | 0 |  450 | 0
+  600 | 0 |  600 | 0
+  750 | 0 |  750 | 0
+(9 rows)
 
 --
 -- N-way join
@@ -650,107 +1020,158 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t
    ->  Result
          ->  Append
                ->  Nested Loop
-                     Join Filter: (t1.a = ((t3.a + t3.b) / 2))
+                     Join Filter: (t1.a = t2.b)
                      ->  Hash Join
-                           Hash Cond: (t2.b = t1.a)
-                           ->  Seq Scan on prt2_p1 t2
+                           Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                           ->  Seq Scan on prt1_e_p0 t3
                            ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                                        Filter: (b = 0)
-                     ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                           Index Cond: (((a + b) / 2) = t2.b)
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t2
+                           Index Cond: (b = ((t3.a + t3.b) / 2))
                ->  Nested Loop
                      Join Filter: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
                      ->  Hash Join
                            Hash Cond: (t2_1.b = t1_1.a)
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                                        Filter: (b = 0)
-                     ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
+                     ->  Index Scan using iprt1_e_p4_ab2 on prt1_e_p1 t3_1
                            Index Cond: (((a + b) / 2) = t2_1.b)
                ->  Nested Loop
                      Join Filter: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
                      ->  Hash Join
                            Hash Cond: (t2_2.b = t1_2.a)
-                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                                        Filter: (b = 0)
-                     ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
+                     ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_2
                            Index Cond: (((a + b) / 2) = t2_2.b)
-(34 rows)
+               ->  Nested Loop
+                     Join Filter: (t1_3.a = ((t3_3.a + t3_3.b) / 2))
+                     ->  Nested Loop
+                           ->  Seq Scan on prt1_p3 t1_3
+                                 Filter: (b = 0)
+                           ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                                 Index Cond: (b = t1_3.a)
+                     ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                           Index Cond: (((a + b) / 2) = t2_3.b)
+               ->  Nested Loop
+                     Join Filter: (t1_4.a = t2_4.b)
+                     ->  Hash Join
+                           Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                           ->  Seq Scan on prt1_e_p4 t3_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_p4 t1_4
+                                       Filter: (b = 0)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                           Index Cond: (b = ((t3_4.a + t3_4.b) / 2))
+(53 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
-(4 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(8 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                             QUERY PLAN                             
---------------------------------------------------------------------
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Result
          ->  Append
                ->  Hash Right Join
                      Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
-                     ->  Seq Scan on prt1_e_p1 t3
+                     ->  Seq Scan on prt1_e_p0 t3
                      ->  Hash
                            ->  Hash Right Join
                                  Hash Cond: (t2.b = t1.a)
-                                 ->  Seq Scan on prt2_p1 t2
+                                 ->  Seq Scan on prt2_p0 t2
                                  ->  Hash
-                                       ->  Seq Scan on prt1_p1 t1
+                                       ->  Seq Scan on prt1_p0 t1
                                              Filter: (b = 0)
                ->  Hash Right Join
                      Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_1.a)
-                     ->  Seq Scan on prt1_e_p2 t3_1
+                     ->  Seq Scan on prt1_e_p1 t3_1
                      ->  Hash
                            ->  Hash Right Join
                                  Hash Cond: (t2_1.b = t1_1.a)
-                                 ->  Seq Scan on prt2_p2 t2_1
+                                 ->  Seq Scan on prt2_p1 t2_1
                                  ->  Hash
-                                       ->  Seq Scan on prt1_p2 t1_1
+                                       ->  Seq Scan on prt1_p1 t1_1
                                              Filter: (b = 0)
                ->  Hash Right Join
                      Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_2.a)
-                     ->  Seq Scan on prt1_e_p3 t3_2
+                     ->  Seq Scan on prt1_e_p2 t3_2
                      ->  Hash
                            ->  Hash Right Join
                                  Hash Cond: (t2_2.b = t1_2.a)
-                                 ->  Seq Scan on prt2_p3 t2_2
+                                 ->  Seq Scan on prt2_p2 t2_2
+                                 ->  Hash
+                                       ->  Seq Scan on prt1_p2 t1_2
+                                             Filter: (b = 0)
+               ->  Nested Loop Left Join
+                     ->  Nested Loop Left Join
+                           ->  Seq Scan on prt1_p3 t1_3
+                                 Filter: (b = 0)
+                           ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                                 Index Cond: (t1_3.a = b)
+                     ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                           Index Cond: (t1_3.a = ((a + b) / 2))
+               ->  Hash Right Join
+                     Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                     ->  Seq Scan on prt1_e_p4 t3_4
+                     ->  Hash
+                           ->  Hash Right Join
+                                 Hash Cond: (t2_4.b = t1_4.a)
+                                 ->  Seq Scan on prt2_p4 t2_4
                                  ->  Hash
-                                       ->  Seq Scan on prt1_p3 t1_2
+                                       ->  Seq Scan on prt1_p4 t1_4
                                              Filter: (b = 0)
-(34 rows)
+(52 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                               QUERY PLAN                                
--------------------------------------------------------------------------
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Result
@@ -758,48 +1179,75 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                ->  Nested Loop Left Join
                      ->  Hash Right Join
                            Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t3
+                                 ->  Seq Scan on prt1_e_p0 t3
                                        Filter: (c = 0)
-                     ->  Index Scan using iprt2_p1_b on prt2_p1 t2
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t2
                            Index Cond: (t1.a = b)
                ->  Nested Loop Left Join
                      ->  Hash Right Join
                            Hash Cond: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                 ->  Seq Scan on prt1_e_p1 t3_1
                                        Filter: (c = 0)
-                     ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
                            Index Cond: (t1_1.a = b)
                ->  Nested Loop Left Join
                      ->  Hash Right Join
                            Hash Cond: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                 ->  Seq Scan on prt1_e_p2 t3_2
                                        Filter: (c = 0)
-                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
                            Index Cond: (t1_2.a = b)
-(31 rows)
+               ->  Nested Loop Left Join
+                     ->  Nested Loop Left Join
+                           ->  Seq Scan on prt1_e_p3 t3_3
+                                 Filter: (c = 0)
+                           ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                                 Index Cond: (a = ((t3_3.a + t3_3.b) / 2))
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (t1_3.a = b)
+               ->  Nested Loop Left Join
+                     ->  Hash Right Join
+                           Hash Cond: (t1_4.a = ((t3_4.a + t3_4.b) / 2))
+                           ->  Seq Scan on prt1_p4 t1_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t3_4
+                                       Filter: (c = 0)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                           Index Cond: (t1_4.a = b)
+(48 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -808,23 +1256,36 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                                                       QUERY PLAN                                                      
 ----------------------------------------------------------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   Sort Key: prt1_p0.a, prt2_p0.b, ((prt1_e_p0.a + prt1_e_p0.b))
    ->  Result
          ->  Append
                ->  Hash Full Join
-                     Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
-                     Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+                     Hash Cond: (prt1_p0.a = ((prt1_e_p0.a + prt1_e_p0.b) / 2))
+                     Filter: ((prt1_p0.a = (50)) OR (prt2_p0.b = (75)) OR (((prt1_e_p0.a + prt1_e_p0.b) / 2) = (50)))
                      ->  Hash Full Join
-                           Hash Cond: (prt1_p1.a = prt2_p1.b)
-                           ->  Seq Scan on prt1_p1
+                           Hash Cond: (prt1_p0.a = prt2_p0.b)
+                           ->  Seq Scan on prt1_p0
                                  Filter: (b = 0)
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1
+                                 ->  Seq Scan on prt2_p0
                                        Filter: (a = 0)
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p1
+                           ->  Seq Scan on prt1_e_p0
                                  Filter: (c = 0)
                ->  Hash Full Join
+                     Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = prt1_p1.a)
+                     Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+                     ->  Seq Scan on prt1_e_p1
+                           Filter: (c = 0)
+                     ->  Hash
+                           ->  Hash Full Join
+                                 Hash Cond: (prt1_p1.a = prt2_p1.b)
+                                 ->  Seq Scan on prt1_p1
+                                       Filter: (b = 0)
+                                 ->  Hash
+                                       ->  Seq Scan on prt2_p1
+                                             Filter: (a = 0)
+               ->  Hash Full Join
                      Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
                      Filter: ((prt1_p2.a = (50)) OR (prt2_p2.b = (75)) OR (((prt1_e_p2.a + prt1_e_p2.b) / 2) = (50)))
                      ->  Hash Full Join
@@ -850,7 +1311,20 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                      ->  Hash
                            ->  Seq Scan on prt1_e_p3
                                  Filter: (c = 0)
-(43 rows)
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = ((prt1_e_p4.a + prt1_e_p4.b) / 2))
+                     Filter: ((prt1_p4.a = (50)) OR (prt2_p4.b = (75)) OR (((prt1_e_p4.a + prt1_e_p4.b) / 2) = (50)))
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_p4.a = prt2_p4.b)
+                           ->  Seq Scan on prt1_p4
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4
+                                       Filter: (a = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p4
+                                 Filter: (c = 0)
+(69 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
@@ -868,397 +1342,2606 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = t1_3.b)
+               Join Filter: (t1.a = t1_5.b)
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Join
-                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
-                           ->  Seq Scan on prt1_e_p1 t2
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t1_3
+                                 ->  Seq Scan on prt2_p0 t1_5
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
                      Index Cond: (a = ((t2.a + t2.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_1.a = t1_4.b)
+               Join Filter: (t1_1.a = t1_6.b)
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Join
-                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
-                           ->  Seq Scan on prt1_e_p2 t2_1
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_6.b)
+                           ->  Seq Scan on prt1_e_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t1_4
+                                 ->  Seq Scan on prt2_p1 t1_6
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
                      Index Cond: (a = ((t2_1.a + t2_1.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_2.a = t1_5.b)
+               Join Filter: (t1_2.a = t1_7.b)
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
+                           ->  Seq Scan on prt2_p2 t1_7
                                  Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
+                           ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t2_2
+                                 Index Cond: (((a + b) / 2) = t1_7.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
-
-SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
-
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t1_8.b)
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Nested Loop
+                           ->  Seq Scan on prt2_p3 t1_8
+                                 Filter: (a = 0)
+                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_3
+                                 Index Cond: (((a + b) / 2) = t1_8.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = ((t2_3.a + t2_3.b) / 2))
+                     Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t1_9.b)
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Join
+                           Hash Cond: (((t2_4.a + t2_4.b) / 2) = t1_9.b)
+                           ->  Seq Scan on prt1_e_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t1_9
+                                       Filter: (a = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = ((t2_4.a + t2_4.b) / 2))
+                     Filter: (b = 0)
+(66 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                                  QUERY PLAN                                   
--------------------------------------------------------------------------------
+                                   QUERY PLAN                                    
+---------------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
-                           ->  Seq Scan on prt2_p1 t1_3
+                           Hash Cond: (t1_5.b = ((t1_10.a + t1_10.b) / 2))
+                           ->  Seq Scan on prt2_p0 t1_5
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t1_6
+                                 ->  Seq Scan on prt1_e_p0 t1_10
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
-                     Index Cond: (a = t1_3.b)
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t1_5.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
-                           ->  Seq Scan on prt2_p2 t1_4
+                           Hash Cond: (t1_6.b = ((t1_11.a + t1_11.b) / 2))
+                           ->  Seq Scan on prt2_p1 t1_6
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t1_7
+                                 ->  Seq Scan on prt1_e_p1 t1_11
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
-                     Index Cond: (a = t1_4.b)
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
+                     Index Cond: (a = t1_6.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_7.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_7.b = ((t1_12.a + t1_12.b) / 2))
+                           ->  Seq Scan on prt2_p2 t1_7
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p2 t1_12
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t1_7.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  Unique
                      ->  Sort
-                           Sort Key: t1_5.b
+                           Sort Key: t1_8.b
                            ->  Hash Semi Join
-                                 Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
-                                 ->  Seq Scan on prt2_p3 t1_5
+                                 Hash Cond: (t1_8.b = ((t1_13.a + t1_13.b) / 2))
+                                 ->  Seq Scan on prt2_p3 t1_8
                                  ->  Hash
-                                       ->  Seq Scan on prt1_e_p3 t1_8
+                                       ->  Seq Scan on prt1_e_p3 t1_13
                                              Filter: (c = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t1_5.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t1_8.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_9.b = ((t1_14.a + t1_14.b) / 2))
+                           ->  Seq Scan on prt2_p4 t1_9
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t1_14
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = t1_9.b)
                      Filter: (b = 0)
-(40 rows)
+(64 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+-- test merge joins
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Merge Cond: (t1.a = t1_5.b)
+         ->  Sort
+               Sort Key: t1.a
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_5.b = (((t1_10.a + t1_10.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on prt2_p0 t1_5
+               ->  Sort
+                     Sort Key: (((t1_10.a + t1_10.b) / 2))
+                     ->  Seq Scan on prt1_e_p0 t1_10
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_1.a = t1_6.b)
+         ->  Sort
+               Sort Key: t1_1.a
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_6.b = (((t1_11.a + t1_11.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_6.b
+                     ->  Seq Scan on prt2_p1 t1_6
+               ->  Sort
+                     Sort Key: (((t1_11.a + t1_11.b) / 2))
+                     ->  Seq Scan on prt1_e_p1 t1_11
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_2.a = t1_7.b)
+         ->  Sort
+               Sort Key: t1_2.a
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_7.b = (((t1_12.a + t1_12.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_7.b
+                     ->  Seq Scan on prt2_p2 t1_7
+               ->  Sort
+                     Sort Key: (((t1_12.a + t1_12.b) / 2))
+                     ->  Seq Scan on prt1_e_p2 t1_12
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_3.a = t1_8.b)
+         ->  Sort
+               Sort Key: t1_3.a
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_8.b = (((t1_13.a + t1_13.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_8.b
+                     ->  Seq Scan on prt2_p3 t1_8
+               ->  Sort
+                     Sort Key: (((t1_13.a + t1_13.b) / 2))
+                     ->  Seq Scan on prt1_e_p3 t1_13
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_4.a = t1_9.b)
+         ->  Sort
+               Sort Key: t1_4.a
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_9.b = (((t1_14.a + t1_14.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_9.b
+                     ->  Seq Scan on prt2_p4 t1_9
+               ->  Sort
+                     Sort Key: (((t1_14.a + t1_14.b) / 2))
+                     ->  Seq Scan on prt1_e_p4 t1_14
+                           Filter: (c = 0)
+(77 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         ->  Append
+               ->  Merge Left Join
+                     Merge Cond: (t1.a = t2.b)
+                     ->  Sort
+                           Sort Key: t1.a
+                           ->  Merge Left Join
+                                 Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
+                                 ->  Sort
+                                       Sort Key: (((t3.a + t3.b) / 2))
+                                       ->  Seq Scan on prt1_e_p0 t3
+                                             Filter: (c = 0)
+                                 ->  Sort
+                                       Sort Key: t1.a
+                                       ->  Seq Scan on prt1_p0 t1
+                     ->  Sort
+                           Sort Key: t2.b
+                           ->  Seq Scan on prt2_p0 t2
+               ->  Merge Left Join
+                     Merge Cond: (t1_1.a = t2_1.b)
+                     ->  Sort
+                           Sort Key: t1_1.a
+                           ->  Merge Left Join
+                                 Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
+                                 ->  Sort
+                                       Sort Key: (((t3_1.a + t3_1.b) / 2))
+                                       ->  Seq Scan on prt1_e_p1 t3_1
+                                             Filter: (c = 0)
+                                 ->  Sort
+                                       Sort Key: t1_1.a
+                                       ->  Seq Scan on prt1_p1 t1_1
+                     ->  Sort
+                           Sort Key: t2_1.b
+                           ->  Seq Scan on prt2_p1 t2_1
+               ->  Merge Left Join
+                     Merge Cond: (t1_2.a = t2_2.b)
+                     ->  Sort
+                           Sort Key: t1_2.a
+                           ->  Merge Left Join
+                                 Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
+                                 ->  Sort
+                                       Sort Key: (((t3_2.a + t3_2.b) / 2))
+                                       ->  Seq Scan on prt1_e_p2 t3_2
+                                             Filter: (c = 0)
+                                 ->  Sort
+                                       Sort Key: t1_2.a
+                                       ->  Seq Scan on prt1_p2 t1_2
+                     ->  Sort
+                           Sort Key: t2_2.b
+                           ->  Seq Scan on prt2_p2 t2_2
+               ->  Merge Left Join
+                     Merge Cond: (t1_3.a = t2_3.b)
+                     ->  Sort
+                           Sort Key: t1_3.a
+                           ->  Merge Left Join
+                                 Merge Cond: ((((t3_3.a + t3_3.b) / 2)) = t1_3.a)
+                                 ->  Sort
+                                       Sort Key: (((t3_3.a + t3_3.b) / 2))
+                                       ->  Seq Scan on prt1_e_p3 t3_3
+                                             Filter: (c = 0)
+                                 ->  Sort
+                                       Sort Key: t1_3.a
+                                       ->  Seq Scan on prt1_p3 t1_3
+                     ->  Sort
+                           Sort Key: t2_3.b
+                           ->  Seq Scan on prt2_p3 t2_3
+               ->  Merge Left Join
+                     Merge Cond: (t1_4.a = t2_4.b)
+                     ->  Sort
+                           Sort Key: t1_4.a
+                           ->  Merge Left Join
+                                 Merge Cond: ((((t3_4.a + t3_4.b) / 2)) = t1_4.a)
+                                 ->  Sort
+                                       Sort Key: (((t3_4.a + t3_4.b) / 2))
+                                       ->  Seq Scan on prt1_e_p4 t3_4
+                                             Filter: (c = 0)
+                                 ->  Sort
+                                       Sort Key: t1_4.a
+                                       ->  Seq Scan on prt1_p4 t1_4
+                     ->  Sort
+                           Sort Key: t2_4.b
+                           ->  Seq Scan on prt2_p4 t2_4
+(84 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -150 | -0150 | 0 | -0150
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+(7 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (t1_3.a = b)
+         ->  Hash Right Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 |   | 
+ -200 | -0200 |   | 
+ -150 | -0150 | 0 | -0150
+ -100 | -0100 |   | 
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+(21 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -200 | 0 | -0200
+ -100 | 0 | -0100
+  -50 | 0 | -0050
+   50 | 0 | 0050
+  100 | 0 | 0100
+  200 | 0 | 0200
+  250 | 0 | 0250
+  350 | 0 | 0350
+  400 | 0 | 0400
+  500 | 0 | 0500
+  550 | 0 | 0550
+  650 | 0 | 0650
+  700 | 0 | 0700
+(14 rows)
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.b
+   ->  Hash Right Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p5 t2_5
+                           Filter: (a = 0)
+(24 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: (t1.a = t2.b)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.a, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
+                     Filter: (a = 0)
+(22 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
+               ->  Seq Scan on prt1_m_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p1
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
+               ->  Seq Scan on prt1_m_p2
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p2
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
+               ->  Seq Scan on prt1_m_p3
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p3
+                           Filter: (c = 0)
+(24 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+  a  | c |  b  | c 
+-----+---+-----+---
+   0 | 0 |   0 | 0
+  50 | 0 |     |  
+ 100 | 0 |     |  
+ 150 | 0 | 150 | 0
+ 200 | 0 |     |  
+ 250 | 0 |     |  
+ 300 | 0 | 300 | 0
+ 350 | 0 |     |  
+ 400 | 0 |     |  
+ 450 | 0 | 450 | 0
+ 500 | 0 |     |  
+ 550 | 0 |     |  
+     |   |  75 | 0
+     |   | 225 | 0
+     |   | 375 | 0
+     |   | 525 | 0
+(16 rows)
+
+--
+-- tests for list partitioned tables.
+--
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Result
+         ->  Append
+               ->  Hash Right Join
+                     Hash Cond: ((t1.c)::text = (t2.c)::text)
+                     Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Hash
+                           ->  Seq Scan on plt2_p4 t2
+               ->  Hash Left Join
+                     Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+                     Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Hash
+                           ->  Seq Scan on plt1_p1 t1_1
+               ->  Hash Left Join
+                     Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+                     Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Hash
+                           ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash Right Join
+                     Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+                     Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+                     ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt2_p3 t2_3
+(28 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  HashAggregate
+                           Group Key: (t2.c)::text
+                           ->  Seq Scan on plt2_p4 t2
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
+--
+-- list partitioned by expression
+--
+CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
+ANALYZE plt1_e;
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+                                       QUERY PLAN                                       
+----------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t3.c
+   ->  HashAggregate
+         Group Key: t1.c, t2.c, t3.c
+         ->  Result
+               ->  Append
+                     ->  Hash Join
+                           Hash Cond: ((t1.c)::text = (t2.c)::text)
+                           ->  Hash Join
+                                 Hash Cond: ((t1.c)::text = ltrim(t3.c, 'A'::text))
+                                 ->  Seq Scan on plt1_p4 t1
+                                 ->  Hash
+                                       ->  Seq Scan on plt1_e_p4 t3
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p4 t2
+                     ->  Hash Join
+                           Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+                           ->  Hash Join
+                                 Hash Cond: ((t2_1.c)::text = ltrim(t3_1.c, 'A'::text))
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Hash
+                                       ->  Seq Scan on plt1_e_p1 t3_1
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p1 t1_1
+                     ->  Hash Join
+                           Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+                           ->  Hash Join
+                                 Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
+                                 ->  Seq Scan on plt1_p2 t1_2
+                                 ->  Hash
+                                       ->  Seq Scan on plt1_e_p2 t3_2
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p2 t2_2
+                     ->  Hash Join
+                           Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+                           ->  Hash Join
+                                 Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                                 ->  Seq Scan on plt1_p3 t1_3
+                                 ->  Hash
+                                       ->  Seq Scan on plt1_e_p3 t3_3
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p3 t2_3
+(42 rows)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  HashAggregate
+                           Group Key: (t2.c)::text
+                           ->  Seq Scan on plt2_p4 t2
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p5 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_3
+                                 ->  Seq Scan on plt2_p3 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Result
+         ->  Append
+               ->  Hash Left Join
+                     Hash Cond: ((t2.c)::text = (t1.c)::text)
+                     Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Hash
+                           ->  Seq Scan on plt1_p4 t1
+               ->  Hash Left Join
+                     Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+                     Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Hash
+                           ->  Seq Scan on plt1_p1 t1_1
+               ->  Hash Left Join
+                     Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+                     Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Hash
+                           ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash Right Join
+                     Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+                     Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+                     ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt2_p3 t2_3
+(28 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
 
--- test merge joins
-SET enable_hashjoin TO off;
-SET enable_nestloop TO off;
+-- full join
 EXPLAIN (COSTS OFF)
-SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                           QUERY PLAN                           
-----------------------------------------------------------------
- Merge Append
-   Sort Key: t1.a
-   ->  Merge Semi Join
-         Merge Cond: (t1.a = t1_3.b)
-         ->  Sort
-               Sort Key: t1.a
-               ->  Seq Scan on prt1_p1 t1
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
                      Filter: (b = 0)
-         ->  Merge Semi Join
-               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
-               ->  Sort
-                     Sort Key: t1_3.b
-                     ->  Seq Scan on prt2_p1 t1_3
-               ->  Sort
-                     Sort Key: (((t1_6.a + t1_6.b) / 2))
-                     ->  Seq Scan on prt1_e_p1 t1_6
-                           Filter: (c = 0)
-   ->  Merge Semi Join
-         Merge Cond: (t1_1.a = t1_4.b)
-         ->  Sort
-               Sort Key: t1_1.a
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on plt1_p1 t1_1
                      Filter: (b = 0)
-         ->  Merge Semi Join
-               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
-               ->  Sort
-                     Sort Key: t1_4.b
-                     ->  Seq Scan on prt2_p2 t1_4
-               ->  Sort
-                     Sort Key: (((t1_7.a + t1_7.b) / 2))
-                     ->  Seq Scan on prt1_e_p2 t1_7
-                           Filter: (c = 0)
-   ->  Merge Semi Join
-         Merge Cond: (t1_2.a = t1_5.b)
-         ->  Sort
-               Sort Key: t1_2.a
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on plt1_p2 t1_2
                      Filter: (b = 0)
-         ->  Merge Semi Join
-               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
-               ->  Sort
-                     Sort Key: t1_5.b
-                     ->  Seq Scan on prt2_p3 t1_5
-               ->  Sort
-                     Sort Key: (((t1_8.a + t1_8.b) / 2))
-                     ->  Seq Scan on prt1_e_p3 t1_8
-                           Filter: (c = 0)
-(47 rows)
-
-SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Seq Scan on plt2_p2 t2_2
+                                 ->  Seq Scan on plt2_p3 t2_3
+(22 rows)
 
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                                    QUERY PLAN                                    
-----------------------------------------------------------------------------------
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
  Sort
-   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
-   ->  Result
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
          ->  Append
-               ->  Merge Left Join
-                     Merge Cond: (t1.a = t2.b)
-                     ->  Sort
-                           Sort Key: t1.a
-                           ->  Merge Left Join
-                                 Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
-                                 ->  Sort
-                                       Sort Key: (((t3.a + t3.b) / 2))
-                                       ->  Seq Scan on prt1_e_p1 t3
-                                             Filter: (c = 0)
-                                 ->  Sort
-                                       Sort Key: t1.a
-                                       ->  Seq Scan on prt1_p1 t1
-                     ->  Sort
-                           Sort Key: t2.b
-                           ->  Seq Scan on prt2_p1 t2
-               ->  Merge Left Join
-                     Merge Cond: (t1_1.a = t2_1.b)
-                     ->  Sort
-                           Sort Key: t1_1.a
-                           ->  Merge Left Join
-                                 Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
-                                 ->  Sort
-                                       Sort Key: (((t3_1.a + t3_1.b) / 2))
-                                       ->  Seq Scan on prt1_e_p2 t3_1
-                                             Filter: (c = 0)
-                                 ->  Sort
-                                       Sort Key: t1_1.a
-                                       ->  Seq Scan on prt1_p2 t1_1
-                     ->  Sort
-                           Sort Key: t2_1.b
-                           ->  Seq Scan on prt2_p2 t2_1
-               ->  Merge Left Join
-                     Merge Cond: (t1_2.a = t2_2.b)
-                     ->  Sort
-                           Sort Key: t1_2.a
-                           ->  Merge Left Join
-                                 Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
-                                 ->  Sort
-                                       Sort Key: (((t3_2.a + t3_2.b) / 2))
-                                       ->  Seq Scan on prt1_e_p3 t3_2
-                                             Filter: (c = 0)
-                                 ->  Sort
-                                       Sort Key: t1_2.a
-                                       ->  Seq Scan on prt1_p3 t1_2
-                     ->  Sort
-                           Sort Key: t2_2.b
-                           ->  Seq Scan on prt2_p3 t2_2
-(52 rows)
-
-SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
 
-RESET enable_hashjoin;
-RESET enable_nestloop;
---
--- partitioned by multiple columns
---
-CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
-CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
-ANALYZE prt1_m;
-CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
-CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
-ANALYZE prt2_m;
+-- anti join
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-                                                             QUERY PLAN                                                             
-------------------------------------------------------------------------------------------------------------------------------------
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
  Sort
-   Sort Key: prt1_m_p1.a, prt2_m_p1.b
-   ->  Append
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
-               ->  Seq Scan on prt1_m_p1
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p1
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
-               ->  Seq Scan on prt1_m_p2
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p2
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
-               ->  Seq Scan on prt1_m_p3
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p3
-                           Filter: (c = 0)
-(24 rows)
-
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
-  50 | 0 |     |  
- 100 | 0 |     |  
- 150 | 0 | 150 | 0
- 200 | 0 |     |  
- 250 | 0 |     |  
- 300 | 0 | 300 | 0
- 350 | 0 |     |  
- 400 | 0 |     |  
- 450 | 0 | 450 | 0
- 500 | 0 |     |  
- 550 | 0 |     |  
-     |   |  75 | 0
-     |   | 225 | 0
-     |   | 375 | 0
-     |   | 525 | 0
-(16 rows)
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+(19 rows)
 
---
--- tests for list partitioned tables.
---
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
---
--- list partitioned by expression
---
-CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1_e;
--- test partition matching with N-way join
 EXPLAIN (COSTS OFF)
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                      QUERY PLAN                                      
---------------------------------------------------------------------------------------
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
  Sort
-   Sort Key: t1.c, t3.c
-   ->  HashAggregate
-         Group Key: t1.c, t2.c, t3.c
-         ->  Result
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
                ->  Append
-                     ->  Hash Join
-                           Hash Cond: (t1.c = t2.c)
-                           ->  Seq Scan on plt1_p1 t1
-                           ->  Hash
-                                 ->  Hash Join
-                                       Hash Cond: (t2.c = ltrim(t3.c, 'A'::text))
-                                       ->  Seq Scan on plt2_p1 t2
-                                       ->  Hash
-                                             ->  Seq Scan on plt1_e_p1 t3
-                     ->  Hash Join
-                           Hash Cond: (t1_1.c = t2_1.c)
-                           ->  Seq Scan on plt1_p2 t1_1
-                           ->  Hash
-                                 ->  Hash Join
-                                       Hash Cond: (t2_1.c = ltrim(t3_1.c, 'A'::text))
-                                       ->  Seq Scan on plt2_p2 t2_1
-                                       ->  Hash
-                                             ->  Seq Scan on plt1_e_p2 t3_1
-                     ->  Hash Join
-                           Hash Cond: (t1_2.c = t2_2.c)
-                           ->  Seq Scan on plt1_p3 t1_2
-                           ->  Hash
-                                 ->  Hash Join
-                                       Hash Cond: (t2_2.c = ltrim(t3_2.c, 'A'::text))
-                                       ->  Seq Scan on plt2_p3 t2_2
-                                       ->  Hash
-                                             ->  Seq Scan on plt1_e_p3 t3_2
-(33 rows)
-
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  74.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 124.0000000000000000 | 124.5000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 224.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 274.0000000000000000 | 274.5000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 374.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 424.0000000000000000 | 424.5000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 524.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 574.0000000000000000 | 574.5000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1286,16 +3969,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p4 t2_4
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
@@ -1306,16 +3995,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p4 t2_4
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 --
 -- multi-leveled partitions
@@ -1676,64 +4371,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
  Hash Join
    Hash Cond: (t1.a = t2.a)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt4_n_p1 t2
                ->  Seq Scan on prt4_n_p2 t2_1
                ->  Seq Scan on prt4_n_p3 t2_2
-(11 rows)
+(13 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a and t1.a = t3.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                        QUERY PLAN                        
+----------------------------------------------------------
  Hash Join
-   Hash Cond: (t2.a = t1.a)
+   Hash Cond: (t3.b = t1.a)
    ->  Append
-         ->  Seq Scan on prt4_n_p1 t2
-         ->  Seq Scan on prt4_n_p2 t2_1
-         ->  Seq Scan on prt4_n_p3 t2_2
+         ->  Seq Scan on prt2_p0 t3
+         ->  Seq Scan on prt2_p1 t3_1
+         ->  Seq Scan on prt2_p2 t3_2
+         ->  Seq Scan on prt2_p3 t3_3
+         ->  Seq Scan on prt2_p4 t3_4
+         ->  Seq Scan on prt2_p5 t3_5
    ->  Hash
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.a = t3_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_2.b)
-                     ->  Seq Scan on prt1_p3 t1_2
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3 t3_2
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt4_n_p1 t2
+                           ->  Seq Scan on prt4_n_p2 t2_1
+                           ->  Seq Scan on prt4_n_p3 t2_2
 (23 rows)
 
 -- partition-wise join can not be applied if there are no equi-join conditions
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (t1.a < b)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (t1.a < b)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (t1.a < b)
-(12 rows)
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+(16 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partition-wise join
@@ -1819,16 +4520,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
-                     ->  Seq Scan on plt1_p1 t3
-                     ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3
+                     ->  Seq Scan on plt1_p1 t3_1
+                     ->  Seq Scan on plt1_p2 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partition-wise join can not be applied for a join between list and range
 -- partitioned table
@@ -1839,12 +4541,14 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
  Hash Full Join
    Hash Cond: ((t2.c)::text = (t1.c)::text)
    ->  Append
-         ->  Seq Scan on prt1_p1 t2
-         ->  Seq Scan on prt1_p2 t2_1
-         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p0 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p2 t2_2
+         ->  Seq Scan on prt1_p3 t2_3
+         ->  Seq Scan on prt1_p4 t2_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt1_n_p1 t1
                ->  Seq Scan on prt1_n_p2 t1_1
-(10 rows)
+(12 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 2bb1aff..0c7d99c 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -10,25 +10,39 @@ SET enable_partition_wise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -77,11 +91,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -103,20 +125,30 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 
 EXPLAIN (COSTS OFF)
@@ -168,6 +200,85 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -192,28 +303,79 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
 -- test partition matching with N-way join
@@ -221,6 +383,175 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
-- 
1.7.9.5

#6Rajkumar Raghuwanshi
rajkumar.raghuwanshi@enterprisedb.com
In reply to: Ashutosh Bapat (#5)
Re: advanced partition matching algorithm for partition-wise join

On Tue, Sep 5, 2017 at 4:34 PM, Ashutosh Bapat <
ashutosh.bapat@enterprisedb.com> wrote:

I have fixed the issues which were marked as TODOs in the attached
patches. Also, I have included your test change patch in my series of
patches. Are there any other issues you have commented out?

Thanks Ashutosh, All commented issue got fixed. I am working on some

combinations of N-way joins
to test partition matching, will send those as well once done.

#7Antonin Houska
ah@cybertec.at
In reply to: Ashutosh Bapat (#5)
Re: advanced partition matching algorithm for partition-wise join

Ashutosh Bapat <ashutosh.bapat@enterprisedb.com> wrote:

I have fixed the issues which were marked as TODOs in the attached
patches. Also, I have included your test change patch in my series of
patches.

I've noticed that partition_bounds_merge() is called twice from
make_join_rel():

* build_join_rel -> build_joinrel_partition_info -> partition_bounds_merge

* try_partition_wise_join -> partition_bounds_merge

Is this intentional, or just a thinko?

--
Antonin Houska
Cybertec Schönig & Schönig GmbH
Gröhrmühlgasse 26
A-2700 Wiener Neustadt
Web: http://www.postgresql-support.de, http://www.cybertec.at

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

#8Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Antonin Houska (#7)
Re: advanced partition matching algorithm for partition-wise join

On Thu, Sep 7, 2017 at 7:34 PM, Antonin Houska <ah@cybertec.at> wrote:

Ashutosh Bapat <ashutosh.bapat@enterprisedb.com> wrote:

I have fixed the issues which were marked as TODOs in the attached
patches. Also, I have included your test change patch in my series of
patches.

I've noticed that partition_bounds_merge() is called twice from
make_join_rel():

* build_join_rel -> build_joinrel_partition_info -> partition_bounds_merge

* try_partition_wise_join -> partition_bounds_merge

Is this intentional, or just a thinko?

This is expected. partition_bounds_merge() also returns the pairs of
matching partitions. So, we have to call that function for every pair
of joining relations.

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

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

#9Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#8)
3 attachment(s)
Re: advanced partition matching algorithm for partition-wise join

Here's updated patch set based on the basic partition-wise join
committed. The patchset applies on top of the patch to optimize the
case of dummy partitioned tables [1]/messages/by-id/CAFjFpRcPvT5ay9_p3e-k2Cwu4bW_rypON7ceJVWhsU3Uk4Nmmg@mail.gmail.com.

Right now, the advanced partition matching algorithm bails out when
either of the joining relations has a default partition.

[1]: /messages/by-id/CAFjFpRcPvT5ay9_p3e-k2Cwu4bW_rypON7ceJVWhsU3Uk4Nmmg@mail.gmail.com

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

Attachments:

0004-Tests-for-0-1-1-1-and-1-0-partition-matching.patchtext/x-patch; charset=US-ASCII; name=0004-Tests-for-0-1-1-1-and-1-0-partition-matching.patchDownload
From 5a450f85c3ddbc953d95188f90115d3c82b91e70 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Tue, 5 Sep 2017 09:51:41 +0530
Subject: [PATCH 4/4] Tests for 0:1, 1:1 and 1:0 partition matching

Rajkumar Raghuvanshi and Ashutosh Bapat.
---
 src/test/regress/expected/partition_join.out | 3891 ++++++++++++++++++++++----
 src/test/regress/sql/partition_join.sql      |  367 ++-
 2 files changed, 3628 insertions(+), 630 deletions(-)

diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 234b8b5..bc8aa5b 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -8,105 +8,152 @@ SET enable_partition_wise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-(4 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+(7 rows)
 
 -- left outer join, with whole-row reference
 EXPLAIN (COSTS OFF)
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b
    ->  Result
          ->  Append
                ->  Hash Right Join
                      Hash Cond: (t2.b = t1.a)
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                      ->  Hash
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                                  Filter: (b = 0)
                ->  Hash Right Join
                      Hash Cond: (t2_1.b = t1_1.a)
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                                  Filter: (b = 0)
                ->  Hash Right Join
                      Hash Cond: (t2_2.b = t1_2.a)
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                                  Filter: (b = 0)
-(22 rows)
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (t1_3.a = b)
+               ->  Hash Right Join
+                     Hash Cond: (t2_4.b = t1_4.a)
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_p4 t1_4
+                                 Filter: (b = 0)
+(33 rows)
 
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-      t1      |      t2      
---------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
- (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
- (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
- (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
- (550,0,0550) | 
-(12 rows)
+       t1       |       t2       
+----------------+----------------
+ (-250,0,-0250) | 
+ (-200,0,-0200) | 
+ (-150,0,-0150) | (0,-150,-0150)
+ (-100,0,-0100) | 
+ (-50,0,-0050)  | 
+ (0,0,0000)     | (0,0,0000)
+ (50,0,0050)    | 
+ (100,0,0100)   | 
+ (150,0,0150)   | (0,150,0150)
+ (200,0,0200)   | 
+ (250,0,0250)   | 
+ (300,0,0300)   | (0,300,0300)
+ (350,0,0350)   | 
+ (400,0,0400)   | 
+ (450,0,0450)   | (0,450,0450)
+ (500,0,0500)   | 
+ (550,0,0550)   | 
+ (600,0,0600)   | (0,600,0600)
+ (650,0,0650)   | 
+ (700,0,0700)   | 
+ (750,0,0750)   | (0,750,0750)
+(21 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
@@ -119,35 +166,53 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
          ->  Append
                ->  Hash Right Join
                      Hash Cond: (t1.a = t2.b)
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                      ->  Hash
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                                  Filter: (a = 0)
                ->  Hash Right Join
                      Hash Cond: (t1_1.a = t2_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                      ->  Hash
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
+                                 Filter: (a = 0)
+               ->  Hash Right Join
+                     Hash Cond: (t1_2.a = t2_2.b)
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Hash
+                           ->  Seq Scan on prt2_p2 t2_2
                                  Filter: (a = 0)
                ->  Nested Loop Left Join
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
                            Filter: (a = 0)
-                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                           Index Cond: (a = t2_2.b)
-(21 rows)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                           Index Cond: (a = t2_3.b)
+               ->  Hash Right Join
+                     Hash Cond: (t1_4.a = t2_4.b)
+                     ->  Seq Scan on prt1_p4 t1_4
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4 t2_4
+                                 Filter: (a = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+      |       | -225 | -0225
+      |       |  -75 | -0075
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(14 rows)
 
 -- full outer join, with placeholder vars
 EXPLAIN (COSTS OFF)
@@ -155,9 +220,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                             QUERY PLAN                            
 ------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
          ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               Filter: (((50) = prt1_p0.a) OR ((75) = prt2_p0.b))
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
+         ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
                ->  Seq Scan on prt1_p1
@@ -181,7 +254,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               Filter: (((50) = prt1_p4.a) OR ((75) = prt2_p4.b))
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
@@ -218,9 +299,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
          ->  Hash Left Join
+               Hash Cond: (prt1_p0.a = b)
+               ->  Seq Scan on prt1_p0
+                     Filter: ((a < 450) AND (b = 0))
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
+         ->  Hash Left Join
                Hash Cond: (prt1_p1.a = b)
                ->  Seq Scan on prt1_p1
                      Filter: ((a < 450) AND (b = 0))
@@ -234,30 +322,43 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                ->  Hash
                      ->  Seq Scan on prt1_p2
                            Filter: ((a < 450) AND (b = 0))
-(17 rows)
+(24 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-(9 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+(14 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
                          QUERY PLAN                         
 ------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
          ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = b)
+               Filter: ((prt1_p0.b = 0) OR (a = 0))
+               ->  Seq Scan on prt1_p0
+                     Filter: (a < 450)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
+         ->  Hash Full Join
                Hash Cond: (prt1_p1.a = b)
                Filter: ((prt1_p1.b = 0) OR (a = 0))
                ->  Seq Scan on prt1_p1
@@ -281,64 +382,153 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                ->  Hash
                      ->  Result
                            One-Time Filter: false
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt2_p4.b = a)
+               Filter: ((b = 0) OR (prt2_p4.a = 0))
+               ->  Seq Scan on prt2_p4
+                     Filter: (b > 250)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
-(12 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+      |       | 375 | 0375
+      |       | 450 | 0450
+      |       | 525 | 0525
+      |       | 600 | 0600
+      |       | 675 | 0675
+      |       | 750 | 0750
+(20 rows)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Semi Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Semi Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                            Filter: (a = 0)
-         ->  Nested Loop Semi Join
-               Join Filter: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
-               ->  Materialize
-                     ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
-(24 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt1_p3 t2_3
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+                           Filter: (b = 0)
+(37 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
 
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
@@ -349,27 +539,82 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
    ->  Append
          ->  Hash Anti Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Hash Anti Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Hash Anti Join
                Hash Cond: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
-(17 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(27 rows)
 
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
-  sum  |         avg          | sum  |         avg         
--------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+  sum  |         avg          | sum  |        avg         
+-------+----------------------+------+--------------------
+ 95550 | 273.0000000000000000 | 2200 | 6.2857142857142857
 (1 row)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Append
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p0 t1
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+               Index Cond: (a = t1.b)
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.b = t2_1.a)
+         ->  Seq Scan on prt2_p1 t1_1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p1 t2_1
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p2 t1_2
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+               Index Cond: (a = t1_2.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p3 t1_3
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+               Index Cond: (a = t1_3.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p4 t1_4
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+               Index Cond: (a = t1_4.b)
+(27 rows)
+
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+  b   |   c   
+------+-------
+ -225 | -0225
+  -75 | -0075
+   75 | 0075
+  225 | 0225
+  375 | 0375
+  525 | 0525
+  675 | 0675
+(7 rows)
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -382,49 +627,74 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
    ->  Result
          ->  Append
                ->  Nested Loop Left Join
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
                      ->  Nested Loop
-                           ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
+                           ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
                                  Index Cond: (a = t1.a)
-                           ->  Index Scan using iprt2_p1_b on prt2_p1 t3
+                           ->  Index Scan using iprt2_p0_b on prt2_p0 t3
                                  Index Cond: (b = t2.a)
                ->  Nested Loop Left Join
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
                      ->  Nested Loop
-                           ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_1
+                           ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
                                  Index Cond: (a = t1_1.a)
-                           ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
+                           ->  Index Scan using iprt2_p1_b on prt2_p1 t3_1
                                  Index Cond: (b = t2_1.a)
                ->  Nested Loop Left Join
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
                      ->  Nested Loop
-                           ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_2
+                           ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
                                  Index Cond: (a = t1_2.a)
-                           ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
+                           ->  Index Scan using iprt2_p2_b on prt2_p2 t3_2
                                  Index Cond: (b = t2_2.a)
-(28 rows)
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Nested Loop
+                           ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                                 Index Cond: (a = t1_3.a)
+                           ->  Index Scan using iprt2_p3_b on prt2_p3 t3_3
+                                 Index Cond: (b = t2_3.a)
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+                     ->  Nested Loop
+                           ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                                 Index Cond: (a = t1_4.a)
+                           ->  Index Scan using iprt2_p4_b on prt2_p4 t3_4
+                                 Index Cond: (b = t2_4.a)
+(44 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   | t2a | t3a | least 
------+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
- 100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
- 250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
- 400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
- 550 | 0 | 0550 |     |     |      
-(12 rows)
+  a   | b |   c   | t2a  | t3a | least 
+------+---+-------+------+-----+-------
+ -250 | 0 | -0250 |      |     |      
+ -200 | 0 | -0200 |      |     |      
+ -150 | 0 | -0150 | -150 |   0 |  -150
+ -100 | 0 | -0100 |      |     |      
+  -50 | 0 | -0050 |      |     |      
+    0 | 0 | 0000  |    0 |   0 |     0
+   50 | 0 | 0050  |      |     |      
+  100 | 0 | 0100  |      |     |      
+  150 | 0 | 0150  |  150 |   0 |   150
+  200 | 0 | 0200  |      |     |      
+  250 | 0 | 0250  |      |     |      
+  300 | 0 | 0300  |  300 |   0 |   300
+  350 | 0 | 0350  |      |     |      
+  400 | 0 | 0400  |      |     |      
+  450 | 0 | 0450  |  450 |   0 |   450
+  500 | 0 | 0500  |      |     |      
+  550 | 0 | 0550  |      |     |      
+  600 | 0 | 0600  |  600 |   0 |   600
+  650 | 0 | 0650  |      |     |      
+  700 | 0 | 0700  |      |     |      
+  750 | 0 | 0750  |  750 |   0 |   750
+(21 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
@@ -438,64 +708,95 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
          Hash Cond: ((t1.c)::text = (t2.c)::text)
          Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Seq Scan on prt1_p1 t1
-               ->  Seq Scan on prt1_p2 t1_1
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
          ->  Hash
                ->  Append
                      ->  Hash Join
                            Hash Cond: (t2.a = t3.b)
-                           ->  Seq Scan on prt1_p1 t2
+                           ->  Seq Scan on prt1_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t3
+                                 ->  Seq Scan on prt2_p0 t3
                      ->  Hash Join
                            Hash Cond: (t2_1.a = t3_1.b)
-                           ->  Seq Scan on prt1_p2 t2_1
+                           ->  Seq Scan on prt1_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t3_1
+                                 ->  Seq Scan on prt2_p1 t3_1
                      ->  Hash Join
                            Hash Cond: (t2_2.a = t3_2.b)
-                           ->  Seq Scan on prt1_p3 t2_2
+                           ->  Seq Scan on prt1_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p3 t3_2
-(26 rows)
+                                 ->  Seq Scan on prt2_p2 t3_2
+                     ->  Hash Join
+                           Hash Cond: (t2_3.a = t3_3.b)
+                           ->  Seq Scan on prt1_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t3_3
+                     ->  Hash Join
+                           Hash Cond: (t2_4.a = t3_4.b)
+                           ->  Seq Scan on prt1_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t3_4
+(38 rows)
 
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, t2.b t2b, t2.c t2c, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
-  a  | t2a | t2c  
------+-----+------
-   0 |   0 | 0000
-  50 |     | 
- 100 |     | 
- 150 | 150 | 0150
- 200 |     | 
- 250 |     | 
- 300 | 300 | 0300
- 350 |     | 
- 400 |     | 
- 450 | 450 | 0450
- 500 |     | 
- 550 |     | 
-(12 rows)
+  a   | t2a  |  t2c  
+------+------+-------
+ -250 |      | 
+ -200 |      | 
+ -150 | -150 | -0150
+ -100 |      | 
+  -50 |      | 
+    0 |    0 | 0000
+   50 |      | 
+  100 |      | 
+  150 |  150 | 0150
+  200 |      | 
+  250 |      | 
+  300 |  300 | 0300
+  350 |      | 
+  400 |      | 
+  450 |  450 | 0450
+  500 |      | 
+  550 |      | 
+  600 |  600 | 0600
+  650 |      | 
+  700 |      | 
+  750 |  750 | 0750
+(21 rows)
 
 --
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
@@ -506,32 +807,49 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
    ->  Append
          ->  Hash Join
                Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
-               ->  Seq Scan on prt2_e_p1 t2
+               ->  Seq Scan on prt2_e_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p0 t1
+                           Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+               ->  Seq Scan on prt1_e_p1 t1_1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_e_p1 t2_1
+         ->  Hash Join
+               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
+               ->  Seq Scan on prt2_e_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 t1
+                     ->  Seq Scan on prt1_e_p2 t1_2
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
-               ->  Seq Scan on prt2_e_p2 t2_1
+               Hash Cond: (((t2_3.b + t2_3.a) / 2) = ((t1_3.a + t1_3.b) / 2))
+               ->  Seq Scan on prt2_e_p3 t2_3
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 t1_1
+                     ->  Seq Scan on prt1_e_p3 t1_3
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
-               ->  Seq Scan on prt2_e_p3 t2_2
+               Hash Cond: (((t2_4.b + t2_4.a) / 2) = ((t1_4.a + t1_4.b) / 2))
+               ->  Seq Scan on prt2_e_p4 t2_4
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 t1_2
+                     ->  Seq Scan on prt1_e_p4 t1_4
                            Filter: (c = 0)
-(21 rows)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
- 150 | 0 | 150 | 0
- 300 | 0 | 300 | 0
- 450 | 0 | 450 | 0
-(4 rows)
+  a   | c |  b   | c 
+------+---+------+---
+ -250 | 0 | -250 | 0
+ -100 | 0 | -100 | 0
+    0 | 0 |    0 | 0
+    0 | 0 |    0 | 0
+  150 | 0 |  150 | 0
+  300 | 0 |  300 | 0
+  450 | 0 |  450 | 0
+  600 | 0 |  600 | 0
+  750 | 0 |  750 | 0
+(9 rows)
 
 --
 -- N-way join
@@ -545,107 +863,158 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t
    ->  Result
          ->  Append
                ->  Nested Loop
-                     Join Filter: (t1.a = ((t3.a + t3.b) / 2))
+                     Join Filter: (t1.a = t2.b)
                      ->  Hash Join
-                           Hash Cond: (t2.b = t1.a)
-                           ->  Seq Scan on prt2_p1 t2
+                           Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                           ->  Seq Scan on prt1_e_p0 t3
                            ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                                        Filter: (b = 0)
-                     ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                           Index Cond: (((a + b) / 2) = t2.b)
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t2
+                           Index Cond: (b = ((t3.a + t3.b) / 2))
                ->  Nested Loop
                      Join Filter: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
                      ->  Hash Join
                            Hash Cond: (t2_1.b = t1_1.a)
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                                        Filter: (b = 0)
-                     ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
+                     ->  Index Scan using iprt1_e_p4_ab2 on prt1_e_p1 t3_1
                            Index Cond: (((a + b) / 2) = t2_1.b)
                ->  Nested Loop
                      Join Filter: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
                      ->  Hash Join
                            Hash Cond: (t2_2.b = t1_2.a)
-                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                                        Filter: (b = 0)
-                     ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
+                     ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_2
                            Index Cond: (((a + b) / 2) = t2_2.b)
-(34 rows)
+               ->  Nested Loop
+                     Join Filter: (t1_3.a = ((t3_3.a + t3_3.b) / 2))
+                     ->  Nested Loop
+                           ->  Seq Scan on prt1_p3 t1_3
+                                 Filter: (b = 0)
+                           ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                                 Index Cond: (b = t1_3.a)
+                     ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                           Index Cond: (((a + b) / 2) = t2_3.b)
+               ->  Nested Loop
+                     Join Filter: (t1_4.a = t2_4.b)
+                     ->  Hash Join
+                           Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                           ->  Seq Scan on prt1_e_p4 t3_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_p4 t1_4
+                                       Filter: (b = 0)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                           Index Cond: (b = ((t3_4.a + t3_4.b) / 2))
+(53 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
-(4 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(8 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                             QUERY PLAN                             
---------------------------------------------------------------------
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Result
          ->  Append
                ->  Hash Right Join
                      Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
-                     ->  Seq Scan on prt1_e_p1 t3
+                     ->  Seq Scan on prt1_e_p0 t3
                      ->  Hash
                            ->  Hash Right Join
                                  Hash Cond: (t2.b = t1.a)
-                                 ->  Seq Scan on prt2_p1 t2
+                                 ->  Seq Scan on prt2_p0 t2
                                  ->  Hash
-                                       ->  Seq Scan on prt1_p1 t1
+                                       ->  Seq Scan on prt1_p0 t1
                                              Filter: (b = 0)
                ->  Hash Right Join
                      Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_1.a)
-                     ->  Seq Scan on prt1_e_p2 t3_1
+                     ->  Seq Scan on prt1_e_p1 t3_1
                      ->  Hash
                            ->  Hash Right Join
                                  Hash Cond: (t2_1.b = t1_1.a)
-                                 ->  Seq Scan on prt2_p2 t2_1
+                                 ->  Seq Scan on prt2_p1 t2_1
                                  ->  Hash
-                                       ->  Seq Scan on prt1_p2 t1_1
+                                       ->  Seq Scan on prt1_p1 t1_1
                                              Filter: (b = 0)
                ->  Hash Right Join
                      Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_2.a)
-                     ->  Seq Scan on prt1_e_p3 t3_2
+                     ->  Seq Scan on prt1_e_p2 t3_2
                      ->  Hash
                            ->  Hash Right Join
                                  Hash Cond: (t2_2.b = t1_2.a)
-                                 ->  Seq Scan on prt2_p3 t2_2
+                                 ->  Seq Scan on prt2_p2 t2_2
+                                 ->  Hash
+                                       ->  Seq Scan on prt1_p2 t1_2
+                                             Filter: (b = 0)
+               ->  Nested Loop Left Join
+                     ->  Nested Loop Left Join
+                           ->  Seq Scan on prt1_p3 t1_3
+                                 Filter: (b = 0)
+                           ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                                 Index Cond: (t1_3.a = b)
+                     ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                           Index Cond: (t1_3.a = ((a + b) / 2))
+               ->  Hash Right Join
+                     Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                     ->  Seq Scan on prt1_e_p4 t3_4
+                     ->  Hash
+                           ->  Hash Right Join
+                                 Hash Cond: (t2_4.b = t1_4.a)
+                                 ->  Seq Scan on prt2_p4 t2_4
                                  ->  Hash
-                                       ->  Seq Scan on prt1_p3 t1_2
+                                       ->  Seq Scan on prt1_p4 t1_4
                                              Filter: (b = 0)
-(34 rows)
+(52 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                               QUERY PLAN                                
--------------------------------------------------------------------------
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Result
@@ -653,48 +1022,75 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                ->  Nested Loop Left Join
                      ->  Hash Right Join
                            Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t3
+                                 ->  Seq Scan on prt1_e_p0 t3
                                        Filter: (c = 0)
-                     ->  Index Scan using iprt2_p1_b on prt2_p1 t2
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t2
                            Index Cond: (t1.a = b)
                ->  Nested Loop Left Join
                      ->  Hash Right Join
                            Hash Cond: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                 ->  Seq Scan on prt1_e_p1 t3_1
                                        Filter: (c = 0)
-                     ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
                            Index Cond: (t1_1.a = b)
                ->  Nested Loop Left Join
                      ->  Hash Right Join
                            Hash Cond: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                 ->  Seq Scan on prt1_e_p2 t3_2
                                        Filter: (c = 0)
-                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
                            Index Cond: (t1_2.a = b)
-(31 rows)
+               ->  Nested Loop Left Join
+                     ->  Nested Loop Left Join
+                           ->  Seq Scan on prt1_e_p3 t3_3
+                                 Filter: (c = 0)
+                           ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                                 Index Cond: (a = ((t3_3.a + t3_3.b) / 2))
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (t1_3.a = b)
+               ->  Nested Loop Left Join
+                     ->  Hash Right Join
+                           Hash Cond: (t1_4.a = ((t3_4.a + t3_4.b) / 2))
+                           ->  Seq Scan on prt1_p4 t1_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t3_4
+                                       Filter: (c = 0)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                           Index Cond: (t1_4.a = b)
+(48 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -703,23 +1099,36 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                                                       QUERY PLAN                                                      
 ----------------------------------------------------------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   Sort Key: prt1_p0.a, prt2_p0.b, ((prt1_e_p0.a + prt1_e_p0.b))
    ->  Result
          ->  Append
                ->  Hash Full Join
-                     Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
-                     Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+                     Hash Cond: (prt1_p0.a = ((prt1_e_p0.a + prt1_e_p0.b) / 2))
+                     Filter: ((prt1_p0.a = (50)) OR (prt2_p0.b = (75)) OR (((prt1_e_p0.a + prt1_e_p0.b) / 2) = (50)))
                      ->  Hash Full Join
-                           Hash Cond: (prt1_p1.a = prt2_p1.b)
-                           ->  Seq Scan on prt1_p1
+                           Hash Cond: (prt1_p0.a = prt2_p0.b)
+                           ->  Seq Scan on prt1_p0
                                  Filter: (b = 0)
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1
+                                 ->  Seq Scan on prt2_p0
                                        Filter: (a = 0)
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p1
+                           ->  Seq Scan on prt1_e_p0
                                  Filter: (c = 0)
                ->  Hash Full Join
+                     Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = prt1_p1.a)
+                     Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+                     ->  Seq Scan on prt1_e_p1
+                           Filter: (c = 0)
+                     ->  Hash
+                           ->  Hash Full Join
+                                 Hash Cond: (prt1_p1.a = prt2_p1.b)
+                                 ->  Seq Scan on prt1_p1
+                                       Filter: (b = 0)
+                                 ->  Hash
+                                       ->  Seq Scan on prt2_p1
+                                             Filter: (a = 0)
+               ->  Hash Full Join
                      Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
                      Filter: ((prt1_p2.a = (50)) OR (prt2_p2.b = (75)) OR (((prt1_e_p2.a + prt1_e_p2.b) / 2) = (50)))
                      ->  Hash Full Join
@@ -745,7 +1154,20 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                      ->  Hash
                            ->  Seq Scan on prt1_e_p3
                                  Filter: (c = 0)
-(43 rows)
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = ((prt1_e_p4.a + prt1_e_p4.b) / 2))
+                     Filter: ((prt1_p4.a = (50)) OR (prt2_p4.b = (75)) OR (((prt1_e_p4.a + prt1_e_p4.b) / 2) = (50)))
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_p4.a = prt2_p4.b)
+                           ->  Seq Scan on prt1_p4
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4
+                                       Filter: (a = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p4
+                                 Filter: (c = 0)
+(69 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
@@ -763,173 +1185,261 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = t1_3.b)
+               Join Filter: (t1.a = t1_5.b)
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Join
-                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
-                           ->  Seq Scan on prt1_e_p1 t2
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t1_3
+                                 ->  Seq Scan on prt2_p0 t1_5
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
                      Index Cond: (a = ((t2.a + t2.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_1.a = t1_4.b)
+               Join Filter: (t1_1.a = t1_6.b)
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Join
-                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
-                           ->  Seq Scan on prt1_e_p2 t2_1
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_6.b)
+                           ->  Seq Scan on prt1_e_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t1_4
+                                 ->  Seq Scan on prt2_p1 t1_6
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
                      Index Cond: (a = ((t2_1.a + t2_1.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_2.a = t1_5.b)
+               Join Filter: (t1_2.a = t1_7.b)
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
+                           ->  Seq Scan on prt2_p2 t1_7
                                  Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
+                           ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t2_2
+                                 Index Cond: (((a + b) / 2) = t1_7.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t1_8.b)
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Nested Loop
+                           ->  Seq Scan on prt2_p3 t1_8
+                                 Filter: (a = 0)
+                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_3
+                                 Index Cond: (((a + b) / 2) = t1_8.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = ((t2_3.a + t2_3.b) / 2))
+                     Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t1_9.b)
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Join
+                           Hash Cond: (((t2_4.a + t2_4.b) / 2) = t1_9.b)
+                           ->  Seq Scan on prt1_e_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t1_9
+                                       Filter: (a = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = ((t2_4.a + t2_4.b) / 2))
+                     Filter: (b = 0)
+(66 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                                  QUERY PLAN                                   
--------------------------------------------------------------------------------
+                                   QUERY PLAN                                    
+---------------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_5.b = ((t1_10.a + t1_10.b) / 2))
+                           ->  Seq Scan on prt2_p0 t1_5
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p0 t1_10
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t1_5.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_6.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
-                           ->  Seq Scan on prt2_p1 t1_3
+                           Hash Cond: (t1_6.b = ((t1_11.a + t1_11.b) / 2))
+                           ->  Seq Scan on prt2_p1 t1_6
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t1_6
+                                 ->  Seq Scan on prt1_e_p1 t1_11
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
-                     Index Cond: (a = t1_3.b)
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
+                     Index Cond: (a = t1_6.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_7.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
-                           ->  Seq Scan on prt2_p2 t1_4
+                           Hash Cond: (t1_7.b = ((t1_12.a + t1_12.b) / 2))
+                           ->  Seq Scan on prt2_p2 t1_7
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t1_7
+                                 ->  Seq Scan on prt1_e_p2 t1_12
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
-                     Index Cond: (a = t1_4.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t1_7.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  Unique
                      ->  Sort
-                           Sort Key: t1_5.b
+                           Sort Key: t1_8.b
                            ->  Hash Semi Join
-                                 Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
-                                 ->  Seq Scan on prt2_p3 t1_5
+                                 Hash Cond: (t1_8.b = ((t1_13.a + t1_13.b) / 2))
+                                 ->  Seq Scan on prt2_p3 t1_8
                                  ->  Hash
-                                       ->  Seq Scan on prt1_e_p3 t1_8
+                                       ->  Seq Scan on prt1_e_p3 t1_13
                                              Filter: (c = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t1_5.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t1_8.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_9.b = ((t1_14.a + t1_14.b) / 2))
+                           ->  Seq Scan on prt2_p4 t1_9
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t1_14
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = t1_9.b)
                      Filter: (b = 0)
-(40 rows)
+(64 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 -- test merge joins
 SET enable_hashjoin TO off;
 SET enable_nestloop TO off;
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                           QUERY PLAN                           
-----------------------------------------------------------------
+                            QUERY PLAN                            
+------------------------------------------------------------------
  Merge Append
    Sort Key: t1.a
    ->  Merge Semi Join
-         Merge Cond: (t1.a = t1_3.b)
+         Merge Cond: (t1.a = t1_5.b)
          ->  Sort
                Sort Key: t1.a
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+               Merge Cond: (t1_5.b = (((t1_10.a + t1_10.b) / 2)))
                ->  Sort
-                     Sort Key: t1_3.b
-                     ->  Seq Scan on prt2_p1 t1_3
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on prt2_p0 t1_5
                ->  Sort
-                     Sort Key: (((t1_6.a + t1_6.b) / 2))
-                     ->  Seq Scan on prt1_e_p1 t1_6
+                     Sort Key: (((t1_10.a + t1_10.b) / 2))
+                     ->  Seq Scan on prt1_e_p0 t1_10
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_1.a = t1_4.b)
+         Merge Cond: (t1_1.a = t1_6.b)
          ->  Sort
                Sort Key: t1_1.a
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+               Merge Cond: (t1_6.b = (((t1_11.a + t1_11.b) / 2)))
                ->  Sort
-                     Sort Key: t1_4.b
-                     ->  Seq Scan on prt2_p2 t1_4
+                     Sort Key: t1_6.b
+                     ->  Seq Scan on prt2_p1 t1_6
                ->  Sort
-                     Sort Key: (((t1_7.a + t1_7.b) / 2))
-                     ->  Seq Scan on prt1_e_p2 t1_7
+                     Sort Key: (((t1_11.a + t1_11.b) / 2))
+                     ->  Seq Scan on prt1_e_p1 t1_11
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_2.a = t1_5.b)
+         Merge Cond: (t1_2.a = t1_7.b)
          ->  Sort
                Sort Key: t1_2.a
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+               Merge Cond: (t1_7.b = (((t1_12.a + t1_12.b) / 2)))
                ->  Sort
-                     Sort Key: t1_5.b
-                     ->  Seq Scan on prt2_p3 t1_5
+                     Sort Key: t1_7.b
+                     ->  Seq Scan on prt2_p2 t1_7
+               ->  Sort
+                     Sort Key: (((t1_12.a + t1_12.b) / 2))
+                     ->  Seq Scan on prt1_e_p2 t1_12
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_3.a = t1_8.b)
+         ->  Sort
+               Sort Key: t1_3.a
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_8.b = (((t1_13.a + t1_13.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_8.b
+                     ->  Seq Scan on prt2_p3 t1_8
+               ->  Sort
+                     Sort Key: (((t1_13.a + t1_13.b) / 2))
+                     ->  Seq Scan on prt1_e_p3 t1_13
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_4.a = t1_9.b)
+         ->  Sort
+               Sort Key: t1_4.a
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_9.b = (((t1_14.a + t1_14.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_9.b
+                     ->  Seq Scan on prt2_p4 t1_9
                ->  Sort
-                     Sort Key: (((t1_8.a + t1_8.b) / 2))
-                     ->  Seq Scan on prt1_e_p3 t1_8
+                     Sort Key: (((t1_14.a + t1_14.b) / 2))
+                     ->  Seq Scan on prt1_e_p4 t1_14
                            Filter: (c = 0)
-(47 rows)
+(77 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
@@ -947,14 +1457,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                                  Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
                                  ->  Sort
                                        Sort Key: (((t3.a + t3.b) / 2))
-                                       ->  Seq Scan on prt1_e_p1 t3
+                                       ->  Seq Scan on prt1_e_p0 t3
                                              Filter: (c = 0)
                                  ->  Sort
                                        Sort Key: t1.a
-                                       ->  Seq Scan on prt1_p1 t1
+                                       ->  Seq Scan on prt1_p0 t1
                      ->  Sort
                            Sort Key: t2.b
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                ->  Merge Left Join
                      Merge Cond: (t1_1.a = t2_1.b)
                      ->  Sort
@@ -963,14 +1473,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                                  Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
                                  ->  Sort
                                        Sort Key: (((t3_1.a + t3_1.b) / 2))
-                                       ->  Seq Scan on prt1_e_p2 t3_1
+                                       ->  Seq Scan on prt1_e_p1 t3_1
                                              Filter: (c = 0)
                                  ->  Sort
                                        Sort Key: t1_1.a
-                                       ->  Seq Scan on prt1_p2 t1_1
+                                       ->  Seq Scan on prt1_p1 t1_1
                      ->  Sort
                            Sort Key: t2_1.b
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                ->  Merge Left Join
                      Merge Cond: (t1_2.a = t2_2.b)
                      ->  Sort
@@ -979,225 +1489,2361 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                                  Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
                                  ->  Sort
                                        Sort Key: (((t3_2.a + t3_2.b) / 2))
-                                       ->  Seq Scan on prt1_e_p3 t3_2
+                                       ->  Seq Scan on prt1_e_p2 t3_2
                                              Filter: (c = 0)
                                  ->  Sort
                                        Sort Key: t1_2.a
-                                       ->  Seq Scan on prt1_p3 t1_2
+                                       ->  Seq Scan on prt1_p2 t1_2
                      ->  Sort
                            Sort Key: t2_2.b
-                           ->  Seq Scan on prt2_p3 t2_2
-(52 rows)
+                           ->  Seq Scan on prt2_p2 t2_2
+               ->  Merge Left Join
+                     Merge Cond: (t1_3.a = t2_3.b)
+                     ->  Sort
+                           Sort Key: t1_3.a
+                           ->  Merge Left Join
+                                 Merge Cond: ((((t3_3.a + t3_3.b) / 2)) = t1_3.a)
+                                 ->  Sort
+                                       Sort Key: (((t3_3.a + t3_3.b) / 2))
+                                       ->  Seq Scan on prt1_e_p3 t3_3
+                                             Filter: (c = 0)
+                                 ->  Sort
+                                       Sort Key: t1_3.a
+                                       ->  Seq Scan on prt1_p3 t1_3
+                     ->  Sort
+                           Sort Key: t2_3.b
+                           ->  Seq Scan on prt2_p3 t2_3
+               ->  Merge Left Join
+                     Merge Cond: (t1_4.a = t2_4.b)
+                     ->  Sort
+                           Sort Key: t1_4.a
+                           ->  Merge Left Join
+                                 Merge Cond: ((((t3_4.a + t3_4.b) / 2)) = t1_4.a)
+                                 ->  Sort
+                                       Sort Key: (((t3_4.a + t3_4.b) / 2))
+                                       ->  Seq Scan on prt1_e_p4 t3_4
+                                             Filter: (c = 0)
+                                 ->  Sort
+                                       Sort Key: t1_4.a
+                                       ->  Seq Scan on prt1_p4 t1_4
+                     ->  Sort
+                           Sort Key: t2_4.b
+                           ->  Seq Scan on prt2_p4 t2_4
+(84 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
+
+-- MergeAppend on nullable column
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                        QUERY PLAN                         
+-----------------------------------------------------------
+ Sort
+   Sort Key: prt1_p0.a, b
+   ->  Append
+         ->  Merge Left Join
+               Merge Cond: (prt1_p0.a = b)
+               ->  Sort
+                     Sort Key: prt1_p0.a
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
+               ->  Sort
+                     Sort Key: b
+                     ->  Result
+                           One-Time Filter: false
+         ->  Merge Left Join
+               Merge Cond: (prt1_p1.a = b)
+               ->  Sort
+                     Sort Key: prt1_p1.a
+                     ->  Seq Scan on prt1_p1
+                           Filter: ((a < 450) AND (b = 0))
+               ->  Sort
+                     Sort Key: b
+                     ->  Result
+                           One-Time Filter: false
+         ->  Merge Left Join
+               Merge Cond: (prt1_p2.a = prt2_p2.b)
+               ->  Sort
+                     Sort Key: prt1_p2.a
+                     ->  Seq Scan on prt1_p2
+                           Filter: ((a < 450) AND (b = 0))
+               ->  Sort
+                     Sort Key: prt2_p2.b
+                     ->  Seq Scan on prt2_p2
+                           Filter: (b > 250)
+(33 rows)
+
+SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a   |  b  
+------+-----
+ -250 |    
+ -200 |    
+ -150 |    
+ -100 |    
+  -50 |    
+    0 |    
+   50 |    
+  100 |    
+  150 |    
+  200 |    
+  250 |    
+  300 | 300
+  350 |    
+  400 |    
+(14 rows)
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -150 | -0150 | 0 | -0150
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+(7 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (t1_3.a = b)
+         ->  Hash Right Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 |   | 
+ -200 | -0200 |   | 
+ -150 | -0150 | 0 | -0150
+ -100 | -0100 |   | 
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+(21 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -200 | 0 | -0200
+ -100 | 0 | -0100
+  -50 | 0 | -0050
+   50 | 0 | 0050
+  100 | 0 | 0100
+  200 | 0 | 0200
+  250 | 0 | 0250
+  350 | 0 | 0350
+  400 | 0 | 0400
+  500 | 0 | 0500
+  550 | 0 | 0550
+  650 | 0 | 0650
+  700 | 0 | 0700
+(14 rows)
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.b
+   ->  Hash Right Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p5 t2_5
+                           Filter: (a = 0)
+(24 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: (t1.a = t2.b)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.a, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
+                     Filter: (a = 0)
+(22 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
+               ->  Seq Scan on prt1_m_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p1
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
+               ->  Seq Scan on prt1_m_p2
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p2
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
+               ->  Seq Scan on prt1_m_p3
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p3
+                           Filter: (c = 0)
+(24 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+  a  | c |  b  | c 
+-----+---+-----+---
+   0 | 0 |   0 | 0
+  50 | 0 |     |  
+ 100 | 0 |     |  
+ 150 | 0 | 150 | 0
+ 200 | 0 |     |  
+ 250 | 0 |     |  
+ 300 | 0 | 300 | 0
+ 350 | 0 |     |  
+ 400 | 0 |     |  
+ 450 | 0 | 450 | 0
+ 500 | 0 |     |  
+ 550 | 0 |     |  
+     |   |  75 | 0
+     |   | 225 | 0
+     |   | 375 | 0
+     |   | 525 | 0
+(16 rows)
+
+--
+-- tests for list partitioned tables.
+--
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Result
+         ->  Append
+               ->  Hash Right Join
+                     Hash Cond: ((t1.c)::text = (t2.c)::text)
+                     Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Hash
+                           ->  Seq Scan on plt2_p4 t2
+               ->  Hash Left Join
+                     Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+                     Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Hash
+                           ->  Seq Scan on plt1_p1 t1_1
+               ->  Hash Left Join
+                     Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+                     Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Hash
+                           ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash Right Join
+                     Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+                     Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+                     ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt2_p3 t2_3
+(28 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  HashAggregate
+                           Group Key: (t2.c)::text
+                           ->  Seq Scan on plt2_p4 t2
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
+--
+-- list partitioned by expression
+--
+CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
+ANALYZE plt1_e;
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+                                       QUERY PLAN                                       
+----------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t3.c
+   ->  HashAggregate
+         Group Key: t1.c, t2.c, t3.c
+         ->  Result
+               ->  Append
+                     ->  Hash Join
+                           Hash Cond: ((t1.c)::text = (t2.c)::text)
+                           ->  Hash Join
+                                 Hash Cond: ((t1.c)::text = ltrim(t3.c, 'A'::text))
+                                 ->  Seq Scan on plt1_p4 t1
+                                 ->  Hash
+                                       ->  Seq Scan on plt1_e_p4 t3
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p4 t2
+                     ->  Hash Join
+                           Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+                           ->  Hash Join
+                                 Hash Cond: ((t2_1.c)::text = ltrim(t3_1.c, 'A'::text))
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Hash
+                                       ->  Seq Scan on plt1_e_p1 t3_1
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p1 t1_1
+                     ->  Hash Join
+                           Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+                           ->  Hash Join
+                                 Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
+                                 ->  Seq Scan on plt1_p2 t1_2
+                                 ->  Hash
+                                       ->  Seq Scan on plt1_e_p2 t3_2
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p2 t2_2
+                     ->  Hash Join
+                           Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+                           ->  Hash Join
+                                 Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                                 ->  Seq Scan on plt1_p3 t1_3
+                                 ->  Hash
+                                       ->  Seq Scan on plt1_e_p3 t3_3
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p3 t2_3
+(42 rows)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  HashAggregate
+                           Group Key: (t2.c)::text
+                           ->  Seq Scan on plt2_p4 t2
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p5 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_3
+                                 ->  Seq Scan on plt2_p3 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Result
+         ->  Append
+               ->  Hash Left Join
+                     Hash Cond: ((t2.c)::text = (t1.c)::text)
+                     Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Hash
+                           ->  Seq Scan on plt1_p4 t1
+               ->  Hash Left Join
+                     Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+                     Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Hash
+                           ->  Seq Scan on plt1_p1 t1_1
+               ->  Hash Left Join
+                     Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+                     Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Hash
+                           ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash Right Join
+                     Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+                     Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+                     ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt2_p3 t2_3
+(28 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
 
--- MergeAppend on nullable column
+-- anti join
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-                        QUERY PLAN                         
------------------------------------------------------------
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: t1.a, t1.c
    ->  Append
-         ->  Merge Left Join
-               Merge Cond: (prt1_p1.a = b)
-               ->  Sort
-                     Sort Key: prt1_p1.a
-                     ->  Seq Scan on prt1_p1
-                           Filter: ((a < 450) AND (b = 0))
-               ->  Sort
-                     Sort Key: b
-                     ->  Result
-                           One-Time Filter: false
-         ->  Merge Left Join
-               Merge Cond: (prt1_p2.a = prt2_p2.b)
-               ->  Sort
-                     Sort Key: prt1_p2.a
-                     ->  Seq Scan on prt1_p2
-                           Filter: ((a < 450) AND (b = 0))
-               ->  Sort
-                     Sort Key: prt2_p2.b
-                     ->  Seq Scan on prt2_p2
-                           Filter: (b > 250)
-(23 rows)
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
 
-SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  b  
------+-----
-   0 |    
-  50 |    
- 100 |    
- 150 |    
- 200 |    
- 250 |    
- 300 | 300
- 350 |    
- 400 |    
-(9 rows)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
 
-RESET enable_hashjoin;
-RESET enable_nestloop;
---
--- partitioned by multiple columns
---
-CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
-CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
-ANALYZE prt1_m;
-CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
-CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
-ANALYZE prt2_m;
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-                                                             QUERY PLAN                                                             
-------------------------------------------------------------------------------------------------------------------------------------
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
  Sort
-   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   Sort Key: t1.a, t1.c
    ->  Append
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
-               ->  Seq Scan on prt1_m_p1
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p1
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
-               ->  Seq Scan on prt1_m_p2
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p2
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
-               ->  Seq Scan on prt1_m_p3
-                     Filter: (c = 0)
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_m_p3
-                           Filter: (c = 0)
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
 (24 rows)
 
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
-  50 | 0 |     |  
- 100 | 0 |     |  
- 150 | 0 | 150 | 0
- 200 | 0 |     |  
- 250 | 0 |     |  
- 300 | 0 | 300 | 0
- 350 | 0 |     |  
- 400 | 0 |     |  
- 450 | 0 | 450 | 0
- 500 | 0 |     |  
- 550 | 0 |     |  
-     |   |  75 | 0
-     |   | 225 | 0
-     |   | 375 | 0
-     |   | 525 | 0
-(16 rows)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
 
---
--- tests for list partitioned tables.
---
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
 ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
---
--- list partitioned by expression
---
-CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1_e;
--- test partition matching with N-way join
+-- inner join
 EXPLAIN (COSTS OFF)
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                      QUERY PLAN                                      
---------------------------------------------------------------------------------------
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
  Sort
-   Sort Key: t1.c, t3.c
-   ->  HashAggregate
-         Group Key: t1.c, t2.c, t3.c
-         ->  Result
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
                ->  Append
-                     ->  Hash Join
-                           Hash Cond: (t1.c = t2.c)
-                           ->  Seq Scan on plt1_p1 t1
-                           ->  Hash
-                                 ->  Hash Join
-                                       Hash Cond: (t2.c = ltrim(t3.c, 'A'::text))
-                                       ->  Seq Scan on plt2_p1 t2
-                                       ->  Hash
-                                             ->  Seq Scan on plt1_e_p1 t3
-                     ->  Hash Join
-                           Hash Cond: (t1_1.c = t2_1.c)
-                           ->  Seq Scan on plt1_p2 t1_1
-                           ->  Hash
-                                 ->  Hash Join
-                                       Hash Cond: (t2_1.c = ltrim(t3_1.c, 'A'::text))
-                                       ->  Seq Scan on plt2_p2 t2_1
-                                       ->  Hash
-                                             ->  Seq Scan on plt1_e_p2 t3_1
-                     ->  Hash Join
-                           Hash Cond: (t1_2.c = t2_2.c)
-                           ->  Seq Scan on plt1_p3 t1_2
-                           ->  Hash
-                                 ->  Hash Join
-                                       Hash Cond: (t2_2.c = ltrim(t3_2.c, 'A'::text))
-                                       ->  Seq Scan on plt2_p3 t2_2
-                                       ->  Hash
-                                             ->  Seq Scan on plt1_e_p3 t3_2
-(33 rows)
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
 
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  74.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 124.0000000000000000 | 124.5000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 224.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 274.0000000000000000 | 274.5000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 374.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 424.0000000000000000 | 424.5000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 524.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 574.0000000000000000 | 574.5000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Seq Scan on plt2_p2 t2_2
+                                 ->  Seq Scan on plt2_p3 t2_3
+(22 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1225,16 +3871,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
@@ -1245,16 +3897,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 --
 -- multiple levels of partitioning 
@@ -1615,64 +4273,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
  Hash Join
    Hash Cond: (t1.a = t2.a)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt4_n_p1 t2
                ->  Seq Scan on prt4_n_p2 t2_1
                ->  Seq Scan on prt4_n_p3 t2_2
-(11 rows)
+(13 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a and t1.a = t3.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                        QUERY PLAN                        
+----------------------------------------------------------
  Hash Join
-   Hash Cond: (t2.a = t1.a)
+   Hash Cond: (t3.b = t1.a)
    ->  Append
-         ->  Seq Scan on prt4_n_p1 t2
-         ->  Seq Scan on prt4_n_p2 t2_1
-         ->  Seq Scan on prt4_n_p3 t2_2
+         ->  Seq Scan on prt2_p0 t3
+         ->  Seq Scan on prt2_p1 t3_1
+         ->  Seq Scan on prt2_p2 t3_2
+         ->  Seq Scan on prt2_p3 t3_3
+         ->  Seq Scan on prt2_p4 t3_4
+         ->  Seq Scan on prt2_p5 t3_5
    ->  Hash
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.a = t3_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_2.b)
-                     ->  Seq Scan on prt1_p3 t1_2
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3 t3_2
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt4_n_p1 t2
+                           ->  Seq Scan on prt4_n_p2 t2_1
+                           ->  Seq Scan on prt4_n_p3 t2_2
 (23 rows)
 
 -- partition-wise join can not be applied if there are no equi-join conditions
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (t1.a < b)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (t1.a < b)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (t1.a < b)
-(12 rows)
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+(16 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partition-wise join
@@ -1758,16 +4422,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
-                     ->  Seq Scan on plt1_p1 t3
-                     ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3
+                     ->  Seq Scan on plt1_p1 t3_1
+                     ->  Seq Scan on plt1_p2 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partition-wise join can not be applied for a join between list and range
 -- partitioned table
@@ -1778,12 +4443,14 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
  Hash Full Join
    Hash Cond: ((t2.c)::text = (t1.c)::text)
    ->  Append
-         ->  Seq Scan on prt1_p1 t2
-         ->  Seq Scan on prt1_p2 t2_1
-         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p0 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p2 t2_2
+         ->  Seq Scan on prt1_p3 t2_3
+         ->  Seq Scan on prt1_p4 t2_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt1_n_p1 t1
                ->  Seq Scan on prt1_n_p2 t1_1
-(10 rows)
+(12 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index ca525d9..5812792 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -10,25 +10,39 @@ SET enable_partition_wise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -67,11 +81,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -93,20 +115,30 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 
 EXPLAIN (COSTS OFF)
@@ -163,6 +195,85 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -187,28 +298,79 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
 -- test partition matching with N-way join
@@ -216,6 +378,175 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
-- 
1.7.9.5

0002-Modify-bound-comparision-functions-to-accept-members.patchtext/x-patch; charset=US-ASCII; name=0002-Modify-bound-comparision-functions-to-accept-members.patchDownload
From cc0b81ee8891677adb8bff3e98d80993788632dc Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Thu, 6 Jul 2017 14:15:22 +0530
Subject: [PATCH 2/4] Modify bound comparision functions to accept members of
 PartitionKey

Functions partition_bound_cmp(), partition_rbound_cmp() and
partition_rbound_datum_cmp() are required to merge partition bounds
from joining relations. While doing so, we do not have access to the
PartitionKey of either relations. So, modify these functions to accept
only required members of PartitionKey so that the functions can be
reused for merging bounds.

Ashutosh Bapat.
---
 src/backend/catalog/partition.c |   76 ++++++++++++++++++++++-----------------
 1 file changed, 44 insertions(+), 32 deletions(-)

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index ebda85e..87b3fbc 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -134,15 +134,17 @@ static List *generate_partition_qual(Relation rel);
 
 static PartitionRangeBound *make_one_range_bound(PartitionKey key, int index,
 					 List *datums, bool lower);
-static int32 partition_rbound_cmp(PartitionKey key,
-					 Datum *datums1, PartitionRangeDatumKind *kind1,
-					 bool lower1, PartitionRangeBound *b2);
-static int32 partition_rbound_datum_cmp(PartitionKey key,
-						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
+static int32 partition_rbound_cmp(int partnatts, FmgrInfo *partsupfunc,
+					 Oid *partcollation, Datum *datums1,
+					 PartitionRangeDatumKind *kind1, bool lower1,
+					 PartitionRangeBound *b2);
+static int32 partition_rbound_datum_cmp(int partnatts, FmgrInfo *partsupfunc,
+						   Oid *partcollation, Datum *rb_datums,
+						   PartitionRangeDatumKind *rb_kind,
 						   Datum *tuple_datums);
 
-static int32 partition_bound_cmp(PartitionKey key,
-					PartitionBoundInfo boundinfo,
+static int32 partition_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+					Oid *partcollation, PartitionBoundInfo boundinfo,
 					int offset, void *probe, bool probe_is_bound);
 static int partition_bound_bsearch(PartitionKey key,
 						PartitionBoundInfo boundinfo,
@@ -859,8 +861,9 @@ check_new_partition_bound(char *relname, Relation parent,
 				 * First check if the resulting range would be empty with
 				 * specified lower and upper bounds
 				 */
-				if (partition_rbound_cmp(key, lower->datums, lower->kind, true,
-										 upper) >= 0)
+				if (partition_rbound_cmp(key->partnatts, key->partsupfunc,
+										 key->partcollation, lower->datums,
+										 lower->kind, true, upper) >= 0)
 				{
 					ereport(ERROR,
 							(errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
@@ -913,9 +916,11 @@ check_new_partition_bound(char *relname, Relation parent,
 						{
 							int32		cmpval;
 
-							cmpval = partition_bound_cmp(key, boundinfo,
-														 offset + 1, upper,
-														 true);
+							cmpval = partition_bound_cmp(key->partnatts,
+														 key->partsupfunc,
+														 key->partcollation,
+														 boundinfo, offset + 1,
+														 upper, true);
 							if (cmpval < 0)
 							{
 								/*
@@ -2595,7 +2600,9 @@ qsort_partition_rbound_cmp(const void *a, const void *b, void *arg)
 	PartitionRangeBound *b2 = (*(PartitionRangeBound *const *) b);
 	PartitionKey key = (PartitionKey) arg;
 
-	return partition_rbound_cmp(key, b1->datums, b1->kind, b1->lower, b2);
+	return partition_rbound_cmp(key->partnatts, key->partsupfunc,
+								key->partcollation, b1->datums, b1->kind,
+								b1->lower, b2);
 }
 
 /*
@@ -2612,7 +2619,7 @@ qsort_partition_rbound_cmp(const void *a, const void *b, void *arg)
  * two contiguous partitions.
  */
 static int32
-partition_rbound_cmp(PartitionKey key,
+partition_rbound_cmp(int partnatts, FmgrInfo *partsupfunc, Oid *partcollation,
 					 Datum *datums1, PartitionRangeDatumKind *kind1,
 					 bool lower1, PartitionRangeBound *b2)
 {
@@ -2622,7 +2629,7 @@ partition_rbound_cmp(PartitionKey key,
 	PartitionRangeDatumKind *kind2 = b2->kind;
 	bool		lower2 = b2->lower;
 
-	for (i = 0; i < key->partnatts; i++)
+	for (i = 0; i < partnatts; i++)
 	{
 		/*
 		 * First, handle cases where the column is unbounded, which should not
@@ -2643,8 +2650,8 @@ partition_rbound_cmp(PartitionKey key,
 			 */
 			break;
 
-		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[i],
-												 key->partcollation[i],
+		cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[i],
+												 partcollation[i],
 												 datums1[i],
 												 datums2[i]));
 		if (cmpval != 0)
@@ -2670,22 +2677,23 @@ partition_rbound_cmp(PartitionKey key,
  * is <, =, or > partition key of tuple (tuple_datums)
  */
 static int32
-partition_rbound_datum_cmp(PartitionKey key,
-						   Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
+partition_rbound_datum_cmp(int partnatts, FmgrInfo *partsupfunc,
+						   Oid *partcollation, Datum *rb_datums,
+						   PartitionRangeDatumKind *rb_kind,
 						   Datum *tuple_datums)
 {
 	int			i;
 	int32		cmpval = -1;
 
-	for (i = 0; i < key->partnatts; i++)
+	for (i = 0; i < partnatts; i++)
 	{
 		if (rb_kind[i] == PARTITION_RANGE_DATUM_MINVALUE)
 			return -1;
 		else if (rb_kind[i] == PARTITION_RANGE_DATUM_MAXVALUE)
 			return 1;
 
-		cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[i],
-												 key->partcollation[i],
+		cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[i],
+												 partcollation[i],
 												 rb_datums[i],
 												 tuple_datums[i]));
 		if (cmpval != 0)
@@ -2702,17 +2710,18 @@ partition_rbound_datum_cmp(PartitionKey key,
  * specified in *probe.
  */
 static int32
-partition_bound_cmp(PartitionKey key, PartitionBoundInfo boundinfo,
-					int offset, void *probe, bool probe_is_bound)
+partition_bound_cmp(int partnatts, FmgrInfo *partsupfunc, Oid *partcollation,
+					PartitionBoundInfo boundinfo, int offset, void *probe,
+					bool probe_is_bound)
 {
 	Datum	   *bound_datums = boundinfo->datums[offset];
 	int32		cmpval = -1;
 
-	switch (key->strategy)
+	switch (boundinfo->strategy)
 	{
 		case PARTITION_STRATEGY_LIST:
-			cmpval = DatumGetInt32(FunctionCall2Coll(&key->partsupfunc[0],
-													 key->partcollation[0],
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
 													 bound_datums[0],
 													 *(Datum *) probe));
 			break;
@@ -2730,12 +2739,14 @@ partition_bound_cmp(PartitionKey key, PartitionBoundInfo boundinfo,
 					 */
 					bool		lower = boundinfo->indexes[offset] < 0;
 
-					cmpval = partition_rbound_cmp(key,
-												  bound_datums, kind, lower,
+					cmpval = partition_rbound_cmp(partnatts, partsupfunc,
+												  partcollation, bound_datums,
+												  kind, lower,
 												  (PartitionRangeBound *) probe);
 				}
 				else
-					cmpval = partition_rbound_datum_cmp(key,
+					cmpval = partition_rbound_datum_cmp(partnatts, partsupfunc,
+														partcollation,
 														bound_datums, kind,
 														(Datum *) probe);
 				break;
@@ -2743,7 +2754,7 @@ partition_bound_cmp(PartitionKey key, PartitionBoundInfo boundinfo,
 
 		default:
 			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) key->strategy);
+				 (int) boundinfo->strategy);
 	}
 
 	return cmpval;
@@ -2777,7 +2788,8 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
 		int32		cmpval;
 
 		mid = (lo + hi + 1) / 2;
-		cmpval = partition_bound_cmp(key, boundinfo, mid, probe,
+		cmpval = partition_bound_cmp(key->partnatts, key->partsupfunc,
+									 key->partcollation, boundinfo, mid, probe,
 									 probe_is_bound);
 		if (cmpval <= 0)
 		{
-- 
1.7.9.5

0003-WIP-Partition-wise-join-for-1-1-1-0-0-1-partition-ma.patchtext/x-patch; charset=US-ASCII; name=0003-WIP-Partition-wise-join-for-1-1-1-0-0-1-partition-ma.patchDownload
From 00144d214fdfc156c50921f7fe217ad701af6f8a Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Wed, 9 Aug 2017 12:30:34 +0530
Subject: [PATCH 3/4] WIP Partition-wise join for 1:1, 1:0, 0:1 partition
 matching.

Earlier version of partition-wise join implementation allowed
partition-wise join between two relations with exactly same partition
bounds. This commit allows partition-wise join to be applied under
following conditions

1. the partition bounds of joining relations are such that rows from
given partition on one side join can join with rows from maximum one
partition on the other side i.e. bounds of a given partition on one
side match/overlap with those of maximum one partition on the other
side. If the mapping happens to be m:n where m > 1 or n > 1, we have
to gang multiple partition relations together into a single relation.
This means that we have to add simple relations during join
processing, something which is not supported right now.  ALso, in such
a case, different pairs of joining relations can produce different
partition bounds for the same join relation, which again is not
supported right now.

2. For every partition on outer side that can contribute to the result
of an OUTER side, there exists at least one (taken along with item 1,
it means exactly one)  matching partition on the inner side. To
support partition-wise join when the inner matching partition doesn't
exist, we have to add a dummy base relation corresponding to the
non-existent inner partition. We don't have support add base relations
during join processing.

This commit is not complete yet.

known TODO:
1. This patch doesn't support default partitions.

Ashutosh Bapat.
---
 src/backend/catalog/partition.c       | 1269 +++++++++++++++++++++++++++++++++
 src/backend/optimizer/path/joinrels.c |   70 +-
 src/backend/optimizer/util/plancat.c  |    5 +
 src/backend/optimizer/util/relnode.c  |   41 +-
 src/include/catalog/partition.h       |    5 +
 src/include/nodes/relation.h          |    2 +
 6 files changed, 1365 insertions(+), 27 deletions(-)

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 87b3fbc..95a54fd 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -151,6 +151,38 @@ static int partition_bound_bsearch(PartitionKey key,
 						void *probe, bool probe_is_bound, bool *is_equal);
 static void get_partition_dispatch_recurse(Relation rel, Relation parent,
 							   List **pds, List **leaf_part_oids);
+static PartitionBoundInfo partition_range_bounds_merge(int partnatts,
+							 FmgrInfo *supfuncs, Oid *collations,
+							 PartitionBoundInfo boundinfo1, int nparts1,
+							 PartitionBoundInfo boundinfo2, int nparts2,
+							 JoinType jointype, List **parts1, List **parts2);
+static PartitionBoundInfo partition_list_bounds_merge(int partnatts,
+							FmgrInfo *partsupfunc, Oid *collations,
+							PartitionBoundInfo boundinfo1, int nparts1,
+							PartitionBoundInfo boundinfo2, int nparts2,
+							JoinType jointype, List **parts1, List **parts2);
+static void generate_matching_part_pairs(int *mergemap1, int npart1,
+							 int *mergemap2, int nparts2, JoinType jointype,
+							 int nparts, List **parts1, List **parts2);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index);
+static int map_and_merge_partitions(int *partmap1, int *mergemap1, int index1,
+						 int *partmap2, int *mergemap2, int index2,
+						 int *next_index);
+static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *collations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2);
+static int partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
+						   Oid *collations, PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, bool *overlap);
+static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes);
 
 /*
  * RelationBuildPartitionDesc
@@ -2906,3 +2938,1240 @@ get_proposed_default_constraint(List *new_part_constraints)
 
 	return list_make1(defPartConstraint);
 }
+
+/*
+ * Merge the given partition bounds.
+ *
+ * If given partition bounds can not be merged, return NULL.
+ *
+ * The function also returns two lists of partition indexes one for each of the
+ * joining relations. Both the lists contain the same number of elements. The
+ * partition indexes at the same positions in the list indicate partitions from
+ * each side to be joined and their position corresponds to the index of
+ * partition to which the results of the child-join belong in the partitioned
+ * join.
+ */
+extern PartitionBoundInfo
+partition_bounds_merge(int partnatts, FmgrInfo *partsupfunc, Oid *partcollation,
+					   PartitionBoundInfo boundinfo1, int nparts1,
+					   PartitionBoundInfo boundinfo2, int nparts2,
+					   JoinType jointype, List **parts1, List **parts2)
+{
+	PartitionBoundInfo merged_bounds;
+	char		strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (boundinfo1->strategy != boundinfo2->strategy)
+		return NULL;
+
+	/* 
+	 * TODO: We should remove this limitation in the final version of the
+	 * patch. See TODO in build_merged_partition_bounds().
+	 * Bail out if there are default partitions. */
+	if (partition_bound_has_default(boundinfo1) ||
+		partition_bound_has_default(boundinfo2))
+		return NULL;
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+	strategy = boundinfo1->strategy;
+	if (strategy == PARTITION_STRATEGY_LIST)
+		merged_bounds = partition_list_bounds_merge(partnatts, partsupfunc,
+													partcollation, boundinfo1,
+													nparts1, boundinfo2,
+													nparts2, jointype, parts1,
+													parts2);
+	else if (strategy == PARTITION_STRATEGY_RANGE)
+		merged_bounds = partition_range_bounds_merge(partnatts, partsupfunc,
+													 partcollation, boundinfo1,
+													 nparts1, boundinfo2,
+													 nparts2, jointype, parts1,
+													 parts2);
+	else
+		elog(ERROR, "unexpected partition strategy: %d", strategy);
+
+	Assert(merged_bounds || (*parts1 == NIL && *parts2 == NIL));
+	return merged_bounds;
+}
+
+/*
+ * partition_get_range_bounds
+ *
+ * Given the index of lower bound in datums array, return lower and upper
+ * bounds and the index of the partition with that lower bound.
+ */
+static int
+partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
+						   PartitionRangeBound *lower,
+						   PartitionRangeBound *upper)
+{
+	int			part_index;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The lower bound should correspond to a valid partition. */
+	part_index = bi->indexes[lb_index + 1];
+	Assert(part_index >= 0);
+
+	lower->kind = bi->kind[lb_index];
+	lower->datums = bi->datums[lb_index];
+	lower->lower = true;
+	upper->kind = bi->kind[lb_index + 1];
+	upper->datums = bi->datums[lb_index + 1];
+	upper->lower = false;
+
+	return part_index;
+}
+
+/*
+ * partition_range_get_next_lb_index
+ *
+ * Given the index of lower bound in datums array return the
+ * index of lower bound of the next partition. When the given index corresponds
+ * to the last partition, return number of datums (ndatums).
+ */
+static int
+partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
+{
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The partition index corresponding to the upper bound should be valid. */
+	Assert(bi->indexes[lb_index + 1] >= 0);
+
+	/*
+	 * If there are no bounds left beyond the upper bound, we have reached the
+	 * last partition.
+	 */
+	if (lb_index + 2 < bi->ndatums)
+	{
+		/*
+		 * If the bound next to the upper bound corresponds to no partition,
+		 * that's the next lower bound of the next partition. Otherwise, the
+		 * current upper bound is the lower bound of the next partition.
+		 */
+		if (bi->indexes[lb_index + 2] < 0)
+			return lb_index + 2;
+		else
+			return lb_index + 1;
+	}
+	else
+		return bi->ndatums;
+}
+
+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc, Oid *collations,
+						  PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2)
+{
+	return partition_rbound_cmp(partnatts, partsupfunc, collations,
+								bound1->datums, bound1->kind, bound1->lower,
+								bound2);
+}
+
+/*
+ * partition_range_cmp
+ *
+ * Compare the bounds of two range partitions and return <, = or > 0, if the
+ * first partition's upper bound is lower than, equal to or higher than the
+ * second partition's upper bound resp.
+ *
+ * Also, set overlaps to true, if the ranges overlap, otherwise set it to
+ * false.
+ */
+static int
+partition_range_cmp(int partnatts, FmgrInfo *supfuncs, Oid *collations,
+						   PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, bool *overlap)
+{
+	/*
+	 * Compare upper bound of the first partition with the lower bound of the
+	 * second and vice-versa. If lower bound is higher than the upper bound,
+	 * the partitions are not overlapping. All other cases indicate overlapping
+	 * partitions.
+	 * TODO: Add a testcase which has lower and upper bound matching exactly.
+	 * Lower bound is inclusive and upper bound is exclusive, so even if the
+	 * datums match, the bounds do not match exactly.
+	 */
+	if (partition_range_bound_cmp(partnatts, supfuncs, collations,
+								  lower_bound1, upper_bound2) > 0)
+	{
+		*overlap = false;
+		return 1;
+	}
+	else if (partition_range_bound_cmp(partnatts, supfuncs, collations,
+									   lower_bound2, upper_bound1) > 0)
+	{
+		*overlap = false;
+		return -1;
+	}
+	else
+	{
+		*overlap = true;
+		return partition_range_bound_cmp(partnatts, supfuncs, collations,
+										 upper_bound1, upper_bound2);
+	}
+}
+
+/*
+ * partition_range_merge
+ *
+ * Merge the partition bounds of given two partitions such that the join
+ * between the given two partitions fits merged bounds.
+ *
+ * "merged_upper" will be set to one of the given upper bounds and
+ * "merged_lower" will be set to one of the given lower bounds.
+ */
+static void
+partition_range_merge(int partnatts, FmgrInfo *supfuncs,
+							 Oid *collations, JoinType jointype,
+							 PartitionRangeBound *left_lb,
+							 PartitionRangeBound *left_ub,
+							 PartitionRangeBound *right_lb,
+							 PartitionRangeBound *right_ub,
+							 PartitionRangeBound **merged_lb,
+							 PartitionRangeBound **merged_ub)
+{
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = left_ub;
+			*merged_lb = left_lb;
+			break;
+
+		case JOIN_RIGHT:
+			*merged_ub = right_ub;
+			*merged_lb = right_lb;
+			break;
+
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			if (partition_range_bound_cmp(partnatts, supfuncs, collations,
+										  left_ub, right_ub) < 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, supfuncs, collations,
+										  left_lb, right_lb) > 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		case JOIN_FULL:
+			if (partition_range_bound_cmp(partnatts, supfuncs, collations,
+										  left_ub, right_ub) > 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, supfuncs, collations,
+										  left_lb, right_lb) < 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		default:
+			elog(ERROR, "Unexpected join type %d", jointype);
+	}
+
+	return;
+}
+
+/*
+ * Add the lower bound of the next range to the list of bounds, if the lower
+ * bound is higher or equal to the previous upper bound. If successful return
+ * true, otherwise false.
+ */
+static bool
+partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		prev_ub.datums = llast(*merged_datums);
+		prev_ub.kind = llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		/*
+		 * TODO: explain why do we pass lower to be false for the next lower
+		 * bound.
+		 */
+		cmpval = partition_rbound_cmp(partnatts, supfuncs, collations,
+									  next_lb_datums, next_lb_kind, false,
+									  &prev_ub);
+	}
+
+	/*
+	 * The lower bound is lower than the last upper bound, thus does not fit
+	 * the bounds created so far and hence can not be merged with the existing
+	 * bounds.
+	 */
+	if (cmpval < 0)
+		return false;
+
+	/*
+	 * Add bounds of the new merged partition. If the next lower bound is
+	 * higher than the last upper bound, add new range with index
+	 * corresponding to the lower bound as -1. If the merged lower bound
+	 * is same as the last merged upper bound, the last upper bound will be
+	 * reused as the lower bound of the next range.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, next_lb_datums);
+		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	return true;
+}
+
+/*
+ * Merge given two range partition bounds.
+ *
+ * Work horse function for partition_bounds_merge() for range partitioned
+ * tables.
+ *
+ * TODO: for an anti-join, the caller is supposed to send the outer relation as
+ * left relation. May be we should rename left and right as inner and outer. We
+ * don't need to handle RIGHT joins in this function, so renaming them as outer
+ * and inner is fine.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(int partnatts, FmgrInfo *supfuncs, Oid *collations,
+							 PartitionBoundInfo left_bi, int left_nparts,
+							 PartitionBoundInfo right_bi, int right_nparts,
+							 JoinType jointype, List **left_parts, List **right_parts)
+{
+	int		   *left_pmap;
+	int		   *left_mmap;
+	int		   *right_pmap;
+	int		   *right_mmap;
+	int			cnt1;
+	int			cnt2;
+	int			left_part;
+	int			right_part;
+	PartitionBoundInfo merged_bounds = NULL;
+	bool		merged = true;	/* By default we ranges are merge-able. */
+	int			left_lb_index;
+	int			right_lb_index;
+	int			next_index;
+	int			cmpval;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	List	   *merged_kinds = NIL;
+
+	Assert(left_bi->strategy == right_bi->strategy &&
+		   left_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	*left_parts = NIL;
+	*right_parts = NIL;
+
+	/* Allocate and initialize partition maps. */
+	left_pmap = (int *) palloc(sizeof(int) * left_nparts);
+	left_mmap = (int *) palloc(sizeof(int) * left_nparts);
+	right_pmap = (int *) palloc(sizeof(int) * right_nparts);
+	right_mmap = (int *) palloc(sizeof(int) * right_nparts);
+
+	for (cnt1 = 0; cnt1 < left_nparts; cnt1++)
+	{
+		left_pmap[cnt1] = -1;
+		left_mmap[cnt1] = -1;
+	}
+	for (cnt2 = 0; cnt2 < right_nparts; cnt2++)
+	{
+		right_pmap[cnt2] = -1;
+		right_mmap[cnt2] = -1;
+	}
+
+	left_lb_index = 0;
+	right_lb_index = 0;
+	next_index = 0;
+	while (left_lb_index < left_bi->ndatums &&
+		   right_lb_index < right_bi->ndatums)
+	{
+		PartitionRangeBound left_lb;
+		PartitionRangeBound left_ub;
+		PartitionRangeBound right_lb;
+		PartitionRangeBound right_ub;
+		PartitionRangeBound *merged_lb = NULL;
+		PartitionRangeBound *merged_ub = NULL;
+		int			merged_index = -1;
+		bool		overlap;
+
+		/* Get the range bounds of the next partition. */
+		left_part = partition_get_range_bounds(left_bi, left_lb_index,
+											   &left_lb, &left_ub);
+		right_part = partition_get_range_bounds(right_bi, right_lb_index,
+												&right_lb, &right_ub);
+
+		cmpval = partition_range_cmp(partnatts, supfuncs, collations,
+									 &left_lb, &left_ub, &right_lb, &right_ub,
+									 &overlap);
+
+		if (overlap)
+		{
+			/* Overlapping ranges, try merging. */
+			partition_range_merge(partnatts, supfuncs, collations, jointype,
+								  &left_lb, &left_ub, &right_lb, &right_ub,
+								  &merged_lb, &merged_ub);
+			merged_index = map_and_merge_partitions(left_pmap, left_mmap,
+													left_part, right_pmap,
+													right_mmap, right_part,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				merged = false;
+				break;
+			}
+		}
+
+		if (cmpval == 0)
+		{
+			Assert(overlap);
+
+			/* Move to the next pair of partitions. */
+			left_lb_index = partition_range_get_next_lb_index(left_bi,
+															  left_lb_index);
+			right_lb_index = partition_range_get_next_lb_index(right_bi,
+															   right_lb_index);
+		}
+		else if (cmpval < 0)
+		{
+			/*
+			 * If the partition on the left was not mapped to any partition on
+			 * the right. Any row from that partition will not have a matching
+			 * row from the other relation. So the partition will be part of
+			 * the join if it's an anti-join or the left side is the outer
+			 * side.
+			 */
+			if (jointype == JOIN_INNER || jointype == JOIN_SEMI ||
+				jointype == JOIN_RIGHT)
+			{
+				/* Nothing to do. */
+			}
+			else if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+					 jointype == JOIN_ANTI)
+			{
+				if (left_mmap[left_part] < 0)
+				{
+					left_mmap[left_part] = next_index++;
+					merged_index = left_mmap[left_part];
+					merged_lb = &left_lb;
+					merged_ub = &left_ub;
+				}
+			}
+			else
+			{
+				/* Don't know what to do with other join types. Bail out. */
+				merged = false;
+			}
+
+			/* Move to the next partition on the left side. */
+			left_lb_index = partition_range_get_next_lb_index(left_bi,
+															  left_lb_index);
+		}
+		else
+		{
+			Assert(cmpval > 0);
+
+			/*
+			 * If the partition on the right was not mapped to any partition on
+			 * the left. Any row from that partition will not have a matching
+			 * row from the other relation. So the partition will be part of
+			 * the join if the right side is the outer side.
+			 */
+			if (jointype == JOIN_INNER || jointype == JOIN_SEMI ||
+				jointype == JOIN_LEFT || jointype == JOIN_ANTI)
+			{
+				/* Nothing to do. */
+			}
+			else if (jointype == JOIN_RIGHT || jointype == JOIN_FULL)
+			{
+				if (right_mmap[right_part] < 0)
+				{
+					right_mmap[right_part] = next_index++;
+					merged_index = right_mmap[right_part];
+					merged_lb = &right_lb;
+					merged_ub = &right_ub;
+				}
+			}
+			else
+			{
+				/* Don't know what to do with other join types. Bail out. */
+				merged = false;
+			}
+
+			/* Move to the next partition on the right side. */
+			right_lb_index = partition_range_get_next_lb_index(right_bi,
+															   right_lb_index);
+		}
+
+		if (!merged)
+			break;
+
+		/* A skipped partition is not added to merged bounds. */
+		if (merged_index < 0)
+			continue;
+
+		/*
+		 * We have a valid partition index for the next partition of join. The
+		 * partition should have valid range.
+		 */
+		Assert(merged_lb && merged_ub);
+
+		/* Try merging merged lower bound with the last upper bound. */
+		merged = partition_range_merge_next_lb(partnatts, supfuncs,
+											   collations, merged_lb->datums,
+											   merged_lb->kind, &merged_datums,
+											   &merged_kinds, &merged_indexes);
+		if (merged)
+		{
+			/* Add upper bound with the merged partition index. */
+			merged_datums = lappend(merged_datums, merged_ub->datums);
+			merged_kinds = lappend(merged_kinds, merged_ub->kind);
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+		}
+		else
+			break;
+	}
+
+	/*
+	 * We will run the above loop till we exhaust ranges of at least one side
+	 * unless we failed to merge the ranges.
+	 */
+	Assert (!merged || (left_lb_index >= left_bi->ndatums ||
+						right_lb_index >= right_bi->ndatums));
+
+	/*
+	 * Handle any remaining partition bounds.  If remaining partitions fall on
+	 * the inner side of the join, none of the rows in those partition are
+	 * going to be joined with any row on the outer side and hence those
+	 * partitions will not be part of the join result. Hence only consider the
+	 * remaining partitions on the outer side of the join.
+	 */
+	if (merged &&
+		((left_lb_index < left_bi->ndatums &&
+		  (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+		   jointype == JOIN_ANTI)) ||
+		 (right_lb_index < right_bi->ndatums &&
+		  (jointype == JOIN_RIGHT || jointype == JOIN_FULL))))
+	{
+		int			bound_index = -1;
+		PartitionBoundInfo rem_bi = NULL;
+		int		   *mmap = NULL;
+		int			part_index;
+		PartitionRangeBound rem_lb;
+		PartitionRangeBound rem_ub;
+
+		if (left_lb_index < left_bi->ndatums)
+		{
+			Assert(jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+				   jointype == JOIN_ANTI);
+			bound_index = left_lb_index;
+			rem_bi = left_bi;
+			mmap = left_mmap;
+		}
+		else if (right_lb_index < right_bi->ndatums)
+		{
+			Assert(jointype == JOIN_RIGHT || jointype == JOIN_FULL);
+			bound_index = right_lb_index;
+			rem_bi = right_bi;
+			mmap = right_mmap;
+		}
+		Assert((bound_index >= 0 && bound_index < rem_bi->ndatums) &&
+			   rem_bi && mmap);
+
+		/*
+		 * If the partition corresponding to this lower bound has been already
+		 * mapped to a merged partition, don't need to add it again. This may
+		 * happen if the range of the last partition on the inner side overlaps
+		 * with this partition's range and has upper bound lesser than upper
+		 * bound of this partition's range.
+		 */
+		part_index = partition_get_range_bounds(rem_bi, bound_index, &rem_lb,
+												&rem_ub);
+		Assert(part_index >= 0);
+		if (mmap[part_index] >= 0)
+			bound_index = partition_range_get_next_lb_index(rem_bi, bound_index);
+
+		/*
+		 * Merge lower bound of the next range with the upper bound of last
+		 * range.
+		 */
+		if (bound_index < rem_bi->ndatums)
+			merged = partition_range_merge_next_lb(partnatts, supfuncs,
+												   collations,
+												   rem_bi->datums[bound_index],
+												   rem_bi->kind[bound_index],
+												   &merged_datums,
+												   &merged_kinds,
+												   &merged_indexes);
+
+		/*
+		 * Rest of the bounds correspond to valid ranges so add them after
+		 * remapping their partitions as required.
+		 */
+		for (bound_index++; merged && bound_index < rem_bi->ndatums;
+			 bound_index++)
+		{
+			Datum	   *datums = rem_bi->datums[bound_index];
+			int			index = rem_bi->indexes[bound_index];
+			int			part_index;
+
+			/*
+			 * Add lower bounds with partition index -1 and assign a new
+			 * partition index to the upper bounds.
+			 */
+			if (index < 0)
+				part_index = index;
+			else
+			{
+				if (mmap[index] < 0)
+					mmap[index] = next_index++;
+				part_index = mmap[index];
+			}
+
+			merged_indexes = lappend_int(merged_indexes, part_index);
+			merged_datums = lappend(merged_datums, datums);
+			merged_kinds = lappend(merged_kinds,
+								   rem_bi->kind[bound_index]);
+		}
+	}
+
+	/* Create PartitionBoundInfo */
+	if (merged)
+	{
+		/* Use maps to match partition from the joining relations. */
+		generate_matching_part_pairs(left_mmap, left_nparts, right_mmap,
+									 right_nparts, jointype, next_index,
+									 left_parts, right_parts);
+
+		/* Craft a PartitionBoundInfo to return. */
+		if (*left_parts && *right_parts)
+		{
+			Assert(list_length(*left_parts) == list_length(*right_parts));
+			Assert(list_length(*left_parts) == next_index);
+			merged_bounds = build_merged_partition_bounds(left_bi->strategy,
+														  merged_datums,
+														  merged_indexes,
+														  merged_kinds,
+														  -1);
+		}
+	}
+
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	pfree(left_pmap);
+	pfree(right_pmap);
+	pfree(left_mmap);
+	pfree(right_mmap);
+
+	/* Free any memory we used in this function. */
+	return merged_bounds;
+}
+
+/*
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ * The function builds the maps of matching partitions from either relation. It
+ * builds the list of partition key values that may appear in the join result
+ * alongwith the list of indexes of partitions of join to which those values
+ * belong. It then crafts a PartitionBoundInfo structure representing the
+ * partition bounds of the join result.
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(int partnatts, FmgrInfo *partsupfunc,
+							Oid *partcollation, PartitionBoundInfo left_bi,
+							int left_nparts, PartitionBoundInfo right_bi,
+							int right_nparts, JoinType jointype, List **left_parts,
+							List **right_parts)
+{
+	int		   *left_pmap;	/* left to right partition map */
+	int		   *left_mmap;	/* left to merged partition map */
+	int		   *right_pmap;	/* right to left partition map */
+	int		   *right_mmap;	/* right to merged partition map */
+	int			cntl;
+	int			cntr;
+	bool		merged = true;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			next_index = 0;
+	int			null_index;
+	PartitionBoundInfo merged_bounds = NULL;
+	int		   *left_indexes = left_bi->indexes;
+	int		   *right_indexes = right_bi->indexes;
+	int			left_ni = left_bi->null_index;
+	int			right_ni = right_bi->null_index;
+
+	Assert(left_bi->strategy == right_bi->strategy &&
+		   left_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!left_bi->kind && !right_bi->kind);
+
+	/* Allocate and initialize partition maps. */
+	left_pmap = (int *) palloc(sizeof(int) * left_nparts);
+	left_mmap = (int *) palloc(sizeof(int) * left_nparts);
+	right_pmap = (int *) palloc(sizeof(int) * right_nparts);
+	right_mmap = (int *) palloc(sizeof(int) * right_nparts);
+
+	/* Initialize partition maps. */
+	for (cntl = 0; cntl < left_nparts; cntl++)
+	{
+		left_pmap[cntl] = -1;
+		left_mmap[cntl] = -1;
+	}
+	for (cntr = 0; cntr < right_nparts; cntr++)
+	{
+		right_pmap[cntr] = -1;
+		right_mmap[cntr] = -1;
+	}
+
+	cntl = cntr = 0;
+	while (cntl < left_bi->ndatums && cntr < right_bi->ndatums)
+	{
+		Datum	   *ldatums = left_bi->datums[cntl];
+		Datum	   *rdatums = right_bi->datums[cntr];
+		int			l_index = left_indexes[cntl];
+		int			r_index = right_indexes[cntr];
+		int			cmpval;
+		int			merged_index;
+		Datum	   *merged_datum;
+
+		/* Every list datum should map to a valid partition index. */
+		Assert(l_index >= 0 && r_index >= 0);
+
+		cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+												 partcollation[0], ldatums[0],
+												 rdatums[0]));
+		if (cmpval == 0)
+		{
+			/*
+			 * Try matching partitions containing the matching datums. If
+			 * successful, add the datum to the merged bounds with index of
+			 * merged partition containing it.
+			 */
+			merged_datum = ldatums;
+			merged_index = map_and_merge_partitions(left_pmap, left_mmap, l_index,
+													right_pmap, right_mmap, r_index,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				merged = false;
+				break;
+			}
+
+			/* Move to the next pair of bounds. */
+			cntl++;
+			cntr++;
+		}
+		else if (cmpval < 0)
+		{
+			/*
+			 * This list datum is present in the left side but not the right
+			 * side. So it will appear in the join when the left side is outer
+			 * side.
+			 */
+			if (jointype == JOIN_INNER || jointype == JOIN_RIGHT ||
+				jointype == JOIN_SEMI)
+				merged_index = -1;
+			else if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+					 jointype == JOIN_ANTI)
+			{
+				if (left_mmap[l_index] < 0)
+					left_mmap[l_index] = next_index++;
+				merged_index = left_mmap[l_index];
+				merged_datum = ldatums;
+			}
+			else
+			{
+				/* Don't know what to do with other join types. */
+				merged = false;
+				break;
+			}
+
+			/* Move to the next datum on the left side. */
+			cntl++;
+		}
+		else
+		{
+			Assert(cmpval > 0);
+
+			/*
+			 * This list datum is present in the right side but not the left
+			 * side. So it will appear in the join when the right side is outer
+			 * side.
+			 */
+			if (jointype == JOIN_INNER || jointype == JOIN_LEFT ||
+				jointype == JOIN_SEMI || jointype == JOIN_ANTI)
+				merged_index = -1;
+			else if (jointype == JOIN_RIGHT || jointype == JOIN_FULL)
+			{
+				/*
+				 * Every list value on the outer side will appear in the
+				 * join.  Find the merged partition to which this value
+				 * belongs.
+				 */
+				if (right_mmap[r_index] < 0)
+					right_mmap[r_index] = next_index++;
+				merged_index = right_mmap[r_index];
+				merged_datum = rdatums;
+			}
+			else
+			{
+				/* Don't know what to do with other join types. */
+				merged = false;
+				break;
+			}
+
+			/* Move to the next datum on the right side. */
+			cntr++;
+		}
+
+		/*
+		 * Add the datum with appropriate index in the list of datums, if the
+		 * rows containing that datum are deemed to be part of the join.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	/*
+	 * If merge is unsuccessful, bail out without any further processing.
+	 * That leaks the memory allocated in this function. So, try not to leak
+	 * memory.
+	 */
+	if (!merged)
+		goto merge_failed;
+
+	/* We should have exhausted datums on at least one side. */
+	Assert(cntr >= right_bi->ndatums || cntl >= left_bi->ndatums);
+
+	/*
+	 * Add any remaining list values on the outer side, assigning partition
+	 * indexes if required.
+	 */
+	if (jointype == JOIN_LEFT || jointype == JOIN_FULL || jointype == JOIN_ANTI)
+	{
+		for (;cntl < left_bi->ndatums; cntl++)
+		{
+			Datum	   *ldatums = left_bi->datums[cntl];
+			int			l_index = left_indexes[cntl];
+
+			if (left_mmap[l_index] < 0)
+				left_mmap[l_index] = next_index++;
+			merged_indexes = lappend_int(merged_indexes, left_mmap[l_index]);
+			merged_datums = lappend(merged_datums, ldatums);
+		}
+	}
+
+	if (jointype == JOIN_RIGHT || jointype == JOIN_FULL)
+	{
+		for (;cntr < right_bi->ndatums; cntr++)
+		{
+			Datum	   *rdatums = right_bi->datums[cntr];
+			int			r_index = right_indexes[cntr];
+
+			if (right_mmap[r_index] < 0)
+				right_mmap[r_index] = next_index++;
+			merged_indexes = lappend_int(merged_indexes, right_mmap[r_index]);
+			merged_datums = lappend(merged_datums, rdatums);
+		}
+	}
+
+	/*
+	 * Merge NULL partitions if any. Find the index of merged partition to
+	 * which the NULL values belong in the join result. We can eliminate a NULL
+	 * partition when it appears only in the inner relation.
+	 */
+	if (!partition_bound_accepts_nulls(left_bi) &&
+		!partition_bound_accepts_nulls(right_bi))
+		null_index = -1;
+	else if (partition_bound_accepts_nulls(left_bi) &&
+			 !partition_bound_accepts_nulls(right_bi))
+	{
+		if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+			jointype == JOIN_ANTI)
+		{
+			if (left_mmap[left_ni] < 0)
+				left_mmap[left_ni] = next_index++;
+			null_index = left_mmap[left_ni];
+		}
+		else
+			null_index = -1;
+	}
+	else if (!partition_bound_accepts_nulls(left_bi) &&
+			 partition_bound_accepts_nulls(right_bi))
+	{
+		if (jointype == JOIN_RIGHT || jointype == JOIN_FULL)
+		{
+			if (right_mmap[right_ni] < 0)
+				right_mmap[right_ni] = next_index++;
+			null_index = right_mmap[right_ni];
+		}
+		else
+			null_index = -1;
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		null_index = map_and_merge_partitions(left_pmap, left_mmap,
+											  left_ni, right_pmap,
+											  right_mmap, right_ni,
+											  &next_index);
+		if (null_index < 0)
+			merged = false;
+	}
+
+	/* If successful build the output structures. */
+	if (merged)
+	{
+
+		/* Use maps to match partition from the joining relations. */
+		generate_matching_part_pairs(left_mmap, left_nparts, right_mmap,
+									 right_nparts, jointype, next_index,
+									 left_parts, right_parts);
+		/* Craft a PartitionBoundInfo to return. */
+		if (*left_parts && *right_parts)
+		{
+			Assert(list_length(*left_parts) == list_length(*right_parts));
+			Assert(list_length(*left_parts) == next_index);
+			merged_bounds = build_merged_partition_bounds(left_bi->strategy,
+														  merged_datums,
+														  merged_indexes, NIL,
+														  null_index);
+		}
+	}
+
+merge_failed:
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	pfree(left_pmap);
+	pfree(right_pmap);
+	pfree(left_mmap);
+	pfree(right_mmap);
+
+	return merged_bounds;
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * If the two given partitions (given by index1 and index2 resp.) are already
+ * mapped to each other return the index of corresponding partition in the
+ * merged set of partitions.  If they do not have a merged partition associated
+ * with them, assign a new merged partition index.  If the partitions are
+ * already mapped and their mapped partitions are different from each other,
+ * they can not be merged, so return -1.
+ *
+ * partmap1[i] gives the partition of relation 2 which matches ith partition of
+ * relation 1. Similarly for partmap2.
+ *
+ * mergemap1[i] gives the partition in the merged set to which ith partition of
+ * relation 1 maps to. Similarly for mergemap2.
+ *
+ * index1 and index2 are the indexes of matching partition from respective
+ * relations.
+ *
+ * *next_index is used and incremented when the given partitions require a new
+ * merged partition.
+ */
+static int
+map_and_merge_partitions(int *partmap1, int *mergemap1, int index1,
+						 int *partmap2, int *mergemap2, int index2,
+						 int *next_index)
+{
+	int			merged_index;
+
+	/*
+	 * If both the partitions are not mapped to each other, update the
+	 * maps.
+	 */
+	if (partmap1[index1] < 0 && partmap2[index2] < 0)
+	{
+		partmap1[index1] = index2;
+		partmap2[index2] = index1;
+	}
+
+	/*
+	 * If the given to partitions map to each other, find the corresponding
+	 * merged partition index .
+	 */
+	if (partmap1[index1] == index2 && partmap2[index2] == index1)
+	{
+		/*
+		 * If both the partitions are mapped to the same merged partition, get
+		 * the index of merged partition.
+		 */
+		if (mergemap1[index1] == mergemap2[index2])
+		{
+			merged_index = mergemap1[index1];
+
+			/*
+			 * If the given two partitions do not have a merged partition
+			 * associated with them, allocate a new merged partition.
+			 */
+			if (merged_index < 0)
+			{
+				merged_index = *next_index;
+				*next_index = *next_index + 1;
+				mergemap1[index1] = merged_index;
+				mergemap2[index2] = merged_index;
+			}
+		}
+
+		/*
+		 * If partition from one relation was mapped to a merged partition but
+		 * not the partition from the other relation, map the same merged
+		 * partition to the partition from other relation, since matching
+		 * partitions map to the same merged partition.
+		 */
+		else if (mergemap1[index1] >= 0 && mergemap2[index2] < 0)
+		{
+			mergemap2[index2] = mergemap1[index1];
+			merged_index = mergemap1[index1];
+		}
+		else if (mergemap1[index1] < 0 && mergemap2[index2] >= 0)
+		{
+			mergemap1[index1] = mergemap2[index2];
+			merged_index = mergemap2[index2];
+		}
+		else
+		{
+			Assert(mergemap1[index1] != mergemap2[index2] &&
+				   mergemap1[index1] >= 0 && mergemap2[index2] >= 0);
+
+			/*
+			 * Both the partitions map to different merged partitions. This
+			 * means that multiple partitions from one relation matches to one
+			 * partition from the other relation. Partition-wise join does not
+			 * handle this case right now, since it requires ganging multiple
+			 * partitions together (into one RelOptInfo).
+			 */
+			merged_index = -1;
+		}
+	}
+	else
+	{
+		/*
+		 * Multiple partitions from one relation map to one partition from the
+		 * other relation. Partition-wise join does not handle this case right
+		 * now, since it requires ganging multiple partitions together (into
+		 * one RelOptInfo).
+		 */
+		merged_index = -1;
+	}
+
+	return merged_index;
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * Given the merged partition to which partition on either side of join map,
+ * produce the list pairs of partitions which when joined produce the merged
+ * partitions in the order of merged partition indexes.
+ *
+ * If successful, the pairs of partitions are returned as two separate lists
+ * one for each side. Otherwise, those lists will be set to NIL.
+ *
+ * TODO: rename the sides as outer and inner. You may not need to support
+ * JOIN_RIGHT, since we won't see that type here.
+ */
+static void
+generate_matching_part_pairs(int *mergemap1, int nparts1, int *mergemap2,
+							 int nparts2, JoinType jointype, int nparts,
+							 List **parts1, List **parts2)
+{
+	bool		merged = true;
+	int		  **matching_parts;
+	int			cnt1;
+	int			cnt2;
+
+	matching_parts = (int **) palloc(sizeof(int *) * 2);
+	matching_parts[0] = (int *) palloc(sizeof(int) * nparts);
+	matching_parts[1] = (int *) palloc(sizeof(int) * nparts);
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt1 = 0; cnt1 < nparts; cnt1++)
+	{
+		matching_parts[0][cnt1] = -1;
+		matching_parts[1][cnt1] = -1;
+	}
+
+	/* Set pairs of matching partitions. */
+	for (cnt1 = 0; cnt1 < nparts1; cnt1++)
+	{
+		if (mergemap1[cnt1] >= 0)
+		{
+			Assert(mergemap1[cnt1] < nparts);
+			matching_parts[0][mergemap1[cnt1]] = cnt1;
+		}
+	}
+	for (cnt2 = 0; cnt2 < nparts2; cnt2++)
+	{
+		if (mergemap2[cnt2] >= 0)
+		{
+			Assert(mergemap2[cnt2] < nparts);
+			matching_parts[1][mergemap2[cnt2]] = cnt2;
+		}
+	}
+
+	/*
+	 * If we have a partition missing on an inner side, we need to add a dummy
+	 * relation which joins with the outer partition. If the inner relation
+	 * happens to be a base relation, it will require adding a dummy child
+	 * base relation during join processing. Right now, we freeze the base
+	 * relation arrays like PlannerInfo::simple_rte_array after planning for
+	 * base relations. Adding a new (dummy) base relation would require some
+	 * changes to that. So, right now, we do not implement partition-wise join
+	 * in such cases.
+	 */
+	for (cnt1 = 0; cnt1 < nparts; cnt1++)
+	{
+		int			part1 = matching_parts[0][cnt1];
+		int			part2 = matching_parts[1][cnt1];
+
+		/* At least one of the partitions should exist. */
+		Assert(part1 >= 0 || part2 >= 0);
+
+		switch (jointype)
+		{
+			case JOIN_INNER:
+			case JOIN_SEMI:
+
+				/*
+				 * An inner or semi join can not return any row when the
+				 * matching partition on either side is missing. We should
+				 * have eliminated all such cases while merging the bounds.
+				 */
+				Assert(part1 >= 0 && part2 >= 0);
+				break;
+
+			case JOIN_LEFT:
+			case JOIN_ANTI:
+				Assert(part1 >= 0);
+				if (part2 < 0)
+					merged = false;
+				break;
+
+			case JOIN_RIGHT:
+				Assert(part2 >= 0);
+				if (part1 < 0)
+					merged = false;
+				break;
+
+			case JOIN_FULL:
+				if (part1 < 0 || part2 < 0)
+					merged = false;
+				break;
+
+			default:
+				/* We do not know what to do in this case. Bail out. */
+				merged = false;
+		}
+
+		if (!merged)
+			break;
+
+		*parts1 = lappend_int(*parts1, part1);
+		*parts2 = lappend_int(*parts2, part2);
+	}
+
+	pfree(matching_parts[0]);
+	pfree(matching_parts[1]);
+	pfree(matching_parts);
+
+	if (!merged)
+	{
+		list_free(*parts1);
+		list_free(*parts2);
+		*parts1 = NIL;
+		*parts2 = NIL;
+	}
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind = (PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+															   list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+
+	/* TODO: For the time being we do not work with default partitions. */
+	merged_bounds->default_index = -1;
+
+	return merged_bounds;
+}
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 1578dea..91b976b 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1309,8 +1309,13 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 						RelOptInfo *joinrel, SpecialJoinInfo *parent_sjinfo,
 						List *parent_restrictlist)
 {
-	int			nparts;
 	int			cnt_parts;
+	PartitionScheme part_scheme;
+	PartitionBoundInfo join_boundinfo;
+	List	   *parts1;
+	List	   *parts2;
+	ListCell   *lc1;
+	ListCell   *lc2;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
@@ -1349,32 +1354,49 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	 */
 	Assert(joinrel->part_scheme == rel1->part_scheme &&
 		   joinrel->part_scheme == rel2->part_scheme);
+	part_scheme = joinrel->part_scheme;
 
 	/*
-	 * Since we allow partition-wise join only when the partition bounds of
-	 * the joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * Get the list of matching partitions from both sides of the join. While
+	 * doing so, we also build the partition bounds of the join relation,
+	 * which should match the bounds calculated for other pairs. TODO: why
+	 * should every pair result in the same partition bounds?
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	join_boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											part_scheme->partsupfunc,
+											part_scheme->parttypcoll,
+											rel1->boundinfo, rel1->nparts,
+											rel2->boundinfo, rel2->nparts,
+											parent_sjinfo->jointype,
+											&parts1, &parts2);
+
+	Assert(join_boundinfo);
+	Assert(partition_bounds_equal(part_scheme->partnatts,
+								  part_scheme->parttyplen,
+								  part_scheme->parttypbyval, join_boundinfo,
+								  joinrel->boundinfo));
+	elog(DEBUG3, "join between relations %s and %s is considered for partition-wise join.",
+		 bmsToString(rel1->relids), bmsToString(rel2->relids));
 
-	nparts = joinrel->nparts;
+	/*
+	 * Every pair of joining relations should result in the same number of
+	 * child-joins.
+	 */
+	Assert(joinrel->nparts == list_length(parts1));
+	Assert(joinrel->nparts == list_length(parts2));
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	cnt_parts = 0;
+	forboth(lc1, parts1, lc2, parts2)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
+		int			part1 = lfirst_int(lc1);
+		int			part2 = lfirst_int(lc2);
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1382,6 +1404,10 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		Assert(part1 >= 0 && part2 >= 0);
+		child_rel1 = rel1->part_rels[part1];
+		child_rel2 = rel2->part_rels[part2];
+
 		/* We should never try to join two overlapping sets of rels. */
 		Assert(!bms_overlap(child_rel1->relids, child_rel2->relids));
 		child_joinrelids = bms_union(child_rel1->relids, child_rel2->relids);
@@ -1415,12 +1441,24 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 			joinrel->part_rels[cnt_parts] = child_joinrel;
 		}
 
+		/*
+		 * For every pair of joining relations, the set of matching partitions
+		 * would change. However, the base relation partitions constituting
+		 * the given child should remain same for all the joining pairs. Since
+		 * the order in which children are stored in the array of child-joins,
+		 * depends upon partition bounds of the join, which are same for all
+		 * the joining pairs, every joining pair yields the child-joins in the
+		 * same order.
+		 */
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
 
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
+		cnt_parts++;
 	}
+
+	Assert(cnt_parts == joinrel->nparts);
 }
 
 /*
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 9d35a41..9d3df33 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -1918,6 +1918,11 @@ find_partition_scheme(PlannerInfo *root, Relation relation)
 	memcpy(part_scheme->parttypbyval, partkey->parttypbyval,
 		   sizeof(bool) * partnatts);
 
+	part_scheme->partsupfunc =
+		(FmgrInfo *) palloc(sizeof(FmgrInfo) * partnatts);
+	memcpy(part_scheme->partsupfunc, partkey->partsupfunc,
+		   sizeof(FmgrInfo) * partnatts);
+
 	/* Add the partitioning scheme to PlannerInfo. */
 	root->part_schemes = lappend(root->part_schemes, part_scheme);
 
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 21fd29f..827ef76 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1601,6 +1601,9 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	int			partnatts;
 	int			cnt;
 	PartitionScheme part_scheme;
+	PartitionBoundInfo join_boundinfo;
+	List	   *parts1;
+	List	   *parts2;
 
 	/* Nothing to do if partition-wise join technique is disabled. */
 	if (!enable_partition_wise_join)
@@ -1634,17 +1637,26 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 		   REL_HAS_ALL_PART_PROPS(inner_rel));
 
 	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
+	 * Every pair of joining relations would yield the same partition bounds
+	 * for a given join (TODO: why?) so we compute the bounds only the first
+	 * time. Then for every pair we find the pairs of matching partitions from
+	 * the joining relations and join those. TODO: Needs a better explanation
+	 * of why is this true.  TODO: Also there is no reason to have
+	 * part_indexes1 and part_indexes2 pulled here just to be freed up later.
+	 * So, we might want to do something better.
 	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
+	join_boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											part_scheme->partsupfunc,
+											part_scheme->parttypcoll,
+											outer_rel->boundinfo,
+											outer_rel->nparts,
+											inner_rel->boundinfo,
+											inner_rel->nparts,
+											jointype, &parts1, &parts2);
+	if (!join_boundinfo)
 	{
 		Assert(!IS_PARTITIONED_REL(joinrel));
+		Assert(!parts1 && !parts2);
 		return;
 	}
 
@@ -1657,13 +1669,16 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 		   !joinrel->nullable_partexprs && !joinrel->part_rels &&
 		   !joinrel->boundinfo);
 
+	Assert(list_length(parts1) == list_length(parts2));
+
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations and has same bounds. It will have as many partitions
+	 * as the pairs of matching partitions we found.
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
-	joinrel->nparts = outer_rel->nparts;
+	joinrel->nparts = list_length(parts1);
+	joinrel->boundinfo = join_boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
@@ -1751,4 +1766,8 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	joinrel->part_rels =
 		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
+	/* TODO: OR we could actually create the child-join relations here.*/
+	list_free(parts1);
+	list_free(parts2);
+
 }
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 945ac02..58d62f1 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -107,5 +107,10 @@ extern void update_default_partition_oid(Oid parentId, Oid defaultPartId);
 extern void check_default_allows_bound(Relation parent, Relation defaultRel,
 						   PartitionBoundSpec *new_spec);
 extern List *get_proposed_default_constraint(List *new_part_constaints);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   FmgrInfo *supfuncs, Oid *collations,
+					   PartitionBoundInfo boundinfo1, int nparts1,
+					   PartitionBoundInfo boundinfo2, int nparts2,
+					   JoinType jointype, List **parts1, List **parts2);
 
 #endif							/* PARTITION_H */
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index e085cef..5d6e5f8 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -15,6 +15,7 @@
 #define RELATION_H
 
 #include "access/sdir.h"
+#include "fmgr.h"
 #include "lib/stringinfo.h"
 #include "nodes/params.h"
 #include "nodes/parsenodes.h"
@@ -354,6 +355,7 @@ typedef struct PartitionSchemeData
 	/* Cached information about partition key data types. */
 	int16	   *parttyplen;
 	bool	   *parttypbyval;
+	FmgrInfo   *partsupfunc;
 }			PartitionSchemeData;
 
 typedef struct PartitionSchemeData *PartitionScheme;
-- 
1.7.9.5

#10Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#9)
Re: advanced partition matching algorithm for partition-wise join

On Wed, Oct 11, 2017 at 7:08 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Here's updated patch set based on the basic partition-wise join
committed. The patchset applies on top of the patch to optimize the
case of dummy partitioned tables [1].

Right now, the advanced partition matching algorithm bails out when
either of the joining relations has a default partition.

So is that something you are going to fix?

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

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

#11Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#10)
Re: advanced partition matching algorithm for partition-wise join

On Thu, Oct 12, 2017 at 9:46 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Wed, Oct 11, 2017 at 7:08 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Here's updated patch set based on the basic partition-wise join
committed. The patchset applies on top of the patch to optimize the
case of dummy partitioned tables [1].

Right now, the advanced partition matching algorithm bails out when
either of the joining relations has a default partition.

So is that something you are going to fix?

Yes, if time permits. I had left the patch unattended while basic
partition-wise join was getting committed. Now that it's committed, I
rebased it. It still has TODOs and some work is required to improve
it. But for the patch to be really complete, we have to deal with the
problem of missing partitions described before. I am fine
collaborating if someone else wants to pick it up.

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

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

#12Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#11)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Fri, Oct 13, 2017 at 7:59 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Thu, Oct 12, 2017 at 9:46 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Wed, Oct 11, 2017 at 7:08 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Here's updated patch set based on the basic partition-wise join
committed. The patchset applies on top of the patch to optimize the
case of dummy partitioned tables [1].

Right now, the advanced partition matching algorithm bails out when
either of the joining relations has a default partition.

So is that something you are going to fix?

Yes, if time permits. I had left the patch unattended while basic
partition-wise join was getting committed. Now that it's committed, I
rebased it. It still has TODOs and some work is required to improve
it. But for the patch to be really complete, we have to deal with the
problem of missing partitions described before. I am fine
collaborating if someone else wants to pick it up.

Here's patchset which support advanced partition matching for
partition bounds with default partition. The patchset is rebased on
the latest head.

When a list value is present in one of the joining relations and not
the other, and the other relation has default partition, match (join)
the partition containing that list value with the default partition,
since the default partition may contain rows with that list value. If
the default partition happens to be on the outer side of the join, the
resulting join partition acts as a default partition as it will
contain all the values from the default partition. If the partition
containing the list value happens to be on the outer side of the join,
the resulting join partition is associated with the list value, since
no other partition key value from the default partition makes it to
the join result.

When a range is present (completely or partly) in one of the joining
relations and not the other, and the other relation has default
partition, match (join) the partition corresponding to that range with
the default partition. If the default partition happens to be on the
outer side of the join, the resulting join partition acts as a default
partition as it will contain all the values from the default
partition. If the non-partition corresponding to the range happens to
be on the outer side of the join, the resulting join partition is
associated with that range, since partition key values from the
default partition outside that range won't make it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join. Such a
default partition may get joined to a non-default partition from the
inner side, if inner side has a range missing in the outer side.

If any of the above causes multiple partitions from one side to match
with one or more partitions on the other side, we won't use
partition-wise join as discussed in the first mail of this thread.

I have tested the patches for two-way join, but haven't added any test
involving default partitions to the patch itself. It needs to be
tested for N-way join as well. So, for now I have kept the two patches
supporting the default partition in case of range and list resp.
separate. Also, some of the code duplication in partition matching
functions can be avoided using macros. I will merge those patches into
the main patch and add macros once they are tested appropriately.

For hash partitioned table, we haven't implemented the advanced
partition matching, since it would be rare that somebody has hash
partitioned tables with holes (even if they are allowed).

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

Attachments:

pg_adv_dp_join_patches_v2.tar.gzapplication/x-gzip; name=pg_adv_dp_join_patches_v2.tar.gzDownload
#13Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#12)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Here's a new patchset with following changes

1. Rebased on the latest head taking care of partition bound
comparison function changes
2. Refactored the code to avoid duplication.
3. There's an extensive test (provided by Rajkumar) set added, which
is not meant to be committed. That testset has testcases which crash
or reveal a bug. I will fix those crashes and add corresponding
testcases to partition_join.sql.

TODO
1. FIx crashes/bugs in the testcases.

On Sun, Dec 3, 2017 at 4:53 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Fri, Oct 13, 2017 at 7:59 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Thu, Oct 12, 2017 at 9:46 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Wed, Oct 11, 2017 at 7:08 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Here's updated patch set based on the basic partition-wise join
committed. The patchset applies on top of the patch to optimize the
case of dummy partitioned tables [1].

Right now, the advanced partition matching algorithm bails out when
either of the joining relations has a default partition.

So is that something you are going to fix?

Yes, if time permits. I had left the patch unattended while basic
partition-wise join was getting committed. Now that it's committed, I
rebased it. It still has TODOs and some work is required to improve
it. But for the patch to be really complete, we have to deal with the
problem of missing partitions described before. I am fine
collaborating if someone else wants to pick it up.

Here's patchset which support advanced partition matching for
partition bounds with default partition. The patchset is rebased on
the latest head.

When a list value is present in one of the joining relations and not
the other, and the other relation has default partition, match (join)
the partition containing that list value with the default partition,
since the default partition may contain rows with that list value. If
the default partition happens to be on the outer side of the join, the
resulting join partition acts as a default partition as it will
contain all the values from the default partition. If the partition
containing the list value happens to be on the outer side of the join,
the resulting join partition is associated with the list value, since
no other partition key value from the default partition makes it to
the join result.

When a range is present (completely or partly) in one of the joining
relations and not the other, and the other relation has default
partition, match (join) the partition corresponding to that range with
the default partition. If the default partition happens to be on the
outer side of the join, the resulting join partition acts as a default
partition as it will contain all the values from the default
partition. If the non-partition corresponding to the range happens to
be on the outer side of the join, the resulting join partition is
associated with that range, since partition key values from the
default partition outside that range won't make it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join. Such a
default partition may get joined to a non-default partition from the
inner side, if inner side has a range missing in the outer side.

If any of the above causes multiple partitions from one side to match
with one or more partitions on the other side, we won't use
partition-wise join as discussed in the first mail of this thread.

I have tested the patches for two-way join, but haven't added any test
involving default partitions to the patch itself. It needs to be
tested for N-way join as well. So, for now I have kept the two patches
supporting the default partition in case of range and list resp.
separate. Also, some of the code duplication in partition matching
functions can be avoided using macros. I will merge those patches into
the main patch and add macros once they are tested appropriately.

For hash partitioned table, we haven't implemented the advanced
partition matching, since it would be rare that somebody has hash
partitioned tables with holes (even if they are allowed).

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

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

Attachments:

pg_adv_dp_join_patches_v3.tar.gzapplication/x-gzip; name=pg_adv_dp_join_patches_v3.tar.gzDownload
#14Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#13)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Hi Ashutosh.

On 2018/02/07 13:51, Ashutosh Bapat wrote:

Here's a new patchset with following changes

1. Rebased on the latest head taking care of partition bound
comparison function changes

I was about to make these changes myself while revising the fast pruning
patch. Instead, I decided to take a look at your patch and try to use it
in my tree.

I looked at the patch 0001 and noticed that git diff --check says:

src/backend/catalog/partition.c:2900: trailing whitespace.
+partition_rbound_datum_cmp(FmgrInfo *partsupfunc, Oid *partcollation,

Also, might be a good idea to write briefly about the new arguments in the
header comment. Something like that they are PartitionKey elements.

Thanks,
Amit

#15Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Amit Langote (#14)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On 2018/02/08 11:55, Amit Langote wrote:

Hi Ashutosh.

On 2018/02/07 13:51, Ashutosh Bapat wrote:

Here's a new patchset with following changes

1. Rebased on the latest head taking care of partition bound
comparison function changes

I was about to make these changes myself while revising the fast pruning
patch. Instead, I decided to take a look at your patch and try to use it
in my tree.

I also noticed that a later patch adds partsupfunc to PartitionScheme,
which the pruning patch needs too. So, perhaps would be nice to take out
that portion of the patch. That is, the changes to PartitionScheme struct
definition and those to find_partition_scheme().

Regarding the latter, wouldn't be nice to have a comment before the code
that does the copying about why we don't compare the partsupfunc field to
decide if we have a match or not. I understand it's because the
partsupfunc array contains pointers, not OIDs. But maybe, that's too
obvious to warrant a comment.

Thanks,
Amit

#16Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Langote (#14)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Thu, Feb 8, 2018 at 8:25 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

Hi Ashutosh.

On 2018/02/07 13:51, Ashutosh Bapat wrote:

Here's a new patchset with following changes

1. Rebased on the latest head taking care of partition bound
comparison function changes

I was about to make these changes myself while revising the fast pruning
patch. Instead, I decided to take a look at your patch and try to use it
in my tree.

I looked at the patch 0001 and noticed that git diff --check says:

src/backend/catalog/partition.c:2900: trailing whitespace.
+partition_rbound_datum_cmp(FmgrInfo *partsupfunc, Oid *partcollation,

Thanks. Fixed.

Also, might be a good idea to write briefly about the new arguments in the
header comment. Something like that they are PartitionKey elements.

Here's updated patch set with those comments added.

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

Attachments:

pg_adv_dp_join_patches_v4.tar.gzapplication/x-gzip; name=pg_adv_dp_join_patches_v4.tar.gzDownload
�.1}Z��ks�8�.����_�/%U�9������W]����u�u����9q�A���Y�������@� )Q�BT�L7�Z��`���]������������s��N�����Lc�NO#���Fc�7O������a����w2���E��B�^��x����������Y����������S_�s����n��z�%9�$����q�:�E��}�p�9^mf��rq�BJ����6�������p;c�;)�?����?�_h��x���a�|A���C��YM���!�����$������������S��r�>%��w����H�C>��#R��I6F�/1^���������'Yl����l�<L��/_�|�S�I��=y��C�'^���3\�x'�����8
�G�^��'���@��_&�?�o��������������������������9�?<�L�E�/���@�*A/���%��?�{�x��f�^����K���hS4�������3���<A��=����������x:Js�Gp�������lC���
<T��f6���	.�^�q�1�$�����dM^���>|��@����}�������'t{��r�~�������?��7���������������y�����������~�#�<�{6 w�X��N�_�nM����(D�$��W��N���*y�}_�;y��<���1�yx��VM��?zG;��[��\�����OP�v���?��'����Y���]'+Rz=x7<B�������!A�2����i�1���U��f�����	/����;���e��W����9�-�����%�Yx~B������/^��#<��Z��t<��;oD���������nq��S-��������S�&��I�|��K��x%�:>]��dAn0������z�zM~ �I�����;�����������y���S����'���3��?g�
�%	������t;��B?�6��;�p��{sI�ywI��\�.�?���i���Wv���B1Z�q���~9}Y��v��t�SjNfM��g�c���<���
3�_g�Y��{���W\��������c��>/��^!�a�~��,?�\Ln/�YF����M��=��0m[
�/��������_~C7���^���:�>����h��o��&��^`rs�3���*��1+��L��:�HU�oR)���U�&���i=?O�A�P^Y(�����_/oq��W�.nR{M}������Fh����h�)��������	��nH���,���0�%��1~��������������	�5m]�zy��nh�NT�M�&���WA��#�����lt}�����/[����k���AI-uk���AI=Fk���A_��b
f+b���kPh����k0����'��p�������/���Wh�����Ox`>	KO�����#��=��?M���}H���7�/���
�"�]�M�~"�S�t}���
Y�Y�8��Vq���zq�����=%/`�p;���M�������#�Hz���M>NL�����e/��a��"}��r��h�������BFR�����{,�a��_���+YM)��x�4}�������N���
����,��\����������l�?���|w-������7
�7
��4 o���A��6�4��44���i����i�vo:�Z~]�?�J�cDU��w�\��7'��K\����8��8�e.���K��2�D~I��iI���$�qzd%�K�0�Y��dy�G1*s�y��AX"�����_�����C��1Y%X�c�d��qG�n����%�	�8��b>������Om�N��Q�:^���e�Tu�`�7�Kn��U�����nfO���L�1�-+6��|O���#�IM���E
w����G7~������nL����p���D��uc������t<&zl�Bh<&hA1&� �`��*�M��~������*E�G��TS�
�{������e�<�o��&��
�#+�1��BcR(�G�82��]� O#z������[���(��kIo����@� /C_��i~/���qYOz+���2[&�eB���������(��y���h�����8������9�V��Xb��*t�����h���f�z#����� �r�,�����>=>cSA��V�A
��� ��n<�����h�V�����z�hO�1�"���I��I`;&Z��`L�m+c�c����4-B���Im�7��q�qW|?J{������>�������!�XQo@K��-IK��}4I�/�����|.��>����y9������;"dy�`~�2B�3���3�'�C�<?�zyIWg^J��_��0[�X(e\>����Lf7�j77��������!����f�����cj3�2�����r<>E�"�,��`@����N
������H�MjP���tm�k3Sxl�iL=fL��M1��tL=�1Ua�N���������J��bL}���c��v�������1
�1�bG���1
,�T�
;SC�^;c2c*���1
�cZ��
[v2�����v��q��(V�S����w���t���L��4�`���!:�"3�z]���k�H��GVW�<Noo��&FLwzI������I(�Sk��V.9�Z��|�����9g��/z���0����T��U����x�\~jMWmQ]��K#������*���u<�O��;��G����#� �{I���'Z<���6E�^Lg��#���������LD�K�<b%�[�r��gUC���������%)�������xvC�&����h���1�2'5�2�-s�n��f����;B�f��Yc'<��-*�����%[#6|����6E���O��������������8�?��#A���/�?w���V���5��}�����}���������<�e�j���
�6^��$������e�{�V��f����\A%H*�75��~2`2��0nm�F�Zg,7���������w�����9(z���Q�g��\?(,�V�� e����q���9�Yf�|��A2����r��r3y7*�z��t~&�9�a>�L��r�F.3���p�.��c7�������L+#�#�����X�l�3������c�]v�����JL2g���L�x��c�
y���!�a�-�����=v�����c��8��i��m���c��H9�������}�v���6W�
�-��O�,�C�K=�D74���Xp8^fc��l\�'����,6T��v���i�<aXcy[���2�p��/O����iyw:���I���zR��H��d��|0����mH��]oSzmG.��e�v�H�WhQ t�J���WhQnk$�
|�����J�XD)_�[�)�����\M�4Oq��y�����������m�>=y?�s���S��N�s��J���A<7����j+�����fr�3��'�K4�Av��y�7�����s��2%>�APo���O=��z�(�����
�*_�sk�q����@ak���������O�V��KaGK�@[�w���*�m�L�Y61rF�&����5
S��e�j$��7�P �y��sg�i�&�5���k��Z�����5o@qPP����x�M�����_��=����O���y
��7h<���h�&���K("e�N��W���E�A�4�?���4���7������jY�F���D%��Jjn�9<2�!U����!MG��H�]D�W����'�rJ��
W��;�P_�ZRu/���#fWe�/�M���"��D�����"�����
�y����?�]E����}E������y��[K��E�/,�u��Mm�6���l�iT������"]��e��DW�O]_5>	�MBZ��
�|+i�|��	<{kl	=uc�^aH��<{m={{I�3o�i�}�e�E^��W��$J�EC���+����v�T��)bQB�G����'����6�Q�M��h����jp~##��c���x�����,HB+Q:���	�Ei���WV)U`6��c&�J:]�����aYH��������8(��T��Z�P�^�^un���5l�}�4���Z���������k�j���x�P�5�a����7G��&>'��Ai�M�6&�$�n�k�Dnki��c���������������1�&A�k^�&<]KoX�M��ad�7HZO���&��U�=d|�&�Mh>�H}B�>����>������/��b�z,LYNx��� RY���He;���7�`e�^�����7�T
��e{��W���((7Y�������k�d,>X=Y`��
lU��qp����^��D*�U����	S`�2{
����A��;9�<�(�U�0�*��VU���`�*ms�.^��J�Y��� cu-c�E{��h�����������L�����"��4��i�]���
jz�~�
�;����{7Z�
��1�l7�s6�'�9��}�O��v���k�v���9��j%�u6���9e�}�O��v��k�v�}������d��hF�>��zwg�	����XCj���������YvB��{�k��Xc�A�p-����v��5��0l�o��3Z�����~�c��n�����d���N<��V����D+��s��`�`��4����T���aY�����5�vf�qp p�3{�7m��.��O���&�-����m&J���]mw�]�
�O{?\�L>�Xb������"qK��~��5Y z	}�O��&���������gr�p�s$G���q.9X��<��wY�t�3\������93\�����:#8�\��f�����y={�4;�F<|���q~�x
��^����a�
h���4�a4l��V�.��a��h���4��4\���?�N��=sy����?{P��E~�L�&0����S}zR=N�^��$y0�����I�`85{
��!0	N!.	Nk�U�i��pz�"Nk�[��pZ�������b��8NN�z�p�����W���S���;
">r�j���X�S��88Q,�f�qp�;XN�^�M��[F�`:e���NMf�-m:m&F����)E:l{)�T5��-� X������xU��:
BVM����;JA��d�a�_���
�+�U��;t�
lUf�qp"���^���)�U���Q���(�����N�0��'lU�%R��J���V`��6w���������+�
2V�d,nba[�����
�L5��K�S��k�D�*��88Y
LUf������i���M�zK��  �>�XuXF+Hi��t)��a�  �Q:l3�4J���c��d$GH	)��l��4 4��/M`8	����z0�����I�`85{�����pj� ��?0��?0��$�Z�����E,0���� n���&���5	��"q0�"0�r�����pz|�;&��~�2�-��l�X���r���3&g"�I�6u����K�POH�d�'���<W���2��P���v��Z�f�S��v������F/���y16q/Z)Z�]�*]l!����
��h�t]s�J�*u����i�T�uKR��i ��������k��s��$+���X+����*�Bk�Z�J���v���SWm���I?�J��F�)�]S��j�^C���~%Y����Z��
��8�V����4�j�h�u�vH^�4���yi�����w��5�&�W�h������Bkc@�K���v���FWm������J��Fnv����<�^Cm��~%Y����)��
�Y+�63���4�j�X�t�vH^����$Z��[�Tg^���l�^�&��`��"\����y��y��A����0��'0�w?`��~���0���GR?m�?`��O�0~����~����O��>�q^C�y����>vUv�>�]�OM���f0��'0�w�>`���}��f0���GR�>m��>`��O�0�}���^�}�RcV�^Y}Z2�,�jK���e���7�s������~Z���l?��$7��N'���X���`j�;��?'2.'e�������`j�;���\Tu:��A&��kP[���5����9�q9�lB&�&��P[����^���&�+��0�nk�z����Mv��@����	�A��@`�;P[���@`P�0X������~*����V�`'X}��S)V����V�������������_���/��l�P���@��O�L�t9=�_��s����x��mf���<�F�5zH����|D~��<��\�g�y��'1+��|��Z��|D�d�'����xs1��`F	
b4[lFx��?��K���WC�������G���r��\��b��A��e�����7�o��_/>��s�����z�n��`���3�jL��#U-�I-�xZO4N��x������^Z"#@���? ?������������;�nHk{J�*�$w�d5K��c\����C\���Bp[����o�U���+�v�2�
�/��i^|��H����/�I-�y��H��������[q"�b"�Y��OdT��@�HJ`�|�
��)���Ds�lTp����<M�~r1�m�.�]��`��C3c#����9������������N�i��n�M��?+����(�DP�{�����W�	k@��ia��i����P
S�y���p
!�#�u�y��f9O�q>Z�U��O��(0�P��7�����q��QR����na_t�k��������������=����`D�m�������@�`L�
+
�K(|���G������!����"���x��o��g���=/���|�7�Eo�Y)�)��B�=r�0�a���y���>�%h������E�=\Kz+�
}�y���M�{���z�[A����	�2!-�u�4_��'�������*�NJ���+V�eE��}E�&�&*!a�r���CX�NC���O�.+0��+�U,'���
<m�To`ioR���M�~���`������t�s����Q�#�+>E�����T��Oi��zZ2���h���xG�_�sQ��<��,i<��_������<x�>?���UE����!Yg�U!/�_�R�/�f�m(5b`���:^��av�,[�-[�m������fd��I>���0����E,�)H�`@���6�d���h��m��cl�%,���T�^������u����s�,N*� ��S�N������w�+(�\��e�����>�(e����u�<���5z\-_R�1[<��L�\?�?��8�$������mX���w�_��z���Z9b�b6�������n
b�)��,(�s�G��_$<�q�s�5���<a	Z1���F���T�Z���kc������$���O��W�h
�(H�Hj����������%)�������xvCR�z:$�peL�M���Z��-��T�7�U����t:����3U� ��,�w����}^�����{|~t�������������O�M����j��>�w�}�>��W��^��2A�LHs�|/�����evl�Y���a1\�Y>��(WP�
����=�*002`�Q�i.��2;�������|��/V�`u�dHi�����b�h������]���CB(��J5���[����v[TiK�O�������2"1;r��L��=0��H�DuJ���w^-Q(�\}�,���#w�-��j�CbCT[?;�?�������)Q����R��hS����p,����p\5V<�&�
��T�?=���x��hJ��/)6�P��)�3�;-�N�2Vuu}�.������'eU��&���`@r�n���P���&�lO��B���x������(M��4R����S��Q����i������y~�~\I�6�	�g����"��s�M}��W9^Nn/n&�<��|��1���^�b���M:=#�i�	*S���(�����N��h�fT?��{��V���h�i�?����*Yo�t�\~��l��R�
���vW"���sM�}��};��3�������5��F�U����Z
��y�(<Z�������]����GY������7���O5�?�(F�����
q���r����Z�a(��T�$�M3���A������0nT
b���������^���D�/3�T
"b>^I�8�������XR1�����c
(�|�*��;���t�����r>X����W�1.�y��!�b����B]�x�"�-��K�������|�
���������B��'��<��C����}pr��,�zI�2��i������e������$���e����+����v�T����
��(��8\�}�[�������!��$jR�
%h� ��V���c�+%�����;���`�25����o=��Ylu�0�Vla���;;sX�RM{���;��F����mL�m�fg��W�i�5��h�
�sU�|������d�"�3yY1@�C+�	�q.��,��:X���w����,���f��he�������,�]����A+�]������U@U{�������t
tW��Z��J���,�]i���+F(�k�eR��C*(w	{�.a�2�C��-4�d�W[a]�i�=	�R�{���'��8o	�;���Ma������
����	Y�>��9������N�lc��N&�P<E�	��>�Q>�g�[�p%�Nv����d��C[�����
GJ�n����U
r�j�����`f����3�e������`�������!��%������B���{X���%�m?�bM�����~�u��&�?�e��Hz��4�xl��$�.�j�(��b�e��E�����^�I^����=H��C,9B�����DH�
h������5n�v��%?M����H}z���f�x<�A	����I�@	��������O����|K�H^$�H�F�;
@�L5h#�6��5�4 �2_ud���&����#��&��"�VM:l�#����3��^���:�|��+�c�]�����C�V��2{
U�J�
|�@wU�i�V�}�AwuXtW��f�������&��@u��4�+�2���:,d�+��x��j�l0��a���t�@�	@�a)�`�Q:lUl5J�_����@��@~���~tie$�dPF��[����4{
���F�H��edM��H\����=���u��ed������c"���/��"����5�,W	�<c� ��dma����B=!���PZOD�d�G�`u6�v6������v�����Tu8x�J�z������0}16q/Z�+�.HE���"���v65��t]s��t�:VM��dAUj]5S�G;T����^C-�Z���@s�b-W�!�Ck���d���T��@k���C���eP��K#sO�Cs�����5���W�h���*W���*ZBk�G�K���v��>JWm���I�J�^+:k���S�����N��"_M�E$��Au�y��y��A�@u�P��'P�wT'�:�	�N@urP���	��9������S�5$�~�P��uT�iP��v'5]�	(N@qb�@qb�=P���'�8���(N�R��I��&-)�LQu_�Mvs�MT�	hO\_FV�'�=i�w;��X��\:Eu�I��p�A��V�v�C�%2~Nd\���&��hR���Ii�{�I�K)4)�����h�����e���@���T�3:��	<P@{����Z{(6�z���F^@o�o��^���7i�7����t;���E:?T��?����.�]N�c�:����j3#ApQ�x���9���h<>>����s���*^<%�����l�>/���t�����BR��|��Z��|D�d���}�������o.&��p�A�f��/���}�W���j�~���~����6^�7���^���:�dSf��@+�p}��6�����N?�J����
���<����?R����B����QW���C�����z>^}����d�KKdm6B3��G�,�� ���>�L>������d���Mr�NV�d= ��G(:?����������\M.��b�W_�����l|���N����WR�~~}��Jj��__6�>�������[q"�b"�Y��0��8��b����a�]��s,��K9SD���\t�z5����@�*�L�X�X�d�M��Y����e��5"��`���BhKo�G�V��M
�pKo�o��i@�T��;�i��m������K�4p�?��!k!�kA/�r1?�'����0B_g�g������X�1Y%����]�3��gG��	��`�n�S���&Vo���Y+��m
�I�)c�
F�XA����h��C
��X�������nL<����;��|��cj���)?�WG�HV��*��G��PC\�7����{^�����or���9�RS*+4&��{�a�����
�4�W}LK��i��=����{���V��
�
�2�U�����/������?-�eBZ&d�i���Oz+���J�Gl�������)��W��Tr����BO*Y~���W�������#8�2�rW+�cE�
	���i�O���7EAc9W�Un`OmR��=�I�Z���~���E�}��%�Q.��;��(�����S�t|*{������bE�-HKF�dDJ<�������(�}���	���/�j�N�Y<d�����������I!�q�a�E��U��e�l�
�������T��%y�� ������������d��0#�U�������5�/�`�MA*��T�9$�6�FC�l3��c�/�o�����n��t�f�5����uv�5-�oz���S�8e�-O��)1c9���N�g���&�&sX+��>�v��X��u�<���5z\-_Rf4[<��,�\��?��=�$������h�"�kn��L
��]k��F��K�U������V�A6��tF3��B�qGv�&�����������o����q�O�D�V�+�V��#�/��U�;��h65��.����c��d��#�R}��k�C��!��Er|;{I����:i:$�������F�]Bt��%#��z���!�v�a} W����N�Q�s����dQ�ET�������G��Y~������c>?4)���q�i�i��}^�R���N������j~^?��gZ&��	i��oc������������1��3�']�
*�A�����
mF�;���e]V�bS���4�������a����I#}������X�����Ai��'U^%�
�q/�����Ie��"j�������nl2:?�8�,���ob5HN��������Y���R��hq��D��hP�R�*c��s#2��H��3"SnDD�LL�&��1��R�c�c[Kf�FWO��^��g�sw\�����p�����s����9(6�G:(��g�so_��)3S���)����B;c�)[�_K��c�R=��c�&=��+n����J5T�?=��'��k������G(��������w�C����E��Hj����hF6�&����`�k�nO�iC~��z�2I;���+�(A�#��B�0��r*;�}�~���JD)_�[�)����9TM�4Oq��y�����������m�4|~�{Qx��G���I��W�	"C�s����s/'�7�K�KM>����@O/|1O��&���4��)�Y��b�yL7�O�C�^3����=nC���K3���4���v����6	���l�]�r���pfj�����A 3#wve&b�<��Im>g��V�w��	��
|��Q���2�+Mt<�6AV�w�?�0Y��QP���g���?���s�����i���n�P��A��3���I�#�7����������F���DO{%U���gkk��Z�tTX_�X1_Ew\
�T�O��
�l�B+��Yk�[1��f|���o��TI3]�����W��S�3n�y��r�?�,�W�k���v��?o��W��b}aY_����g6��f{���HW_T���W���x��c��~9}1��UXO��&!O��U���xu	C^]^]�$Q�'����U��B�����y�qt�����}��akCl;��*��K��X����
uu���o�����}��$���������F�o'ZJ*���������A
�c���fw���x����b����,�R��V ma���f;{�Y�RM{���;��FQ����mL�m<cg=�W�i�5q��w��5^��jor��5��-^��=��<l=UB���T"��h���g4�.,������1����G)�[��o�sdm;�z'3�{��o�s|m;�z=�{��oo�Pv{8@���l>,�l�Y����k����k����k���8�wp�;��4��S�����[@����6w��t���e�
g��d��C��ph�@7������!>%MF��1�6��mE�����������M�+�w9s
�+1"�n�:��3c��nl�l���D�t��(��s�8��n';m�|'�]��O�m�}�O
!i�u���l���-M����)��}.DV��N���1����d���4���i>S�=�5vZ�1��$�8�v���A�56�j�w�S�Z+yEz-�/���F.S
^��/��88-0xY������nO���WN������6C�{�iQ�]e�]#�i���E���-�m?�.�,@�B������"�9��M�����Oy�>=�I*�<�B{�����=���Me���B�x��Xq^�x�$�g/����J��C,9JU��^��[���z�#X%yC[���+�e��h�YP]1�.�7����[��`#i���B�8��-^`�0{�����a� `�>R0q�6R0q�$@�`��
&@�`��E�{L��&0q4��xy�q��C�i����T����$/�q����I[`�0{�7-amP���WF��K9��}0�"�e�uM:l����t��@�5ig({?�j��d��/���Ag
�t���Y����5S����5M�c>�i���u}�S#�	��a!k�Yk�;t|
:kms��w��d�
dsn���Vw�5r�jPY����k����k�i0�]<`z���[r��P2������J�(�
B���VdC(��+��J� ����� �W~���B�8��-^`�0{�����a� `�S>�8�'L5	�5�8i��P7�8�o����>�8���VM���c����/��"����5�,W	�<c� ��dmS|�}���zB�'cM�����	i=Q;� ��B6�# ���1�D�F�hg�z���t�l^�M�K�V�p�R�8t����e�]�M
��9+t�:VM��
����?jf��vh�3�L��Z�b�J��	��~��
��%��}���4�j���t�vH\�TX��yidD�vhD6���Z7g�J�-�[�]�R-^h�U�#yi24��5Pv�����8)GU����U!��������k�����$+�y��	W�!�����>��&CS�^5�������]�Du��Icu��*����W�5&��`���"��@3�Yy��y��A�@3�Y���'��w4����,hfA3�Y���fVR4�m��`4�����b�W�Yn�8�!!�>x��^����N�^�������eA/zY�zY���^�����,�eA/zY��J
�^�����^v/Uj����Z�%���"�S��nN���9�����
�r����nG�YKR��9C�x���5)*Z�0�h����T��D����K�NP��E-W�m�og�Z["���N�`A]kR��\aP�����P����
���R�
���������v��Z������P��	��E*ZP�V
������-(gA9[)����^����l�0�e���^�e�R�j��j��n�A���\^���C����?�2�����<���
��63r���9�������h/�D�|���h�����l:OP��e�T�H��Y������3*�M�r�?z�������-4�q��^y��=�������2���x��l�lo&W���$u�I�������
������O�3��
���
��j<�j�:He�Z|�Z��D����W�.nn�����%2"4���#��.������>�L�������d���Mr�NV�d������C\���B	p[����o�U���+�vs2�
�/��zN|��H��+���j�I�TS�I=����.>L~��g�/f1����,�Vg1H����'4����[�����_\\e7#B����I�ek��J��4���������Q��
����F�=��tc`��8-u���P!m�`����*�S�9��)0Ql�	|��
[M�7lE<�-���G�j���i��i1�t�(S�y��A��MF�Fl|}^��c����d�,��:"F�`;���m/���I����!3�^N�:��p}4�7��h����B��(=�|�k�����qy���y4��Mn�4GV�cJe���Pv��!�p����A�F�*��i	Z1���G3zQv����{C_��A^��*}��^@�����V��'�e�LH��l�!�2�Io�]<}^���$����B�"YV�sXt��9�e���?���URk��Q����k�To`�iR����xg.���o���)��A���~����O��)}:>�=E�S�CZ������%#Z2��<�oz2��:���H�=��p�Bt/�x���|.b����>�8�������a���]���e��g+]��O
2�d�����(~�JQ�,�-��T{_`\>��D��f7��`��j����9U��2�@z�Q�*��@fR���b����X2VP�A��cs�QN��.�g�L�P�q�U��VkS1��T���Z�����~,�����{K����S�<-��$�e��
jB��8�A���
���e�c���}^�.���4�F���K�2f�'�J�qz{���1���%��g�(wUV��5M����W�t��JG����������z�R����������.H�\wJ�&�$K���K%Mc��o�0��+:n��Y���q�hb�|��5��~����+���H�r*jO��&f}�c�#b��_�%]}��E=�7��$rD+NU9v���h���1���0����g��|~$hg��%����z�����"��O��v��O�������W��LP-�\!�FU��)y������l+d�O�,�T��BP����yjN�fg
o�L��-��[�Z���T��V�����l�S�la*����t��\T��zrQ9^h�-B�l��ieDbv���r#"�z`��Z�)%~��0��]�?;u������vlQ1jLO��C����w�1=���?�XjD-�s�X����X���&��{�d�����8���U�o�%���%�>���<�fw����P�F����?>�Z����l�W�$��|xrH�
�T
����r@PzT4�����B�
��iD)_�[�)����lPM�4OS�!5�{~�~\I�6�	�g�J�8��oc�M�W�W��]Nn/n&�<y�|��1���^�b���M:=#�i�	*S�������P��h�fT��{��Vy�V=�j���b�U��$�r���.�sq�"��)8���J�&�-�q#��
�Rr���$���Jr(�R]��v�M�@L���������������Z������VVe�o�k�O@`o�+��/3o1������d��TW��b�����V�W����.��T�U����WU0�~����Q-W��b}e�Xu�$�a��v35I9~�:Z���V=-���.��b�.����������P�?^��x?��R	���x��c�I�~9}1s/�^}����I�N
v�>��mtZ5��,�|��,�`�dl	��'�`�pe���b{�
_��|#�F������'��^rN�q�BJQG$�K�5���4T��\{����Y�#�H�J�Ho�����Z0�h��w)�@�J��.�i���4�����fT��uQ��e�<��|+�
�� �TiY��y-�)q@��~�n���J�B4����K����i�Ghw�q��%9;9�98	q�qRv�xR�h'<�����2���;�'�O�;��Q���2���2����[d�TOM<l��2�@PA���J��]�I�}l��M�����x�y:�m��O��t��W=,\�m��l���yy��iw|����n;�	��=u��zu��s'=��w����g����Mx��)�g�y��#[�i�kp�
p�<h�kh�A����5��� ��Jl0���(�v���i����&��.t�[��]rL1�b�S^�O������c[^,	�(���LS��(�<���bG���^1��$�g�����z�KB����1h�����+���
����;��n��mG]mh;\��m���am��{�������o������v�%m���[p�uG�<�;j�;@������[f������d���T���+J�-7�%�vI%�������2�u	e�]��A�28t��AP�+�����Ay?��OZO>7�`P9�d��k_�d�o{-'o���7���d���y���}��V�&��#������?\�
mh;�h;@����>h;�)�
k;�'���m�}������v���������1OB����lof��
^��m�1ayL�n��@�POH�d�!���<!�'ja��m��)��Gd����5}�x��a�6���F��c��s?�����M��E�*�j�R��?mJ�s�|1�f��~P����:�A1��9�V%7|�tyZ�%�MK%��B���"u�j��T��
;Y��B�*�h�����MK���B��bu�j�����
;Y���UU�v�������^��MV�����"l@ �"�RAJU)��Y�RH��)��,M ���b)�T��3)����R�9IR���T�QAFU�@F�!��jYdT�>��
2*���8�~�����Q[�s�,���Q����v7_;����%���j"9�R�`,o�����)c��,b�M��B�e��D�}��D����[����[)��G�\����O�=�]y��O�c�7��%�y�W�XK'�U�i���MyL�`2�(&C�*z��a�<�����mn}�!�������{'^���XD����t;���E:?T��?����.��.7��s���h����K����k4[�U�xJ�����3z������=$���|S�B�G��8�+Y����?;�:['i�$4[��[���������8A��\.\�f��M��:�]+�/9�����br{��/����^���=�������2���x�x��L��z1��?H� ��|��V����mr���'�a~�x�^��5�Uy&U�i~���7��SZO��'0�'��9��	%��t�a���-.������-&D���QF3g#4C@~�I�������������nH�J�*�$w�d5K�0n�N����0�[i%)��\��3tu}��rq�����4D�M>^�f��0{�7�\M.�������o������e��wZM�l5I����$�>_��$��y=��q9�����/[N�z9��r��������S�^��D��g��a�=���B��'������19l1*E��sL���A����h����US=VXyg���6n��F�<p����z�����T�W 5�>$��=.$�d}Cy�]:�j���k��k1����'4�IV���C|�FE��q�����d�(&����'�8Z%��*Y�'u�����/);5���mq���@��l@����������GO���$����B��(
�=���{������e�<�o��&��
�#+�1��BcR(�G��0��]� O#z������[���(��kIo����@� /C_��i~/���qYOz+���2[&�eB���������(��y���h������;a���:��sB���e�9��V)�.�\c�D���,Y��>,���v�����[Qf+:kE5
i`����� ��%7�ShR��7C��8����bP�����xC����@�c�����_�):>�O����(}�xH+V�����dDKF��'��	)9��?=�����y�'X{������;�dy�`~�2B�3��.5�'WC�����������:J�v��3������Y���������,3�[����2�d&5����xQK�
"3H��*��QN�!W�V�q��������b%n��8�[m*F���u��L�!t������
�-e�S�F�����]k�+�iy$T�S�1g1<y�s�����%�^�k�\-_R�9[<�U2��m���?v������{�\M-�Ll�M_-�044Q#��,���P'��*���s�-�!5�R���"���l�-4Z(JU�
|�#SU��e�*�"`I�q�'�
�Wt����7��������H��X��$�vCsA�^W��d�F4�m��6-��
�����.���Uu�t���<K�M���Yz�W�1���a)���yE���v���s\������/�����b��y�b�9���x�*5��|�)y������g�O�,�T2�BS3���x>D�fg� ��Q:�����TG���:��t��-*����W��Sk����q*�]�+��J4M��(�%Z�
s�'g�|n���1���d�h���h����@E61�	S����p���$<��J3�`U�O����'Uv�������'�y��7�=c����g���<�CzK�D'���XpW>
c�1a,x#��.���bC�j����V�S�Ij����4G(��������w�C�%{q.������'��,�&�����s2V�R�7�2l���0����.����
�N>�4R����S��Q�-���i��vO�g)����}�I���v:>9�����R��?9=�� ����l��N�l���*��n��^��}9����\���	��4r@z��y�7���H�~�O�����b(`:��������qZ�Z�_��wW�hW���&y@���gtI��T�#hP�U�"�qR�%��j���9���Q�yup%���| ~%c46e�&�	M��g��&�����bkbK��H1-�f ��[b|��epU�7���' �T�+�^M�������������U5E��*��Rn��ZzuT����*���lU5]��x
!������J�V��%�W��5uI�qzH�M>o7�N���������`Mcb>^��x����&���L�/,�u��u��s�n�i���"]}QY_����e�
��f�N��N��Uw�-^��4z��r��b�q�P����,O���/�Z�������~�%��<^�U�\ A &�����#�����[b�AlP��X2F�r?�D�����O~K=������b!JB*6�n^�iHu5I�C�<�����Te�����LdV�O2aY)���c���P�@�up��Qd����7����q`������V�1W�h� ���A`�H�6!{�K�^��za�����.N1: <Sb%h��P
�@5����Ps�����t���4�g�Xu��6�<�-6���M�����3��QfTcZPM}T��,��-�*5}5f!	<>$�r���oS�*kb���& ������� ��a�BR#1�b`�0���K�w��p3n���;qCe���*����P�ww�T��qG�lG���4�v���F�L�Z�?���������d,�+�c@c�	��l��7���.�}L��1m/�C�v;r,Ml6=���&���|c�,���w
���������i���8I�"���Q�#�O �<��d�Ab�y������=�A��K�Xr0�,�=�O���z�$yC{h�E�#���F��,2���ky�1bt���h\��e|N���u�iA���	��K����\�-A�g�P���}���/����o��,����.u�@������x)	��@��>���x��M�m��-�)P�1hm�~&+���@��P
��f��4�-T�^}jvj�+�4�gL��O�g\�<nYhZX)���Z@5�j��5����m���p�P�)��������m8p���6��v���C�3��H[�`�\}1�7�|��f���O
l�c l���s%����9B>Y��u�iA���	��K���%K�-A�g�P�#�}���/����o��,���w�u�@����}���c����/��"����5�,W	�<cB ��dmH�
o�a!��<���D4OH��Z��l2:R�I�O����
��5��xn��D6�=����
l�:RH����*��JD�/���*D���
1��� RU���9c7a��Um�X6(�H1s��B���B���B,��B#dm�&dL��,�s],�M���0oW���oW���oW��R��B��@W����n��J*T�^N�����bW��Tj����J�]u�Rib��Ax�X����X���~Y�LV�������� +#����AV6-��6���2�����,/�r��	�r���wY����{+*�	g���hF`UI$e��ARI�-�2H�-T�r5��������RpAyo��43&���e������Q9� .���2�� .������,/�r��	�r����������{+.�t
q5��m�e�+����6����iA��B���AP�}7APn��}��RpAy���O7�����_./����z�����v�H�7�zH�����$d���������_�g�J�i����x�������uB#�-��y�^�������U�xJH����d�,�4�������4�qK�^���=�������2���x�/z��o&W���$u^Vf��@+�p}��6������?�J����
��u���~��0����x��������QFGf#4C@~�?��y�������r�nH���,�U�I���j����x<B�x���O9��|���������*�P�����Me�������t�����!��%��:��l�fVC������] �����i�=E�F�/�oN����'=�^k��#jC�#�_��?*�C#x����-0v�f�F�-�x�!&iu
��Q�?��L����+����=H��f!/�<y����&��F�������yr��a.�������#}4p�c_�Y}����-�;|���;^��>�h(Y\I��{WY]3�G����}C�=W�{������|A�/`�Y����g�|&�����|^��c�yY>���e�l�=r��a6
�
2#O�����.�Y�B��L���9,����
�:��;�b��,������H�A��|����0x"�+X���W�����O���i������������������Z?+�N9���:�����<�O0�x���P�v'�,����/#�=3]t&����w6@^�T��_��q(U$��`�o�0�A������@��&-�V� =���*��@fRC����U�r�
����lsHFm08��\�f�O1�u�EA �kk��vG>m)�����/����T��;�4SR��%���E>����^�C��[����%����S��k@�%��1%��%�����
G	I]��kM*�L6���+im�M�0���`u'����N��`9��:��>�w��������P$��PqR��?�|]��k!i7�o=����r�h�y��������o�O_o{e�1oV�t��nE-�Q&�����0���w7�ry�,o��
����zXv�r����v��������!]�n��K�Vu=��������{^rf�63��T��Z�����w<<�"c�3�N�[�ot�}�2N��D���mee�R�OE$���>��i��1��L��\�E%W��S��rJ?��"A�7�X��%k�Y
Uv c~�n!��rj�Yv���?K�;'�
��T?=���x��H�����uG(��������w�C]���E��Hj�����%�9m������d��JRS �x�X� v�jp�<�'�Ly�-��M��\B���N�� ���O����?G^p����<<;I���R�G��ip��0O�������6���*���G|��H_Nn/n&�<�|��1�eM/|1O��&���4���Y�S��S���p��kFZ���6���Z!P�p�@��d�I��r�]��
5�{�,��R��&C&���������$2��g5�&�n�3��/�8[�5�x���5�d��'T2e�!��7�'�o�)N7�c�`��Jx}	��e�'�E���-T�0�����@]�����k���|umS����cj���eJ^�#��8LJ������Y���1q�_�__�<z
��O�=yM7h�]Z�h�{��s��R��23����e(%C
��P5�r1���@�2������R������3+�0	6�4���nZ'J����k��l/�C��nwhj��v�>�E0�j���b�]��N�fe���@�y�[yW���vK���q��n��k�)"�z��S��Q8p�)�v��P�������}��B7��N8�aI�n��T	��.�5��Q��4�/�x�B����;<y�!:����}~NVOI�q�����xwL�rN\i���r����@������F���4��Tw9
��u6P��8=�yR��f�l��f��p�M
|��y[��+��������$�����V�A������k���g/Dq�k�	��A'��n�>��rYR���,qN=�n�M�3�}$�����1�%�z�\�.q�/�+�u��~�\�.q0���u����\�.q�0�����l�s��P�&b.�XSZ���&�-Y0��$L��{��8�9$���N�}I$��h
F����!�����=�Z:P������@=�z�Q8p�	
h@�@7����M�@����u���+�����zx9�:t��XM��l�7���Urp�d�����tT�	����^{��`�dA��p0uzc��f�t��g���j;t��A�T8�g��/_��=
^��m��'����~���	i��E��z"�'��D-pQ6�pT65���\�\V�Z�U�`DJ�u$���j�M�;�N��ya��W5//��6q���jS���,"$&��tU����E
����������`9���(��:�#��UYMF�p�Q)+i�2U�}�u��wsx[	��?��������������A� ���.���
�c'�A����1�j'���.(�Oe�$I��~�V����2~�G~����%�j2�U37�'d�0�c	 ��%D����T ���}Q����D����*:([:(�3zI�O�"�>1U|r�d���>��-���n�A���\^���z������]&��?��63��<��������7����/��/�<��{K?�/(]zA���=�_��<�������3�,��}���^q��xs���������fk�y�^���]�+���>^�Y~����^0s�[�h����
J����2���x�/L��n&W�����i����?�U��A�\�z��~k��?^�����!��F��h���3��T���G��_Y�oV_^MZg4����:�:�j�:��#��e�����.>L~����>^}����dW�=�h�l�������N<%�do��u��%��23B�����T'�O��a�A�u_a"Z�u\���j�\M.��|%c�����(���/_�~�������f�������>�������N�������[�~k?��Oi��k���l����@\�~��1k�-p
��0�����n?^��B�v��	wy����c��-N{8�!��o��0�N������u1����P=�	����^�9����������c���u�Az��T����_������%�����4����%�k?J^+��e����D]�T����H���<��T��dE�������H�H�H�9��z�2R*�[��
�.�h�r����0��a�B2R*u[��
�.��� ���Yg
`J��J���l���
���Q�^���F7�W��l���C����,��/��g��^	?�3����'���4����-_7Q����r��|h�<&�dq�#%2���85�y�(V7��2�y�*W�Vi����������a�(����A�0���jy���v��:���7������)��H�u���)��C*���:EV�k�
�����Boy����b���t�hr����z�V��v��(R\�*s��Crr������S|�4���[�����/�%���<���/F3��N+�H�Y��-���s�-\�W6�\��f�AF��� �
zZ���=����R��3|������(([e�uO��1�>������G�����Z�4k��Q�Zy�{d����9��L���"K����:�n�N���oE��Ap��UJQKD����j�!2~#U'���j��^�I
���V��M�~�o�7�u�0g�4����S�Y���q�
;At7�BLu�����8�"@*!c�����8� ���
!b��������3)R)�qL
�~c�T���1�����D1�A>����-��V��v�4��4d�4��4tS���1�
<b�'QC}o�@�r�/��"����G����G����F����gH�����u�� Y_P�E���m/���:#Y�'Y�Z�(�g�;�g\d�_�sQ[�y�'�����%^�[���,3����_F�{�".�r�����O�
���/�q��dJa�}���
�M$�n����6fwF��%�R^LG%"��d��B2U{1��s�4�����s5*H�`@|��������`p
+�Y��h]-��|y�����������T���c�)���v@��bL�J������/������+��W���}�������6���@��S�W�i�SK���1��s�>�!7��0�!7�:T��P1��v
ZWc6p��)H������"��?8g���c��f�)$�r�s �,���^���bG�=��/)4�-��*���v�e�?��@zI����e3T��.��W����_��d6}�Y��p���I��E�
U�M�W�z����������+�n�o�j[��h�v�K��a�����]��5�q�t����}����UpT����r��/O�[��9���S1���],�%�ykd~�T�&��s��]��VD����x_�F7[.Y�S�L�;q1"�
�������%):���I��I�s����n����y��q>����=w$=@#�C��u]%�������k+��-�b��$��A�]�����~T�������L��K�zp�?�p�&�IH���Z�'���O��t#>�
���U�y�e�����:%�[�"��Tlg��l��q����'�q~�F��	O8+OQvPc ����}��v�����������U;+]��^e������R���`?�RF�,�C
�?����2A�*'�+g�+;G7S��S��/��pv�0������fu6�
��`mah�]�:��X
<��0���Z�
m��Y[Zc�-���x������e�Z�����z%:\�nV�s��R%:RU����1�:��5���D����J��7o����c�e1�c@s���K����1���cHs�)�[8���1���cHs�1��y}�c�b������y}�c�b�����c����6�����c4Yl���:��OO��)�$5�7T�y�������-���GR8yO1�:�����?�7������_r`l�W����_W����|�����$���z7s��j�l[jZS�Yb�A��-�"c4�xZ��Y��&#��a���]e�5�j2&
��|L��RY��&c�>���]��5�i2&9bcEp������	w����bv��W����&}\�g�>.��I������
j���5�#L+?�V^
��y�*��s{��7����0�OG�$^o�H�te���������C�h�	��V��bJ�H/�8��O���R�8*�(=�`C��'����o(-N$H�c��G���!Z�g���=�,�j�;S���!�\^L>��N��U���y�l��i�<t�A��P�F9��r/����&>�<v��h[�����z�����L��|�.����{�`�]�{�2B\�������.��f:V����@�i�;M*M(��>]Nx�:R�	�i~3����rA'�40��~�&�tW�.'�u���������>Z�n�t��N�L�Dt��Rk���P�I�`+P9�d�?e��1gJ���e�\A�N�E���v�����5���PFU�%,<Q�	�����ue;��)�Gt.��(k���(<��a���O��=i���%eBe�� ?i�F��O������"e;��o��z����������<�2�����������v��K��+v������<��w�������*<�� t����1s���
��!`�r8�����,�Y9�� Sn��?��i�Y�*���c�e;����>�-�N�,"
���J����e`O}h��1�����j����r�6��-(�
�Y�0��5�v�j��"�0V�,��JT�\av��S���Jr�
T�h�����:�*F���a����R�����@D5�l���B��l���'6�)�~:�r�s���h����6>;�M��'KU��hg`k#n|���9�
���v�����F���*6ms��nU�F�}�����	Xl����6���aTe��o�!���#����8�=�#��]��?�r�%H>�#*��NN�����������N�L��h����D����8�)�Tm�A�qJ�S�@�q�&�d���T���)���&4�S`��	��>Cy�H�� ����"��$K��:�� =����2��@x[�S�Z@��3�q�$��S N)�S��Oq�K�
��aJS�'g���<�uU�6<�+K�;�������������H�L����.wf��sv�l�������"��+vE
�����:}�Q���5���'!p]��hr"	�L���AhM��I�\��GN��������������km��� ��x�I6-��x���>.�d����;���X�-���=X�~�\����O�ko�B��$:���5Y�g�r���9+�k�\�,� _�Ak�5��I�+�n�*��{w��p����OC��%�������������>/�0w�y��[�������HP���3�,����CY_9��^lQ5I{m}�Y\�u���Z��l�`k�'��$����U���*K]�Z�Q��z���m,L��*:��`�G��Zg����l��Z6� ^�kYk_B�B uCn�H�g��g��	��Y. >����#;8p>�L����r I�-�	���	�fJ�--hI�n�������Z��na��dN�(�G��%�=�CEj��.��N��3HY�4���&$���u������9��*�C���tx�C����J����:���3{i�w��~<m�'���y�P�d�~}��|���	9�U7�	!��sB���!�!tb���A�iu��uB���u�!vbN��x�����n'r'�:�������u���~�� ��2���4zi?�~�?���.�G�����@,�#����\E'�#�{	���~���,����G~.-������#��`j�G~���G~���G��[{���$��6	��TziG���@���.�H����v	z�Q���L�=pO��	!	Op��Slq*O N$�@�R$�d	�)����=�F��[��`Skq
lQ <����Q�	�'E�I�@x2Ho@x�]� =�Sz�m�
�'�E�8�&����Aq
�)EqJ�@�2Ho@����� M��4�-�^����������N ;%��@vR$��d	d'��W�S� *�N ;�.L��*:A|U��P�H_U�@|�%��
r �W�����$��
�yO���N���_�.'�W�����!���XliuV����F/M�G`������G������[[�?��
~���`C���~v/�~!�:��@�@|�%�#9�����E�#?�t�~{�9���@����@��/~���c����/��"����5�,W	�<c� ��d�Q��|���w"L�+�������0I[!-���0o+��������D���!��e��lul�Ji�J��<ED:?�H���������5��D����ADn~fF�F�*�&
\/�o�Q�����L*�*`L*���&h�����|bk,��_��x��eW��G��Bom+[it���fQ��
�uM}Gh�}�H���<j-���&�o�n���e����
�*�j����VU����u��Yo���v#����N�����J}^-�2t�nd��U��
p���R�W@K&	]��*tw��L�����eA��F]���'�*�����rZ([	o!?&�K�����`�Z7����
t��fw��W���]��+���)�O�������>��O�T�SP�������>�+��K�i}A}
�SP�������������jO�%������k��m�i�@y
�S}fP����� (O�����<�)(OAy
�SP������<�����A��tou�[T��-\1��N;���5"��jB6I��L����M��j
���8�QYA���
���j@�j�]���V�=*�QA�
zT�������G}��(����U���Z�bh�]��m���QeT�A5+�����)�NeAw�Fe�;�)�NAw
�S����t��;��^�N��
t�����ts������"����������v�H���IP���*F���fF��B_�g��h�F�c�:�0�h���������l:Op�d!���,_�����-�*���o�?z�������,4��l��%�����!�ers���#�����\��b����u��,p�!����
������O�4~�x�^��?���
����f��fREZ��G�e}�Y}y5i��x<���:?^}��������f$r6J�C@~�h-O�"Y���n��f�z@�u���|��Q:��'��=��\��_�*���o3]�Q,_>]~�����%��d�|�tI�SM���@Z�O&�^��s�;�qT��@���*��U����
A�����1�������Rq�����bh9i�H��R�:x+Ga4X6�Km���@����Q�w�7w��X�Q��@���(8i,u	��` ��H��������%�� 0�g	i ��X�2����e�',8I�d�Y����&CP�$��������<9��0�zLV��>�#���
���F������M{��`��d�&Yc[�l��-@4�`Kp�I�
��uZ�1j�yw��7-cmxw��@O���n4^��c	'�%�d�7@�������U��5Pl���%�o���[c��Gryy.��>'E<��A��������5y�$F3���
�[8�WV�\����>_ ��[����2s�>���@��|��M�B��5^�:W��S�����`A�,��Q��GcV�&C<�)_?';����5���Yac��h.�dfF|3�G+q����0�q��;s�%����+T�Q������x���"�]y
�4o��xG�_�sQ��<��,�<��_����h�<x>?����@��^�i��QZn����*�}���70'���������)��Hl�:���������OA�|�Y:6�T����'c:��)��gA1���Ol��d6��hXi�)S���( es-�������e�\�K�%o�U���7D#��OY�[p���F��F�`��?�a�cV�*ej��HJ�=��/)�-�
�f��L��?�|?sJ��?#?�t��p�f�dOL)��_b6�j���������^�~p)ToQ��d��,|f�s	���������v��	1��bl9��X��{�B��N=K�����1f���%����<6�#BU�-'����$��|���AL����v���?��u��py��������t�L#�< -(u.8J�<��[���3������f_	,����or���b�D��x����$9��$'�B�MYNi������H�F2N�o�r�b�/�@���evl�������3��
��!fu��,�hv������5
9�"2�&��Xc_�oi"65snO�n?H��95E���SS�
��Y�;�F����cWVn�Nm��5w4`.�'�
�|�yv\5?��$�D��hn��/Sy�:<������G){��bx���D(2���^-����r"���vA�z�Py��[��n��{���=g�<C���j���^gT~�d�?T#��bC�=���OO��)�$5Z���KJ��#y����N��dK���^]���|$����)�����7�0�G�����5�o����V�y[�������b��w*P�}����7'�p�j�����Y�U�<N��)�������R}��1g� �01X�sSgA�(j������%���������E�/�I��d�J��a�|����]l���������q;Zm�����&�@�r��7��\.?K��m�6����}�(���h�����������"T^N����4�P�M��(��f@�
c9�U4S�}��I1i�8�QK�����7�#���;%�	�H����_�'��Lt��`��G��4�T,"�|����#�����b7��l(S�$/��?��D�Cu��������V�g�S�����_f�Au�Q��?��<��TL<�����b��������'��o�X���1	*q�����m�($��[��q��q�G]��q�
Q-��fJ��X\�X��X����
V`��9���X �I���g�O��N\�}�[�c��WL)�����`eq����V��}�h�+�I��Q���
r��"��F�
Z�1�`�Afg�����6��%�oq��1����0���������8��0��Q|%�Q���{��$;z��w�{k��J-�ru�b���^+�{��?�|v�2�NAW��0�F�@n�� @]�71�dG�0v�0�(01@�B��Y�\�����9�����l����k ���1t�	c��F���&A�M�y]���Mb\����Z�zl'������5o�y����v�nq`�IPw�gKH����$��:����x�'vG��.N&=e��8���	P�;��-�r�l�I�J��}�:�`����������?k��p=[AQg��A�R��:����]���pmR���
�nh��v��=��v�����;u�D��/��{��;8���Sm����3"y�/�~� B�rfP��Ek���4uue ����d��_mH��,���I�������kZt�P4u�td����xW�7�
�����*_�$Z�rM�.��&6���
c��%=�?��(�� |�:�)���� ��g�E���)|
�}���u� ��g�E��`(|
�((�5����hP�1A��o�{�^�rO�����9#��.��N}�(���=}��(���(�*�O���;��r�{mp^<���Dm�^��{���^M����i�z��@��t�!�@�6b�z
C��1v��a�a�w�/��.�m������=\����	�������a�@Y�b�� ��� �����^�j�� P_�l��5`�{�9l�?F�b�"'���\��`s����l�Jl�rX:�����6W�c���"l����*��|�@��d���
P��z
�=�{�ro���{��9�f�r�{��c{
�=�A��=�t�������r��*����D�/_��E��-k4xY��y�a��c��]��v���k������Pf���a�P�VDKe��Fy[���Q�rA�l�"u%)��*�HPQFm�(�SX5H%�SQF�*�:�������VZ;V��P[Cg}�v���K�5�",���	�.�����(�:������-n�����\y���
���O�����t���4^�Ry3��7�.)���B���nK�w��x.����$��iWK\&��rq�%�tZ���Y�-)
�]m�F�W��wR5�IT>��-(�������N�in�Gb_��n��A���H u����H��]���C uK��������}(R��8��i�Hb!�Zec�@��dn��
�g��
d��H@�f�� st	dn����������&������|�
! r7�
��G��T����� x�������@��[�%�
�
��Q�@����1�����E'���p{1���6V��o�u��
"����]���; rK�����@��O r[tD�vD�7-4�A�����_���/��x�P���������D:�/���1~�o��x���(q(^<���9���h<>>����s���*^<%����w4[����z6�'��s�����,_������-�*������c���y�������7nh��b3�+'��'?C�������G�B�r��\��b����u�W���W����mr���'��~�x�^��?���
����f��fREZ��G�e}�Y}y5i�x^F�e��Y�y5i�x�G'���W�.nn1�����������!Z�S�HV�&�['�Y����#��q>J/���`��'W������H�/��,��(�
�/_~�%�����f�%�����>�%��������_/o�9����@������U��']���:bxj��T\5�������\�v����%x;�Zd��,��*{@���Q��j��qR���(�5k�7k  
��"���
Qt����*0�x���b�q�L��y�����#�u�y��j9O�q>��U��O�h(yyC�[���b#17��iQ���|��mu�����D����Hu���k4Z�
������I��h���5��Wn��0�����c���Ik�e�s�%?Ik�7@S�WN�@����*b��`�����������\^�����I�+Br�(��/�%�mvM�DD�������������U2��)����$9�
����\�Oq��/��!_-����GyM�%��y?���w��U������M�5Xv�^q�a���|��81���N�^
�fI��	[�����Y2��5k�����-�R4�,�u���W���H�q9��H���Oew�����@�7��<�������(^~���	1��/�j�N�Y<�������J�SJ/��w�R^�T������&���:3!/'�0�AJ�{����u��'gD�'�S�*[�@V��+1���b��xT�|�=�9VAL�����!����4Vh���g"_H�b�����k}*r�����%����v�k-��oz�����8��S���<���yj�]�����<q�s��P����^�Ci�^����%e���Sa�n��)�c��Gd�S���:g32�R�X�Z*�@Q��Q�4�:d]G'��Vk��3f4`��|����K��x���&�/f�8��K0�}�%�.p� ���Wxb(#���r�/�DI����-CI��7�t�y�ZN
��Kb����y�)G��|���{��_��F$1]/����KR4���I��I����Z��������Ey@.�P�\p��y�=��/[g.
9�#��+���Aru��|d
"+g"g7=YNO��r�qLe7e9�u�ZG�~"��8m�<�,g(�9�BDJ^f�F����Z@:���0JQbVG����fWI�Z�\�P��Zk�W�b�� 1h\ll4(������c������� �1(*w8H}l�����p5�98����r�����b�������
�Tz��7��\.?k�-�*�l�:�1
�����FN�`2r?�8�,���ob9�N6���������v���MH�!�$��D�S8q�3Q�?<�(�H��ZO�,�Vi"]���RA����y�C���Jo��a��d��`m�Uz�a�SdX%�@���d���Y%�V�0P��k�*�W�%�O���
}���r�d��r#���OO��	C����%�����<�fw��]�	V�0��o��?>�z����Bc��c����f��2�G��9�������h�J[�����^o2Z[����mC�r|��-|_ �=�Y����'�p�j���b� Z5�q��y�����������m�c�H�E!6��n��R}9����\r�M>��x�EW���'�z�I�)G��y��"RC�a��kF��^���hU�VS�e�0e%��d&����irv�j*qt��jQ�c�5��� �@�+W�j���SG���}�9a��|�(|V�2r �����M���q����3�;���������� �������E~����;�NF�3�g�Z�9*����$/D�f?��Qa2�XN��2+j��*�������O&���P]}�c���@\��� ������_f�Eu�Q��?������u�Q�o��+��	�����%�"Z_TW_D�E��U��-^�����_�__�v�Z���-%p�0p���;v��Z'��8�`�N^��%��EpP�u�B�g9;��9-q��:����t���N�������4�^��C���6x����>n�q�x7}�\�����e��F���y���w��&����5��j�����S�R�LVQ��Yw6�B�����I�^���b
���3�)�x������Y 0��������A�X@����]�.�����>��(P���g(�)�,�, ��6�� @����t�<4���oh�����Uik�(�M}��������xw�����O$������]~����Y8?���;�V��q=�E
O�����Y�I.���$���>E�i0�Q�F��TG���;Ou��Q�S��r�l��S|N${;��>��i0�'��Ocv�O�C�����=P�R8�z�}��f�]�At*��J�n5fPw�}Z�6\���m�L�|�2��i���^�C��"m/�'$D�j(�9V��
)��P���v���lP�@��U-������u���L6�|U3F|��$���P���y93^o��5��sI��:�7���
�B|���6Z�v�K���k^��Y�rM�.8�����,|ZP>M���1k��t�^���H���kZtA�4u�td�����t�7TD.�{���������4�*n��5-��4�GhzP�^��6I����4��$�
��Nd�E�j��
�3�
=��@|
�}����A��P+xIB��5Hb�T2�E7it�IX6����L=0��Az`���m�xH`7��z�4�X�@�@z`E�@z�J=0��@���O�=0[�����l��.uE0(�A�`P�\�C����`_!j @��W���N�8��n�@�z�7�@�Z�
 @�A�����x���^���xT�	�b��@�c�v� �X`��c����=�~����4��1����H��cK���������S�?Vd��c{����1�����=p_���w� �X`=0��A����V-���&	���B��n�0��}wA=0���l�����c2���/��"����5�,W	�<��8@�1�����;su�&i+��2��me�7T��R!��m����e��^,�N0����H�fGmj�#mvd������QW�lc�@�7mv���/���E�P^_�E(��BA��k���e�r"l\	���Bp��-�����aw�f�������-��P-�@oe^��F��JcI��Jc	�Z��R�n0e%-i�doY�����iu�t���Nr�AGa�����>w"!:���-�D����"�  ����J�J@@���C� ���Z��#�[�k��Y��CqA<�-
�!��y �x�a]���v��7��A:�����}�A:l)�~T{EF[��2"�� #���T2"�� #�%�AF�[b�P>���s����x����2"XA:��C�A:l^%H�l% �t�'�����oP:����t;���E6�?T������?�2����!y�_��9^mf$���9�������h/�D}���h�����l:OP��d�T�H��Y�q{������|������xs1����
b��f�WM�sO~���������xu��v3���� M�?H� �,p�!����
������O�S��U#���^��x4V����H�H�"5��U��f����uF���O��x������~V6�S�Q���#DkyJ�*�$w�d5K���Qx~>������'4�{r5�����E\}�f���X6�|�|�i���$�1"�MT��Q�(i�����J��������[qv}��=��n��������-�����/.���!��B���g�:f�EN������o�E;V��#�OlF�)�k{5S�R�U���z�Xj�S��KI�RJQe��V=����3|��7k�����3
����'A�
�'�zr�'Gur('�o<h�4l�<n��u��E-_�����C��1Y%�������7t#��n
��I[%0����9I
@�����~������r����z�k(a-��!zkLn��[�����k����"$-��[��^��f��I
��g43�,�����xe��5y�ah�@���[�*��2s�>���@��|����B��5Y<]_�d�$�- T�N�`A��L�����9��jn�47S*��1�i�@��������Y��f
L����w�t�����x�(�@����w����.����H�F��'cI�������{!G�_�s����	����x�v'��,n������[KtQ��0������/U)���}���}��'��B�
~�� ���n�<n���������)Hk�S} +B����x��J3�U>����PR������*��N���/��E�Q���'R��r}*�Z��eC]_�U��U�2�%3�}+fzpt1��������86{��7B��?��9����?�
���u�<���5z\-_R�5[<�U2�g_�J��?�h!�,O����H�u�F�������W�������*�����]*������~�]*�Zi�F1�jp���&q��.��R
.,��B�x�%=�y�"c%��r�<�D�=������O{�T<LQ-r�������OD�A?�I<�I7�K2�N9x��\'$?H�b���y�MO������W�V�MYNi������H�F2R=�.��uV�@������z���*��`*V�R�������t��UR�c?N���Gj}���1��>�&eA=8t�T����IN�����	�$�V����@� �R Z	D��W�gr�3Q9��x�2OnP-�G�����|�V~NVOI�4�{U,�M9U,�����
�����QI��Em������m
�7���JC���!�T?%>g�.��gj-��WK�055-6���
��Hb�9Yl(������i�<���F��~}I��p��/O����iy�����!��}�����|x=��(�wL'��P��c���A��8�='W�����G�n����	7\��7���kP|5�Se*un������R�X��S�/f�i�������sK������fr��o4���Q��_�b���M�S"3���6�^�b������e�~���*W��(^
���d�I��r�]��
iqY%.��� ��J>�Vi�.N����Czhh�:������x�l�7�l�e�TLi��J&��5��	-�gNA����^���������?q��O�3��d42ScW����R�]QpK�����3�IK�wE
.����+�qI>^p(�,2������C�����������4��n�2%<����T�����RU_��K�����_����?��+��/^����@�_�__�����F���f07Kks�U[s��P����l����e�����5#�?V� ��Xv��,��8�����p�Bb��:�����U��=�����91z�~3���7t!���_ti��F��z��W7�f���gk7
[�y�X���������e}��N�������X��QL�r������En�p��[��f5p�j�N��(�������vI�������!�0���W�_�����vJ��������m�M�
�+r�� ���m��2���{l��"v�I��*m
��������MwS�j�g��������WZ��eS���t�����)�a�.���.�z��H����������(M����Nz���N�������\�2�M_��C�]qPE��@�j����e���H0���o�G��d�7�i���@�jY���r�C�����M����&R�nJ>C���?�j�m<�3�r}�VS���>����q�m'�3�"�R����J��������@%*�';�'�wup*���v*P	�J�>��%����@%�xQ�,n�0�:�	@9,J������(���%hj��&m��2����Y�i��
�+���Vw�V��}x|��Z�A~���(������Q>���r����:�����
�+���Uq���`�����o��
.���
�l}��K��w;,�d�K���X�A%*P	>�>������=T��@}��K��w�6���J���w����e:[���r�F���*A�g���<&Y�p��1K�VHKe( ���0@�h+��B�+��
i-Q+��M�g�����S�1��b#�E��bC<ZC�B;�Fh���Rl@�-�@��6�S	���^�Tm��E(Qj�I%x#��Ap��3�#Y��bSzC]�,��Z�I�h[�����J�^RW���W7������*��B����v�QKa�����|g�!<%�N�'���#��t��Yv�n@����n@�aH7-�
:�fo���#��hl�@��FB�mj��l��y�d��m��l���)�6�c�q[>�#)�6���,��n;��Q�r+�9����6�l��%7@�NhxR��,�k������m+Q����(d}�j��v@��.�;�e��\ag�~7��ZT6K(��GR�q��#6vm@��,�
�6�u^���m~����N�ry����{~z���O�L�h��<���j3����/	z�?��l�V��)A_�g���q6�'h�/��<���MY
-Q�<��d�V�K����l����l�K\l�^��sRoB>�mbr�Hp��e�7���w��P�|�������7�h�.oFx��?��g�~���~���W3^�7���^��t���B�����?\���M.��D?������Q���^��x4V����L�H��������7�������W�(�����:���|��W�J��������[\������[L�pe�����F����m�)Y$�x�����,YH��:9?�|� ��S�������E������U���&�5~+�a�=���L�&����	W_��Yt�Q,"_�������/:i���l������������|��u��EU�����$�����/_v�n��m,�@���u%�9��<�e��aUd�?�r�=�H��F�TF<)��
������:E���	����r��VV�^����M�"���@�i�������4���5���oI:�P@���H$
�x!�)���\z��r9��=)���y'l�24O7h�����������yr���U�����}RG���������a��1t*P{��@���S����MY�H\9Ma�^�(zO����7Do���1{�#��<wM��"W���Pv_�K����<Ie���L*�*,n�<^Y%sM�f�[ ��[�
����N���_ �C��LsP�Q^�wI<�@&���2�����N���'l��w���UJ�K�������S���b��9_d4��C���WrK�lId�If�|��4�L�M
��00�7k@e�k@?
8KA]�}��������w����.����H�F��'y��p���u>%����>�����%^���� ���������[5t��]C��r����W��lq�)n`�,��3����?�n�R�jv�i]x���h����T��R?����L5x��z2�Z>�r�yn[��������nHfc08�����k���QY%Go�>�q�O�[\�S�u}��.!E�h�K �%���gV����C�#�)�S
��a�,�o�d��N�s��P��9�^�����Z���{�xB�dN����-F,��i�?�jG�o�����eN�5�L���E���������G-��6��M�jw��d�6�:J+����������CM�����m�������������Q�R\,��A�����x�(c���r�i�D���[k��Jr�-2E�L.	e�c�ys~�>3h���l��o��o��o�������H�.R~�E��C����,�'�Y9���d7e9�u�ZG�~"��`���,g(�9�BN��������b�WU�tS�n��n���t�����L����=������Y��`49��tX�Y�����J6&v���<��n#�����Ne,w7���K����j�
vH0���:�H'��������tDU=����Yp�l1`=r��C&�T���`9��[�~�����]0�/����x]3�k�u�`���6�e��g&�
�g��&~zZ%O�&�1��__R:���S�gvwZ�%;�e\������GRo>����kla�`H�|�xD��g2���J	N��	�;!�:�=�NB��/7Zx���._���q��*>�������YJ}?x�����?;�����G������l�_�*��n�Rx9����\rLM>��x�����'�z��>)'
��yw�"hE�b��kF��^���h����ML$���d�I��r�]�-\j���W
��@�|4��B�L�,��kc/iX�:���E.,"�L�����Q3q��bd�I2�&�HO��g L����l�j{����A
Pd�aEt�FzG���"?i�d�x'cq���bx��F��-N���?�����0�U,vb}���b���������'bi}��>�1T�?^.��}��8���/����������=*�����>jh��e�8q��3�!�$_D�����h�������3�!�i>Z��������|���_�__��z���������U��wd��a��
�+��J��X8�X��_0Or�Q�,�`��X� ��v���\�c�_����O~Kw.8�Vh�I�P�H����j�Qh���:G�vS����x)�
h�C�V@#��zvm�rh{Y����F�Sf2����]�M���l[��a�H��5l����
i���{��HdmY��O+����Iu�p� �P 8m:(F@QmQE��(�Z������������8go��������.8d��.�P���P?��x����C�#P��/�����6�KP�%Ld��,�`�;��v6�uU�:b��gk�n�� �8�w	�t�=F���>�8S�8�baSe��C��7f���ES�����\a�ww�
���SWX�����:�����ic�f��_�g�����g����*QL�9Z��s��>Q�`M2��P������Aqb�p;���qFP��
�]c��M���0��5�Yx��,���5���������/��i�I�>C����j�E��3�"4I\���>�b��q����3��DI��I��J�:�)	��� ���g�A��8l
\�`Y��h��V
t��a���
���������]/q�����f�[�Q
<�mx��������[�>k�xQ��	��4�=P�v���E��]mn���5�nu��s�����j=E�m�aE�
���W0Q�s�_�#8p�R�P
�mx�p��� z�`�#P�@��Pd��.AQ��������^�J���TD��
��
u�0�!���AC�����z������e����B��h��m�;�G}�'���C��a�w:}����=��t���E�]m�!t�����{��_���e��#���.xd���Q���@��=���j������/_��E��-k4xY��y�$ @�1��.�u;����A�VHKe�0��� a�h+��B�+��
i-Q+�MG�g���M!��~2��Q��I�4i
v��`xd�#;u��4�W�q�j����B82)��L�*�&���U-d��f�M����x��W����YD!v�#D�Z`z�8Z t�hZ�m���g�#+`�z����������w�m+4�m�4P�Vi� ��R�8�Uf�:��,����
U���m�i���]G-����Bdw�T�wt���Xy���P�F.�������$FT�A�7��>�� ��T�>�� ������}�A����(�H�]K�Ng�9-/I`�]+��� �Kja�}�����_�@�a��2�t���;
�~+����@�? �����������
Q1��?leS-H�����_[$~��A����*A�W�$�v;
�����'������������V(��������>��y!���*Yd}��A�WW�>�� ��+��iad}Q�����t;���E6�?T������?�2���l�/��<���
��I���M����5���6�h�����i���j3#W��l���	�b�A����l:O�����br{������/������/������r�k�fr���A<���AxUY��C^�����������9���Uz=����k48>�+���u�t�a���-.������-��pU���f�F������S�HV�&�['�Y�G��y�!�F?�4;��p�����o��,�j}���b�K���e��G��d�~��#��e������w�D��00���������~�P�C���������Nk���jE�d,�������VL�*
*U>Z����P�#Z60�C�����3F?-�<�?��s��=����p�h�<n��u�}�#�H0�����'�8f7��*Y�'u.�������(����:��/����n��h"��$K��
)rI����Joa\<J�an�\!�+ �.W@r\.����\>��s�<���ry$�������(��/�%}������F�F��*iR�A)L����|��1�qs;��a����i]�I���4��#:\�9� JR��=� �U���z���y�?�#�S���x|��E�y�'�<��_������<����_F�{n2���:�C������S)���}�����`4��M�%�������f���^N�
�B�������Y:��T����'#���)��'���<Dx����nHfc08����RU���/%������%Y���'V��hC)OJ�F�s��Sh^�$�]F�W�����R���j������Pe6 ����1e*�r�������22"��h�~*�*�Z6�F4�Z����<;�VW6����R�U58��-*�|S������'O�e��@N�K��9_s�j������^(e�V!U��|�y����4 \��Y�$X��U}��K�<C��FN,�o��E�����%�
�\���s�b.O��	����[�'@z���������ty�Wt�S�;J
b&1]�|��Y�+?G�j��#O(���LyW��m%M�����GE�����k���Q��"���W�a:�B��(����[��F�?g����s�zJ**��|��X���W;,���"������������Z��}��Y��>g�pyQ�T�'\�g�i'�����Y�d�����p���U�o�%���%]����<�fw��]�9)#iW����I��x
2��|���O���7����~�������S�Vn'J���./��e(�j�f 8��?�SW]?<}>���,����yxvB�k3NAx5�������RJi�rr{q3����|���(s����'�z����������p�.���h�f���=nG+�5R�6Q�HW�z�<����3�$nZjr�^���%��=��Z%H���$��Y����T��+�@gK_2T&��.�����c�I��0�)H�o�[���_�J���)QR�R��){�/����?�\��P_UqZ"Y.ZW���*�8]�,����EUM��I����i��R�^J���V�E���j�:���y�����3?���]3Y���vi��mg��/���)��J.���_0L���ygh8C�j��\��
�������RW'���������Q�6���aA��0��&}����v��d�bD�y3���9�84�q��VB��Co��mX&���}e�e����j��!��q��H���q���H�uA ��J�;$;�J�%���MR
x��U7�H���*��K�M"
xHu]R�dg���
7���������5tr^/���M�)Y6vr��D?9r��8�+G�M������]�����.����^���|s�G��c]+WA�$���)-o���`�u`\M6e7�����_��s��D;�g�g�����D��!Q��^�l�)wu��?q�����K�R����.��Ky�R��T5���Y��$�K�K	�� p��dSvxW���
o��	���h�>$
�S����:0q���K�M�
`^��8�9�x����6�T�kR�'��mJ
|��@�5���"�?�s�T����O�^H�U7�T�����Rs/�e�&��R�m�
���#8J���rb�D��!Q p�A{����@�.M6e7�w������v�� p�	�6�C�@ p��	��T����t��7��b�/�U�6�x����a?�n��&%m��T�����'���"Z*���������c�dwTI]U��-��H��6����G]�M��a��f'���������A�e�
���
����j���,��1&�����
������
�4�Z���nYN@��������jeV�T-�`d�JcVU�������������
��vd����M�Dy0`"�De&���
��&�+��+�x 0��(q{A+%^��K� @"�D���Hl��~����B������G��������
R[���h]�" E@���)6�f���[�"�G�[�'���6�;��D�����0"`D�J#v��2
�#�ts������"���{~z���O�L���9^mfd?N����B��z���Cy��M��y����*^3��;�W^���}����v}���t�yF���=$���|�t��?K�yop�W\�%��?'�������~�%~��<A����st��,?�\Ln/��$�b4[lFx�?��K���WC�������G�^������vp?�AR��g��{�������mr�����\�����������'�NQ�oV[H�������@R�O&�^���>]���:o�����Fh����S��-��x�����w~����Rj��,�U�I���j������!.N�cZ]]��=��1���jr��'�II��������w�_6/��L���������KTb53��j�2�L���wC�
M��n��J����F��(�~G�F�����[�K����Y�������/�5E�f��t�|�����5?�5��f\U���JaEaQ	�(	��0�Ui�hM�;���
���{<�4::A>�K����b���I���/N�o~)J����1�EA����D���$5����/�^�����
�W��_��a��
i�iy�n��X������
����!�`	��&;�qU��?�E��3!�R*<%�$qA������<).�/���9�)/���3zA����D���S^�'g>� m��IpzB/���?	��^�z���tL/H����s6��'���h<#�L��2�'��d@���!��K2��}���x=�����E1�=#��c`�$��	r�z,��	����H8;.k�Ol��,��E,_7�00�Z����yr���U�����}R�BFDG�3�����7��u�1������M�L���!������*c;�����ce��t��5�41���y����d�l��n�,���������-4\y��e�>����w�����0�!)�\���!�:��K��+zy>$�����������<�|�^��S�2}z���S���$O1���~���L���e�>e.�����}Z^
|m��a�#���O���S���(������eebcC�K
���-�Q1��l�L�,f;NUV�Y�2f�����h�t��N�Tk��z�������c��c���r�����,R���N�@s�_���8�Cc;��ed����_n���j�l��5������|2��wm����j���i?T��������C�n�\?��G�j��<�O���|�K�Z���O?�~���+ ��A���m���_F��rY�u�����X�,�Q�EC��
l����s�O��D�����!����,��7�-���~cJ����"e��.f�������4��4�9d23����R8TSL�8�Y������_�����E�:����{��@�f���u�� Y6���+��N[*Z;���:E�d��%-���)�O�o8E��u���h��(�LQ�6E�d���SNQ�:E��_M���D�90��a&-1)C3����_~�H��_
Bk����N�]'��~S��i�M�q�|)`���d������f�Y���Kn�}�t��B�6�	Mi��	�o����m�����U��$�r����0l)���%\k�`���Xl�����f'�:X��^P*h������`|s{�B\��J"��/<
��P�1P���O��O���N������
��"�]"��MD�^Q�}"������Q���n�?"�qi����K�
`�Z�:!~����������ab�����q���������n�R�n�R�n�R�v��Q*�!���M7j�\�z?�'�zsGbo(�u/'�7��B���H'���Y�r1M��'��C�S�F�%��P9$��<�6����"��=
qs?����p���qOX���	[O6�dall���$�%1%�����*y�����_W���e�<���b��v���W��5�F�[�s���4�}�e�f�����,�w(��2���w����8�>)y���6�S� [���f��5a	��:��e@8
<�z~�l������a����(pj����j�����F����
3oj?%Y�O�n����g��]�3��F�R��y�1?�fXz�e:��#���7OJO�L���1?��Y�=Fuf���SU�-^����I�_�__�=��s�H�{�?��q!�[5b�(�������[��y�]�C(V�V�.[A���>����o�/�-���%Jg��S��O��;OV�H�e7������0����<b����y�M�6�O3q�������:���`C��)
:�?�~��O���:BU!���\�6����-�����Hc��+ ��r����\��?YC���O�\A��@�w�TKm�%]e��l��	;�G���� �zg�ms�����
q���qY�v�A&���x�J[K:��6]ZO3����/�v;��8Vtk�s1�t?��bh$�5Y�|1�-/������s������:����Mz[~���$q��&�m}f�����Gc>��H�����$������a����z�=��T���sU�y����?I%K��������U3���i��{O�-����SU\v�K�Y�4|�����������e������x[*���K+%wO���c���L��� ��I�*$��2-�<E���U!�=�i*���+�O�t��
��q���RH= �(���.��S���z:��RH=�7%�l�0�ys�bi�@�����=�z�\������{L]~����1G���x��d�\�=4�.�������8Rct���Y��*�%���%�f��~e�LR]�LG��Q�_��5s�M�1�~���WN+�.#D�V�B�ABPN���1�M��]v�'��u�G�����aK�������[~���/Yor�		Mm��V���z����u�[�����cK��XXe�j�
���@�CXl��)��R���D�`~�w%���Ja�������12*��"kP��4*�����%�t[s.ix��'����L�@1k.Y����z*�z��y|YV�c�K�����'>#~�8�g�J�$�;���Y9A��'g�y�S�21�SFR��>#��y_�S��d�}�NA�3�_�w@:���=3��5*�a�{x�7��x �}�>,��gB��,��n*�53 �9�vMk������3�3m�@<�I!�e'?��g>�g���xv���Y�g�hVd��ee6�HVf��cE6�(Vf��ae6�Vd��_e6��Uf��]e6��Ud��[e6��Uf��YE6���g��x����@;t-_��@��i����-w�"�8��u�G���0_�!y"��c�����#nk���1_g�`�J����(���[��������f�:Y�7?_o~���K�J1�S��!����v1���)���0U�a����.�[r��3��.�������c�J0�����^�K���Y����Y��Y�ot"j�<�7���Jn��bV�5��47���}�X�Qk������Yr�+�e'��
nr��ub�.''���d�6}NVf����l�������
l59yM����9YiM����99Y�&'#���d$5}NVP�������H�d�b�Q�b�E�'&L9�����P[3�Ce���W��%�4�Q0
G���]�kA������Ie��t1�2�\O��G���I�/��
K�#��)�$>:��MV���/%���6�����}�C������e��O�f�!.�
��xj�5������KE�����r����W��d�����}f9���\s@��r6���X�.���^�-�U��p�9�.�@��-�b��s�,����k�wY�&�],gsM���s��������@4`�X,���,�����e���f�"�l'f�N����e;�g��T��fx+�y]��q�|3u�j�-�V3n>O�l���8=#��� ���/b��1l)�.y^�-�7W�Z�X�_j�3���/��r,�6��h��������U����n��g��m(/X�d��F!�����d�o�U��u��ZL��MZO�����O\(�8�h�J�2����j�n2y�knd=yf���?y�|�����W����2j�T��w�y�����"O!��
���TNA�8x%��R�Q�NK�JEO�D�2=���`|����u�#���QT��BlG��2P���WH�����b$n�1�c�����xw'}G3��mh$	�N�
yw�}GC�������(���b)����X
w��n]�}�/���a��1�1^'kz\�b�(GN��y�����b�05X�N��5Y��R���i�����h�g�9Y'�i�6KD�@��T��cr�������,�����i�@}~�2r6����70�pgp�"$��\M��-=��?��9�H��Tz"��&�J(E����|dv���5���Z��~E�+dl�n0A5.�U=e��eT���NP���98���
(��0U��9>d,h�Q=2��<}������\����wr*3XJ2V]'�HvJ��2����Hj�d<��B%S�I]��2s�$c��A�1��D%���'=�g\�20��W3�����d\sh�YtL�d>��|j���&39��3<�G O�
LE��*n��<���cG�D��@��H�	��@��4���-''dy�J���e2D�$!.�T�,O��*���AE�|�GuR
��)�.�V�����m�/�4����T��N
*�e����^M�Q5K���\G��6i%��z�UQ�VE'mUt�VEgmUt���X.�o<��4����-M�C����j�!�!8�$8Z!4J}��"����{9�N�!��O�T�������.#'���Ea��5��0�"��� ;�d%����~�a4�a�FE�&�a4����[�=�=�=!��m��M�U�-�����{G�$[%�d�"�d��d�d[�d�%�{�QG���G�M�G���G���G�-�G���G�m�#�TX%�W+�|���MB��g��qjP@e�
D����zDP�s������-���5�����&����k!��l��Pk��S�
�=I'��C������!�.8�������8oQ��T�y��p_�t����B�?�|-��F(,��+�W��@X�����Hq���b���~ji?k�jl����Na����)k^v�e�
d��]$>��R�x�	kLO����H�j����'�oJ8�w��3
��������]mH9��(�,��dW��]��_��e6��o�j�@�v�M�-~�
~�|�9���7�f����������4 ���4�"�V�������T�FZ��J,��J4���J<��JD���JL��JT���J\,���I���x�V�c�J��.?�����~�*{$K������fry������P�[2uP�������"��$^o(9H	@z�[��PD���qIT��:k�
���z���WQ[���@��"�����$���������E����5���41.���i�E}4���m����Vc!�&+�4��34��!�+�=��^SQ3�9�XsI�2��~c{��m��Xm�ir��-9L���?���XgJ�&~E������l�m�u�9L���;���X_K�&V���@��0���O�F����O�OYT��$����=���B)�2H��P
��#!~���H� �9"=H�����#!&���H��J���VBJ���gB5������D�P
o%L�jx+�+�����L}�Ez�D�&���t��8�;��+�c���L�$��,�y��2���L��&h(�������V�g���*��{Rq�P~������JP��:�G�0����=7�����jGV��4�������x��!i7
����cK������_p:�G���N@���v���tF�vc��H������Wc��u�vc�#������V�p��A��<�z�R����DK�g�3��
V�*s�+*�����u���:�B�sQ�O���(��}�\n��h���Ee���ws8j���������I����|�9b'n�Bk�#��p�n|�{������G��}�����B�l-�v7�@�mP�6<�%q�9���$���y�5����~�{�+����f|�Ok����%�����M_����X
h�8�7_o9Uc�B����7��5��[X�Z[o651��)S���^v�_�~iq�%�[���*G_�rr'_��d��s��^��dB�ksr�^�s��^��Tl~rrG^�s�'^�s�^jsr�]��dd59�>'{��6g�xB���t��cja;�n���X����L(�Qe��;f�
��M+��G'�
{L�W=&����E �J_}v����	BQv{Lz�c��T�.g���e [�?���\
����h�����.������]�Au9��{����p9WC��{*ZZ!�i����X
B�i���@��o,�����9�������������N;>�p�����ee�yqW�U8�l�lwc����fcf�����}}|V�,�1��������������\�S�X5�pB�����*��������\VV��!�YY����U��Ve���f�B�l�Y��l�f���������
��V3���[��j�����e��b��Y6a�8=4�-Op"t�-N����Dh8N����]�����O�~�2�-�
�-Gn��d��������W����$����yr��J�T��j:W��b��:�Jr���9j�5Z6��M���sB6�G�wT���hX���\=���QEC=�3Fp�a7�������d�D�V�}���d�F��c<R�nyvS�Vq���Vg��������
035�C�������%����x>O�.�9�������D�D�0�U�D�W�]����;�`��:h�B��y47�� ��������+��S��?_]�u�"�������-(:�+"wD�vd���g�PD�(SD�`����A�z�����7�7�Qy��-��/o������n����F���qyc��9�J��J�g����g��8p�e���g���1���� �z$�\[�Y��f��0`�e��f���1�����b�}-��$�DPl��� .@\���G\�g�����Sq�Q�$�-�[��L��x��^�[���������X(�.��5�� @�,Y����Y�a���~�nED��ez6�� 4��5������������!�1;c�N�)�z��$'z����p!-�a^ P��!-���@��6���FZ:���6(����W!�n��M3F���*�6i�p=��6o�s��
2�v���������;`��j�m~����{O:�s������&�����.�wV9��(����r�T�#l��H��BAd 4�v�C+)�!�s���]hL����jW�D11�����"�]S�f�l��j��mQ@����[]�@�J�N�u��~����Mi�������4�Y�P����AL����g�i�4���F[����!�;����VQ��
N��Q�uMq����/���BM�50���[�/�m4P+���_j�h`U&6my����p�L���~���Qf�@���R{G��2�u���F���e��K-�V��-���?]�����kw�����p������:����Y�����C���%���]���������s"r��5_/��������|�>6�����h�@����6��e
��C�>���W���7���j�_�=w�C.�wy�xgJ>{oS�$��j�P���]��������aS�Z�F�]���4���,���~�&W�Zv����[�X������t+:L���S`�BW��Lo�@�'���j��NK��5g�����4���6P��'*��������xt���T�
�*�BIjI���EI[����jIoJS�(iM*M}��N�U����������M)1;W�[���^^��c%�.<A���5��K��T�>��-�1���#h0���4�h0��Ot+���5	�K�(/M��Km����|/{������Z
������Sp����\/5i?�c76A�fs#&:Vz�����}����	��`i�d�K6��%������%Z����
s�����i��6P���'R
N��N�h�i�z���i������>�jwZ&�
�4����'&�~@��/:��n�A���\^�s��p�.����4����v�H����$s�U�>��������>����l����u�)��_'?/���t��S�bV��Y��=$h��V�<=m����o.&����A�f��/���}�W���j�~���~����3��\~�t;�� ���e�����7�o��_/>!����,��F���?�	�ST��U�jN��)�9K+�x���������3j8��^o�,��������w~����
)=|J�
���5&��z0�h���,�/����M�3���1izr5���S�`���7v_6�~�a�e�.�T3��JI�?]|��zy+����yf��.��j�����e���w����U�H?�N��%�~0��->\�n>��Z�+��	���7�_>!���>�D�9����VfQ��2��'j��ke��0������1g�	�Og�3��53��
w��N���}x�}�E_���Z�n��fQ#��q�^���S�����j��
M������	q�`����"�Z���V�Q/j�E-��e�@�I!g������v��t��r����<Y�'*����D���<g�&���'��~�(�g��x�����1#A�X�E��]7>9��:B�F��l���>/��1��V�c�J�Iaargt��p���e@����Y?nt&�e7��Z\�~UN?����?�y����v�`�J�{"!�[�8q�i��]������>��\�#BZ��4sI�����>,/�CLa��7$�K�SYz�O�����O/O����<���^��S�2����(}�\����3����py�|s�)7��s[Nj���B��$�z��Bh��g��88%�y�m�c!|�5d��������C�ee��]M�^��Z^���J���ZF�yS#k��t��u>o�U���i�nF:�D�l�Xp$!\��vM��aP1e��Tlk���-]�����>�����%^��D����n?^�?���� �A
I������\C~]��}!7����x�P�8�=��'��D��b�7${v���ry#�bcM���h�r�T������kRP��&�TR��,�	�H��{��NF�i��uQC��&L-y%����-*��t���~��%�Y4�_��u^���%u�)�r�O4��wP����DW���=��d��a:���E�(�+��_��4�W��#���e����d��oJ�<��W���}NV�k���� +�yR�\h:��wMG+ �\�0���TO�2�s��v
3���4�I��l���J���������g:F�k4WMvh�vP �pL�Wi��d�[O>��s f���8-�h<�Y��B�����D4}���hvI4��#������D�s$�[����hb�/6m���r��i$�7���Rvg��������:5����h��e�.���3"sO�s�?���=$�}�N���,����2_��u�8.f���o�_�v����j3O������0d
�R���7)<����xn��Vh�re��h����]<��|e.'�7��BGw�~JS�Zz1e�b�E@���'����~?�t�n��<�6���$a��p��k���^��=a]w�$=n�����]��H��I�X]*WGhpy1�t;`o��3�R6A$�>�E��������5���o��">�5a���S�95A����G=W���84���2�3o:�EK�&��,C��������Ovs
g��m5�8��0?�fXzd�yf�g~����� �3?���<+��2Q}�:��d���r��R��$5�1f��5�15�������E�?g�����p���v���R�[�i����Xn�f����~�H�og/I��=��u�[��e(�y���1R�`��Ux�7����u
�:����.��D��x8�o����}L|��N{p�g����|K6&�N
�t�3���y�-�����~��$�C���89t�����%]!cQ����tg�dn�c�����#]�,XMW��/������ZG���vb�����A��1��7c������s�{�9Z�;\��7�m�]��S�uSM����������|"�������$������aj��O���x[�*��\Uq^6<=�O��X����oK���U3���i��{O�-E�~��*.�u��%uv=
�{�m)B�#_UqZ��wZq��oKQ6�����Pr�t,�=�!u2y����K'������>�<E���U!�=���!@s��T	��9@s�����s4��7�q1�3����A]T����i�s�4p���)��^�T��h��$Y1�7U@c�������.
e�&i=U��p|b��i��\�<S5��J�����:�=�V�B��8V��k�_��b^Y�IyO���YL�5�~���ZN+�.#��)�.$D�a1�+Lk������Uv��3��8{����Q���/5�<������'5����u,������������O�(����Y���V^��<YV/GadMx>��^���,1�������;�����}s�YO|��8�g�
�����,k�������kprV�kp��"B��gD�0��t\C���a����>m���q
�������B���������y��R���
i/�m����l�^�m	l������`����`x����������e6-�-���l�Mb�lZ[d���2����������e6-X-�i�j�MR�l=�|���h�5B��N�i�h�M�F���8k��-Q�C����`�W,a�$�glS���U������,=�u�C��7Xk�8�u�jh�'r�!'�=t�,�d{��U�;w��-�T��u�����jku���?��%�l���5(xOt�����L[[�hG����
t��	������w�X��5����G-�����]}N���d��6'v�9Y�[���������d��6'�skr20�&'�r�9Y������	��#(�8Ehs1.@�`���9WF$���	�(�t����������`����nn������89��d��2.L��F������T^aT=b.+A�j��):�g�i"g�E�n�4Y�������8�������$:?�����Ou0S��I���:6���9��#�������L���ZS��/3��W�Z�&~�������h�A|��`�?��N~[��7��e��Ef�N����e;3�v����+�.[��V>��l5�V�����[��f��l��
��j�)y����=��5�0����N���T�R	��1�T�dJ�X1�,E�n��!�GjC���q
q��u��Pwl�5���}�e'� M�mcm�B���o1i�CH`=mf�����@:q�!(�n\SW3������y(��*�-�mc��V�J��H{"b�H28-�
���E�2��(/d�C����s�+�f(*�C��:�m���	�
q����R��\�P����4�:�.�;��]lKEt(|�v��������,qL�,����`����0�,����F�^,e�����W�z=[.�������&��U���!-��w���,>'�=-�f�H(^�*�{LJ��~���G�+��&�!��Y=A�J)��z�r�r��8��gI������K�
��b�#42�%������C��m�9�������;���Q>��9�f��\�DWBdy{x*�)�)���������UK������{,��Kj���jI�S��Z������5}*�YK2VML�������G,���2���b�A~��km2�F!M����6��l2����O�Nv��@���)O�|����#��$H�X}c�#�7G9��]�I���I��[��?�U�����'�5����~T���$Z��}-������
^���
*�qi��x���"�6l!K
�UN�eE�MEZ��V���*�������N�������Uc~��kU
�G'��r�V-A�/�M�e��c��c�Q�nkdM���k�2�)�������0p��w��������,���P���qm��0��;}Oh��c]��^�v��p��H����d��G2��#�����\��S��;��r����d��G2O�#��������n����lnc���������s�m�-��LB�x���&R��y�;�V)�&��1�0&��n�R�@65W��D�����q�� :�!�J�!�#!l!Wm�V����}a+=o����y�p��r�����_}��G*�W�_p5��j�����6�����W�E�-�/L�	�yqS�������W�v'��I�!�=��6�QI�������j�[�����z����vs�����Z�����jU���j��i���#�3�ap�� ��?Jg��G�(���(���(��)�)1�)Q�)q�U���cL,�9Z%��*Y�'zx�0[of��
�x��J����������������o<�V~������"��$^o�G�~�i@�r��$V���
H���:k�
	��z����Y���A���y~���<���|����0q`���ui�4���	�j���2y���sgn�i����d|�|�+����V����Q��������|jh���8M�6H��ll<��a(i�l�b�T�c��9Ae��7���'H�~�6I$���?�,�(I���K��#a7���H��$�9�=I���P��#a����H�6���^*��U6X�����J5n��X�q���R�[e��b�z���
�[!A�WL�)"��;~P	���=�P[��)��(B���oU"�
��)2L�Pn��������g�h����^�Z�G��i��U{n�'�
KV\����h�*9x���i��5j��`�u;A/�N�-���������qsU���������M�R"���i���W@���y�i�)*�����>����:���� k�o��o�>W��-��2kQ�k��p��:$U���C����0p��2��C-�18u�(��Cp��:��ip��$�(p��!8u��@=�Y8u�|N�S���!1�:T��U��r��� OpF�� }�-8�A>�pF����
o����3�_�Lg�xC�x���b��P�x@s��b��W"'"�^'��~C��TM������XA�����h��1��#�F�4#���>'���(��Q!�v�E�\=���Q���Q����3��_�
~�s�z�[���\�w�Z�_�1)L���9+��V�>���F��x��!��?`�`����r�F�e�L��:�Sm��x���������:�`=UX�#������
E2{o����}z�2����40��"�d�d��-�M�����K���7���AT� <�?���)#;)o��l~P��R�5.o`�88cT��V�������p#�F��H5�����!7���?;z$&��q�jk���I0#�P3�k����+����[��$B(I���	�R���d������)��p���%�)I��\I��+����]�}�J���}eG,n����+���C����-���P�}�B�W����e���s�cD��nn�.��}������w��MV|O7�T -�[i�������i�(/�2�_�y�������T��/��l���4�u[�.� �f�4j�b������T�i������U�1�k3�IG�d+�M�Z���h�8�AO�|�>4�l�6����EJ�\�#�1��b8|�Y���D�T���-�����N������6���Z.�d��;��
��<���/�,��2B�����_�
Y,����~)Kd���?���|���/�,��2G�����_�!Y����~)�d'���yye�bw�w�����K!�xY�����D��Hs	�4H$M{	�q� ����7&�4L2@�uMu �����I���d�$�"�pk��5+D���U�>NQE6�=�����j\��Vp�vF���9g��=�b����i�gr��,���ob�7�[E�����i��\��3c�=/;`�����m�tHrnc��"O:S�jcQ�c��d���9c;�M��H����S;���]i ��4�R]��=W����b��A� �����j�z�����j�%����0�Za�3������xf'� ����IA���e�nmg
�m�����cw�D�@��%6��3�Yw��-�����HSV��E3���?oK���K����-J�t��}�W���t�u�N���O�%z�����v%%���-�nn����g"!5��Z���D��B�~��a�k�����7���
��V8����W������v7�L}��N����X[�Ah_�6�%z��GC}�M��m8����D�%����?�\��n'�����w��\�x'�u|��]&�����/	zH�����F/Jy�|�|I�5��g�
��_�5��|�����r���1$A6�
qi={H����y��=:F���h@c|��f��=[<�O_7�q�B�x���Y�|���y���xs1��`&
b4[lF�KJ�qoW���j�~���~���?Z�
^~�t;�� ���2��Z����������c��#��w#�����;Em�Ym!�&�?'����Ni5g��<�������[������H�l�f���DiyGk@�=o����3.>��S{J�*�$w�d5K������5��)������m������'W���>�M�������&�w�4_6i��L&��M��~��0���V���KW[b53���>��M�n���J���F�O,�~d�M>3\rZ�h�������)�����O��F�P:��F�G��Y����J�Iq��R�E����Eh,���
�����j�`��/l�!���/�E\F�SjC���Z���k�Y�e����'X��B��4�R�n��X������
���f�!������;��0��`�b�ZV��0;6���[^TEW���R���ZU����y�F��1v���R�S��/�y��OT3�75�X}���x=�����E1�=#��c`�$��	r�z,��	����H8;.k�Ol��,��E�1�G��l���>/��1��V�c�J�I��82u�n����{�n��!"45�vf2e7����&,C��qrq]m��t��5�61�����Y�)���2+��)XV�1+��/a_[$h��4}�.}�[�U�24OG�aTCR��$�CzuB���!�W��|H2����3z����%y������e�������iyI�b�E/��)s�����(}�\����3����Z����#7�3_NyF����V�o��Y���T�����cnKf�^j�D�N��*+��X3Vw�hi�k���:�5���z�������c��c����^�
�,g�rV�u���(V1��cU}�,f���Lx�����fW�p6���Z���LDW
>�����x@\�}r���?��qT��Y�!��F����#A5�y�'��99��K�Z���O?�~���+ ��A���m���_F��rY�u�����X����'�����t=���&����!����_�2�F�iB6Y�$dS7�	5L����^��;*7�x���i�	�j�IG7���Y4\�+���X���(��������(k7��A�l��W����T�v�emu�|��KZ�)�%S�7�"�p�|�)�@E���;LQ �����)
$S4���p��)�;��E�Lp�����8_���� ��d8&���%"IS~59��N^f:��=�Q����;�����s���� +�yR`Y����=����uV������	+[6��'2�R�!:K���A�]�F"�������<��
���h%q:��^��3�9�;���9�7dv�1B� �	"� �	� �I�7'"��Dt�D�o��z�"�~�D����'��#��~ D��x�i�����M�����abS�W�Wlj����M;�������O6��w�����-S��mR��mR�~���r#JE6 �A���F�B��kU�G�$^o���\��{9����\�T��SZ��I���(��/9�>	����K�a���y���q�5fq��C�^3���=�	�=\5a��f�,����$�$�D�>yzZ%Ox��w��j������l\^L>��[t�*tK��Y�C��
��3��w�
���?�-�E���QOV���A���g�=���w��Vl<��'��������J�:�_��3�7��!�y���k��&
6b�t���o�yS�)���v+��w<[e���y57��z��;���7���,��ea��yRz�e
������7�J�1�3������"m�:��x�~9}��p��#y�����I�n��E����3[zn:Wl���v��bR�U�����X�W��'a��"9���$����"�m��2���� ���8/������K}�����2p���<��?���d�F������n����������nS1��Mjw�GI�����ymez���K� ��T���������.�p#A��S(��W��{���Nbl�0"):������(�L0v0X$�nA�Z��c+�g���w���������a%���U
��dU>!�Kp��O�U��o(T]l2�@]�z������KP9�%��(n~�2�����-)��e�������:K�j�X~1�"��,�^Q�����R�S{1����w>#���#!P��(�&������?pr>��M�OO&C��v/���l2�D�lt��cr�����L9�h|f+���N�)�b�Yt�� �M����v�BF���Ih&�\4	m�f��a!�A��Ih&�\`4�l�������Ih&��$��!X�H�
��!�����M�g�V�<�8Z<y��C�a�HB\1�<��h�<3�*��ag����s=���\����=���J�\��z	�����x�g������oD�����������$�x����
]85 Eo�7���qr;�0�������Ji�cwC��w@����~G����+?�FVL�.fV��<������z>g�l���M{��
���<_�	,�����W�,����Us<-UZY�e`��X]�&0@�]a�,����e�V����r}]V_k��l
�6gNR~Z_o_o��`�)0�g�]o�\���jR����V_o��u
�7gfU~l��?U�=���������5�;�35�$��h"��o�Lds���������&�X6@C�YZ�i�0�T��a$y�3A9g�����u��Ilo@�F��^F�e�N#���
8�A'�=�� �Qx*�`�N�hz*�`�N�h&���$N�G�>�m�`I������q$q~c�p
�����d(q��$�����`�3��9�`��:����{F�r��g5���U�vAm�f���|w�,��Yn�<{F{����g�g�r���GA`u�=#�fX>[&�rY2���e`����e�X.����_,��`�l��e�2�|vK���Z,�����Y,�A�J�SC:3j���O
.�>������|W#LHe�����rj�t���4��e:�Q���F��XI�F��8cJ8����h>���Y�
��.�[9��l�E|�E��['5*���|�)��3^�^���=�{1k��*�����snEO�9�P�1�3�*������vldcLx��/�$�B7fEvc%�m��,�����������u�s��"������X������X�B~�`������U�sse�J�m*�����A*\[����C�<[����B*[>���@J�Z.����C��Z>����B*�Z���@J�Z>����B��O�;\$��.��}4�.��Wr�H��@��(�d��D�.�U%eKr���h0L���_��>�?�&-�>q4�x��)ET���+x�����E&��8�^s|������2�U!5��$Y��Djn�����m�C��R�X��m�����.��:{��E���[�L�,lscUg9G�r����x��9��s�/��`�U��	���\�qy��������G�r��y���K
��/����G�r.\^e9�H�����%y�[�u��?���=��h�/`�l��@���2�l���Fn`c7����
l�v�v��+��`x�}^V0n��)+7V0n���"�>>&����Y���+5P��OM:���8��/6"o����#(fv��������r�vd4��w#����=��HT������_(qy1[C�~i��^�6���y��>���6���O�����P����V�c���������������������#]��1y�k(��u�t�U�H��t�'�u2�&0��T��O	vd
��r��\�<�h&�T�*�G`�c�����2~,��R�&QD:�LD)�]����2���Ef\��r��$���1������>N�Lx9�:5����D�1Be�����Y��!��}O}����S�R��i�i)��:�u�u�����Lk�L;>���$��RW�����v�I�t�^�����?���`C4G��$�]�k�>m�L�&i|\�u�������'��;K��")���
����A��4@Z~�+%�b (
����7�H
Q$'��&RF�����2r��b���U��v�����^��r���1A.���2^�eT��!O�����9*/�T�S9TQ?9}(��)�z ��}G&-�3��R}���t]X���D4�B�p*q]�N��Pp^���U����t
��V�1x%
�?�O��fVn��lF�Qg���q`S1!xZxV��0��:�1�K@X���UY��"��@<����<7P�������
xn�s{���q��	��? ��;��!�8	���_Pi3�����c�5
R��hR��� L���D��Fn�T��F�T��FA*oT� �w�Q��[�l�����t����J�)(��k�E��
�4U����fMt�TAg����R��H��T]2R�4���q3j���������h��6y�)pR���,+�B�������]!��]*NJ"%���\�k�ha�B����������a�i�*��lanF�������������6�A������R��M���d��{�J� ��d���{�Ic2^���P=���� s�d8��L{�1`2����G_`�S�&V�����u\�4�g��q
�m�2�`/�Q�)�b��
k��t&H��QB����"�����B���� [�0�9��lt��MR�s
��,A��u?�*�$�81����R#2�����G=�G�J�H7<��F���O��6�u��Y�%]�WFaY���g4�%]of<nI'7������NJ
\��y�����5o:;�E��	��=�&��&�2�!�'������SP�h��/�L�M�r�������)�";����R��)��;�+<��~��q�0���#j�u�����J#��2�#��[��k���z����v?�j�Nw�:�>�3�K�:+E
����ZjS#�Ha+-l%��������V��J[Ib+Ml%��T��,���0a�9NZ����`��$�d5O����E�]���@��r�7�������x��Y �%����;g�	���,�8�Rt@	'#�!2o�s�{LSVe��4=A5���K��.A7K���$L�}]�yu���/'�a�V|4�����TPm�SO��U��@�(�Y�Inre1�=$���v������!U+O�T^
S����\�,�����E�a{��m�~Q�����F[�0��jO����>9�+������Z�Ms^��
��p��4;�k����B�id�p����O������i'�U�|-�?��9`2f$����W
�O�������O�������O��	��G����G��Ws)a^���mx5����<R��WsSa^�w�ex;��"�.��{&� o�Go, �}������c�O	5ms�l��+S��1
2�����-#����L������B<���^e�@!(U�t��@q���s�������^-�����^)��e}T���)�P���<mY��s�\e�*E��5�+��ub`����#&�0�K���-g�Wp�u��q�W��|��F��6����~�����Q���9kI�$��M��z��y��^Qn��K��u�=Ci(��T���TcTJ��<���k,�V��F�G������\�yt��<B�&����G�EM�p`]9�{
���g�Y{C�7������$�Z�vww�@��N���P�d+�*%9�Ft�k�1n�7�,��~�+���}�0D%.}[mWY���z������NI�����c5�R����JZCT������p�f>vn�Ug���^��_��,�R��R����y�J��H�Ax>���Rry��D�����^@Z�H%�e>��2Rx����,��.�
 �;�|H9�e.��b��R���O�(�5Cq�m��7^I�����Jq���J=�����&=���W��j�����H��q�}��(�
���X��*��s�/g����jq9��8�-5�����v�s�cU��R^�#}9C���-�Z\���R{K��V�����\g��c0��'�+���xg>�#oX�����6u2�����v*��l(��)�23oZ���F��d�&�Ffd��d�����i���������:2����@a�M�@tf�U�M����V�*���j���7-P��7�eP�����*��6��
l�6v���M��fn`�n`g�`�����
���b+7m���[fo�f��"�����G�.�K��}Dh�G���}D��G�-l'"4������bo��N\�V�&�W��C���
��!-�]�,���
A���H:n6��l��2I!\�-uE�����Q@�h6�|�`�=����J6^`	YH��5���
��8g
�9�
���x����l>R�[o~M6i��2F#��V��n�0i�r�"��r��V����
B��/��Wm�����Y���7�r�,�x��eW��~��J�f_'�F�#�p����|A[Z��]��w��� k���c��+�x�+"�+"���VVD-�9+���G��<�+�+"*Y?�YhR�Q8&�D��E&��q(Exz*��F�����4	3B�ME1��F"aB���H@�`�L&���!��;#���<���,Ogy:��Y%K�t���
�m��j�4���
V�������,Ofy2��Y��*Y�'�<�UXn�dV[�����$y_�2-@��[������<��).Oq�������?V��9���*=���-Oo����-Oo���-Oo���j�,4��L�6Ki��=���,Ody"�Y���D��}"k?��������"��5���&gD
�Vue:PMZ���~���{���<c�R��z=R3�@DO����j�#�a�3��5Lh�1�0��<3���w����csfQ��d����u�)���L5N��yU�Z�S���^���#?�P�z3*�����U�I7��+h�{�
�)�z���b�i/�����;�\�����y����2M����D��T�t:S0q`
ja;��q).�&�����M�����azS*�b��	����|������!��h���
�fM����t�M�:�tX����+�)�&����%zS*	��8�9����j�C8�8�����q���R^�y���r����$D�8��)����W��d�:��}�f�B�F���n�t!���U%��"]H��;Z���H�:�eVr�+����z����t!��nm%�"]H���Z���H�?�]V�S�����
����������L�2>������%#-�������>�/�{����jM	�*!9/��b�N�=/�k��;�U��iN�W
�y	���u�y	_�������B�!��W��%{OJ��,���N;��T	,�f���|�/x����7�~����U�Ug����M���4R4�<�J��%��I��'�S�^��@���
J,�S���B�����0��$���V�����5d!�9��6Es����9E�v��s8����>]�����y"��M+c�Z"�b�C��5���9Ma�F$��Y
�M��9LZC�J�.a�J�U�<
��#��B�'%�l]<_ze����7^���s��^�)��b��&y1�����1���w�L#��`�4�K0s�.!��H0����^9����Y^x��t	��Dx�u/;-��R�'%�4�]�[Y�^��#RK�z������u�F��1�t�Z�(��[;k�1M�uD������/���_�>�^6�B�V���0wm��7�aDZCK�.�R���?^	38t���)t����
3wm�7�atZC0K�.�S���E?^�y(2�����9�8���+2�/����H��u|��>���j��
���vAC����� ]�%����&�$�m�\���S�|H����b~�,�Kt_}	����a�E�� ����#u���^�	�F�H��e[���j�� D�$�6����]XR��j���A��^|�����+i
�~,V��C�g������������/^���v���.����@���~�����}���o~y�!�b��x]?���������-fl+f����ys�2�~����T��;��a�� X@k���%�>~�A����P}��G�~LV���):&��?�!�\��h��������7�����������Q������JODMX��sj+����>Q�1����������v����#����#xE&�A���#	�K��
�I�T�}���{�-�����i��i�*LSO����]NS�7M��	�K��Fs�bQwK�jR����^�f�(�3�F�Ma��`�@��^v���T�p1u�E�Z��)�4t��)%vtd���nN�����T�4�s��p\�����e�'��Q8N�����7v����(��("�0�5���� 2�����(i��f�h�h���:^&�<�SF|xt�T*��p\����7�>|�*�x�3������FB� & �6���JVG���DX��-���?�|�^&�.�$Dv0O���ay�����W6*�<����������^K��7�L�-^�7�a���C(��1M���D�������+�zzJ_C�U��������
�+�	��J���>�_�+ix4���Uz5���^����)�	3����h��+�Qt�Ys	�����\��@y5����G���k��I�"�zi���a�ls��%f�����2�8i��V`�h�����m\�I��B|;�)�-F��J��E��Co���2�'�}�F���M��T�����[��X.l���P	�o?
���:R����t]���8CU���04*	��l��cp��Q�Xs� *arD���G�i��HC����|H2l��|�����@�:O�~O�(�AV�t(dUXu��k^�Q<�Y!4�,B�f�x�.�x=P�H�STS��2KA���,g��m��>Xk�=����5g9�0����#C:�l�N5�$��d�dao��H���o������n�G��2�o���To6�����|�l� �m�n�����!2��<�+-6@�z����/FU�)��(5��������X9R6�"tQ�/I1V+2W�^d.��?���I>�[�������=�1y�s��a{�9�G\�����O>��%V�#=V�z%:� �
�A���!�0�l����+a��aC�)���Q��2jSFS�[EL���aEL���~�Q��j���U�wLe��j�O����x5�Y�����EE}Ue��8�EWD�M+Qqmo&�4���FB6p3�~�����v�'�<?��4��'x*9E�M	������<�����`��F#Ru��L�KW��L������2���e�\����_�z�&R��$3����3�E/#j�K���hd/��/q�?��G�%�a�{t��t���'�%���~���B���f���+�S�J���!�7��?\��$:h�ws6��e.}�m�+�S�S��{g����G.��^E��������*r�b=��N�<D������~(���&��zKJ:qiI?���zzG�����G���x�P��!���+1�f���	�������������`�LD���gB��R�6���k���Kb�?_/��U���L�
���&����SY-f�(��v��/6Mux�h�\$J�D4�lo��H�P� d
e��gi���+��J6�6l��P���u��]I�#k_�\W�h�d� �p��u{;�������,��Pah��0�������b��*j�=�q�{�?}��?>�?�R9�stc=�I��d�<p����_�O�$sB���d�a{|�	���w����������f���dy�������9mi��9m��D���z�j]�
��j9��95$��B�Q���EJ�;2|bc��U�����q`s������
Je����hf���B
��;��JZ�����������`4��L�������Y8����x4��FS�GE��Id&�X4>�<=�L'��	)8<	�d����g����,�E'��r��$�<<UM
���f2�+�����T4�@2�oF��Rp4U�p6�������R�����P2���O=.DP�xz��������G==���cO�==�Mzt�Z;+^��w~���S-�c����u���Q��Sd���bz����#�Y��h�l.�������I��(V+����������:$3���$���_�:��6��$�|��
��NM���������^	k�imT�����p�Q���q���d�[���#|���M��T%����1RC#���
��&)�O�.F�~A�����LA���F��D:d8	�"�$MBN�F��
r� qh����h(���x��a$s�4�H<�w��Q�?"���Qx*F~�k�F~4=#?�s��h&9H�$�Z�G�=���F~����'�g@�fb���vh����u�d(Q�:�$#�����������H�)p���������i�+Iy����pOR�Y��&)[���&)��-�����H�*��Py$���#3�\�Q@���*�d��r�E�G*
�<2Q@���T.y(��HC�GfP�$!�� 9�����d������=�������OCV���X��wT7b�r��t��]r���S�h�;�%���_�A��6���v���$dC���*�^Ul��X?�#��������{�����VJ
Z%�+���!��'�C]4�����'�Rd��A��xdc}
(x/.�]k#Z�1\jo��v��1j�9��A��;��Hj:����Gf���D�N!1s�$3N&/s��2.T����d�2N!+s�$�2N&)��d�2N!'s�$b2N&%��dB2n'd����S���@���'BR�H�Y!�s�[2�D�M������Ka0�����4��0z��h�de$ ^4�O)����e@=����,&������yL)��5��T'�����Te�����6���iQ[Q`
c����
��@��[r+h����f^�R�����R�U^XY��|���t���N���~tV����6?�UIu��2
���R��\6B�V�r��:�����y��&��c)4�uW����Mf��[hB�<�B�Z�q��:�����y|�&��c54�u�����Mv��sh���#P?���'��"7����
l�v�v���"u�o��sX��j���`x9X��F*f+3��?�(N�>$w�c�p�_�.S%�k�#��xd6�D��0g�U$$`�����Zu�.Si��P��2���&<b:F#)�.�%�R�[x[��W)�;���C��l>&��&L�~�����2EG�Ik{�5z�:��+��&Dj����>�z%^�kB����1-�F��v��g)9A"������&������nH@V��X�r\2��Q
�Dh���t�SAMS�&��3���&k1�	��rc�vJ����)���X�����An�@�����Yf_mB-pl:����r9�����r��#3O�<��L%��;�A�$�������������E-R���n}Z��hJJ���u@��B�x����	;m�-M�9�<�R�V9>���$�AV��I��v�I�t�^��b�W�|H6_�
����$�]�k�>m���&i|\�u�������u>M�>�q_{t=H���������oC�i�nE_}��&���jx�TR7h_�������-��Z����NS/��E����n7��v������6��o�����O���fQ?��/�%(�M!�����n�d8������iB�juNg��8��������
8/���g�58���)x9�?�O4�\�rzOA��C�2��6o]s��e�ge�O���&~V�37����X�"G���WqE�)��L���COz������=.���!`�rd/������UYD[��u��������gb�h�K��?�cP��\�,� ���Q�J��(H��k�R�u��P�5
R��:��P�5
2@
hv������M5U�����M4k����
:k������9X_��P]�PM�*��1U4\�0V0m�n��K�6��>��z�_��Z���|�f}V���M5�\��Z���5��l��F����C�b�U���p�h��l�~�:z2���.v�����{����8[�n�z���*����M{������8/�n�z�C��z�8�m��0�IDKLf����yi�P�d2\�<����c]Ve���KbhgD���d?X���4����x=�����~2�o(>b�c	0����7�0	�@��1�u�����;%Q.]��S8Y�X"�L����N��N���S8y'���g`2�j�+kq\`-�E/.�����\���Z<��`�g���><%G �o��@����^G�&'��|8�Q|�D%|��=s-���.w�������!���W;�wp�����x��:�8|�
���;}0�Qr:��+����+����+����+���,���<,���\,���|,��v�{S���CS��������Xu^��-
af%�����<�hV�J�Y�4+�f!�4c�e�pM�6�M�IV�$���^���j�
4K!�zs~�������g���f�\�~@�<�9~��e�[�#V �]��#��e��rN��`�5i�*�,���'��|9YI�q%��f)�a������������D�I!� �,I��V8u��_}pEo�t�\o����Mrc�\U��}��n��u��@������m>�s�P��r>�c5t+rl5q�����3�r6�b�5��y)nGI��[� �*��~d���Q������J~F�����[9�'��
���'��3;�'��3�;��gX��?5>
�Vm�f����>��g���?=��6������A���m��Y��U33�
+�2fP�4�D��j6��a�-��H�`���KD�1c���������������~��)a�.�mn� 5��F��o��!<p}��'h)
���tu�lma����9r�L���@�IC�0'���K{v�
��������Hh�����MX��p�//�f<J��4��������4�����8���gE��}F7�3�8VQ��":DEV�`��@�7�V�����rE1�4��?B�_�������(����Rk,�����C��h��#��G�QU��G4��(�>
`��}��>|@�G�|@��hB�0{�Q}��
�D�Kt�D�G�Q}@�G��&n`>
�����:����c>>����:>������_';�||�R�u0������*�bus��X��M���%�}[?�u������I�e2�R�����f}��Z&)�����ha���#
(��o���Ui_PI�,� Kb��F���P�<��q�_�YP+~����4&-i����_�M������s(����q��t4^,KV�d��DEd�T����k��I���&^.�%L�����w4��)a��~.�"y��M�Ji��&bw���lX�ON+JG��05�RC.)��gc����p�r������gY9�F��� ���H$�Sl&b������XFu����?����O���~�'Vk���������K��X���?���U�����jq�y%q�
�Y������]f���e�����>��k���0O���.� U����\�������F�����<�<�#K>��=�h������,\�����F�����<����a��[��h�GdH0hUOU������o_�7�����E�����!�\sg"`T��f�c�ad�aB3�i�	�0������|�8Yc���h�������ju�z���S����jX
	�;%�A��cRx��*	�Rl�@
l�>l��W�F��?�����U<�'��R��*vq�D��S��L����q�r��6�f����o`��t�o��<�Y���v��9W-PyK]��T����x[5��2�������:��"�ps�z����"}c3}e�t�x�\���R\"������
�Y�O_���h����G�5F��o.;|5x��U9��Wer'��|�RwT��5lY��G�xG?<���tR�;�S�UU�7��R���Fq|c�y���1�,���S�	�$$��B*���?E����A�O�.�',@��S��
$����e��y����9{���U����{�"n�!G�$��i(q"�p���{��4*���<PT�r'��g�l���.�8����,^����
�;�Du��2���x5��"b�|�p_�x�H����y�2,E���j���n�z����&�� ����+�k���E.-�8��B[�{/#��_��$iG����������N�b/@P���e^N�����H[��Q�	�?kG��,��W��J�R�E�F]W�pd����(������1AY�+nz
h@��@�������}�9�_q!��N�hi��
w������^�R�8��HUm�����O�J����.Je���n*6���njP��7")*/vm�����V����y��kTX�"|�W��@N��
��V�3RB ���*�7�p��QRx��n/�C1�j�������9����O��V�&-��\�FdKnLo#�%7F����hU:��yY���0�[�ZY�����*@I�������%�U��%�?�����rU�MG����;!Ye�Q#7unR�Fn����w��3�����U7|�mA[qZ�7�us}Qyi�I�I� �O���kL%[���SR�������6��t�j�%�t����w?���yEf���r�i��B�����������Kp����m�H�[�;�����4X�����&YO�$�}��.����E�Po���6�_���j�q���\������M�IV��������W���U���J~���x3��7G����/^_�F{-�7�?\��G/�"0��~�����}���o~y��D�~������[�����b��bF@1������7(���^��@�]���:[�E��`<Cy}�G����p<��'��x����*����2M6�$�8v��N��}wA�<L��)n����7��![�z����A�A�8��8G1�&,fF�9�cNFd����� �Jcm2"p!%�����gA�����;�S~�$�n��*�/	�cPQ�=�>�i��m.���*��*���`'�Se)�`7�S�MU�����)v�J!�J	i)A�t� ���F����E��/��"��2Uf��������Ux\=��U�r�OZ���qcd;�2�~8.b�Z^�2���iX���P��GP�����d?�4hF3�TF��|/�t��)W:<:��)������7�>|�*��3�!���F���/#����2�B�&4u,�Z��*R���s=R� ��@����29Fp��&��<)BJ��x�q���A;\��p�0�
T2y�E��X���@�b�����`��������&�ah��|==����G����y
B4$%��W�u6����tD����,%7gCLC�������H�3�%���/�Xx4��a�N�K�;�3��&��(_�3V�����&�F���@CYH��XH�J+13�� /��I������2x�<�m�-O+��Ny�,���X�����/vt)��~�������<A�"�o�)�������.^�E�e�H�A������� �J��#u������N@[�Y��AUu�.:�J��8����%� ��Z������2D���m��������I���1R���P&':���Y�a�U�6��f�oVM3�����e������48��@5U�9.�t^lS�,��[�
y�5�,�FZS�\zdH��i�1�����5x�p��&w�<*W�����4D��JC�&�-�m�Y�0�!�x&�ASI�C=�^Um��.��%k9js�}t��.G��v.�%	�jE�j_���?��q�|���xl?n��t����Ac���a�o��;�������F?<�!�q�W	{lD9���W���\����c���\�4����r��r��T����\X��h����e��MQvX#z���.�m!�v���r;#����,�����"JiZ�Rj{���*�d�I!z2&ud�G!V�aI���;"��PO�<az���H	TRD��#%P�7�*�w	�>�]!�kvs n����4��e8�z�t�#�$u\P7,�8�^���M	�������o2����@2S�y�0H�2�f������A����d�l��<���#��� M�K�4=A-�u��[�|��>�,�b@�y�!t�7��?\��$:h������:x�^���/��{R���R������"'��B�}S��SP�� r�G`=���X�(������0��Q4F����8��c��[R��`NK�Q����;r�EE?�����h��k&����.$=�� �,d	#=a�'�	e
&�V��GW�Z=,�16����w��\���t��P�?�{>l�����%����]
��@S��
�$��J��
�����"@	%y�����X9>����PD6�y�������v���i�*���T+�����'��F���'�FY���\���k��P����>(��x;�]�>R������/���������O������X�wDRt�Z9�\"5�������(��P{�-������&����/?�c�*F�~)�%U^?�M�(;�3��+0@�c�|Z�����b��m��Df�z��A�
���9��95$��E�Q���EJ�;2lec��CU�����q��������Ie�����hf���B
��;��J����������,�N&C������,�����p<�N�)�����$2�A�-��'C��h�<3R�J�h�
�R�J��"
���HO+yZ��J�V�����<�t�i�J��V��T<�%*���y��p�tE^q;����&��Y���#�[�Gb�������	+mw��J�S�QSVZ*�a_A!Ge��$3��I`�I�2������(y�:�9�j�e�4PC.i��l�����!g�����*W�W��]2Lf��W������F���?�o�,�WzW�W@�p/�v����D���^}+���`"���� t�p8DC�I���
��0R�
h����Y������C���A�a��#ze"�
9
�G9�eB�4
O�(���A�4���fx��(��P������8�QO$J�����(�y��(�O%���H��;���}|�2�}�zR�yR���r%�"O�yR
��I�6k��Z�����Z�=/��iT.�&���3�G�eP�d���#�2�\rL@��b*���rI0�G~	�<��Cu��R�;�`�F��]�S��}v��J~���*�U���?��e��S:��.9��T�)E����W�|�/���?���|v;���{�����G�~�����N���(����i��0��mJ�����v-���JJW��wC<�O.��h3Og�y�;����-���RP�^�Y���F�L����&�RcT\s���:bw"���s&���;�|��)�[.�D�����[�B����D[�B���I[�L���)�Z.�D�����Z�.�4�A.�S�'W��U.�mG)_��Z��>�-�t���DmUINN�����&�i5)j-!��e�0I�H,�h���/�+�-�������,&�������(�#�5��T'�����Te����6���iQ[q
����v�u��@��r+h����f^�Re��M�R��[XY��'���t���N���~tV����6?�U	ku���2�RB�\2]���Q���5�X���y��&h�#�5Ik���Z�wM��G�k��<�]���Q���5�����yt�&p�!��3w�)VS���n`37�S7�3'�p��K��`@8X��p���`��������)w�c��_���$B	��w����!��h�g���d�av�\������I���eN�����U��W)�����B�z��4���[p� }U���Q�P�*g'�a\9;��IaK�8��Y\��.�����q��#\gf����N��X��eZX'y��P5��D����X"I�X�E��H"�(!	z�?ugV?�0c/d^E�%�*
������O���dhC�dx9���5�^�I4%%�7*_?������m!v�P[,��rpu�4@<�wq�����j������7I�.��4X��������K�!a���$�]�k�>m��&i|\�u�������u>	�>�qO@t�%@�cc�=q)�,1=,q�|V�$�hf���z.O����/<Q������\p���]�Jw�;T���w��^��+��b?
����Y�O~�J�a��t[�����d�=��;�Q-2���vbifV��v�eo#(���
�@����&���LM#����F\s����>-|V�pr�R�K@;����Y��"��������L<��i.Osy�����Es=9�Pc��? M�;�����5��N�1e�<Z�LL����
��4����'��SnA*�X� ���Q�Ja�(H�@���B��(�)�c��.�N�6U����fMt�TAg
Zg��
�5Ou��jtO��{-CLm^X�z��`�Q�"c���[W�q������|�����n��w�r�
�����q7�;������sJ3*��nB��Z�m���t�au�a�$y��6#��
�=�Fpz��[�G�Lv���P��=��Dp"��E��8�W�f������<�i��$�>I��E�zd}��.i�8�c�'�� ��R���'h��8�s�
*�$�����r��C���&��b'O���|
'�V�B /'gVp�N���38�N>���AgMY���89/zq��sX��[E�����S<���	=�'��+���l3k��6��"�Od�o'�s�2=*a�^�\�,�:��9�5�������V;�wp��$�x��>B����&���=����Av�A��A��	A��
A��A�0����
V-���U;����9V2�J�XI+�c%w�����LA�1���2�$7�&Y��|B�z�n��6������������oz/�={ucra���0��e�^�I�n�^#��8���dg]���95�����UF�s�%MOPM�2��~xJ�;�R4������������D�I!K�c)��S��U�W�6I��u�f���$7��U����FTZGP
�_dy��	r�v-���2�����91�Z�����d9�Q1��y[������i�-q���mI?2�<����Lg~�m%?#�C�	��?��
��Q��m�����y��Q��%�D��Q�P��j��d�m�7*I��
�3_1PI���m(}<�J-g~a@���6��t-i�$E����!
Z�H�$_M�#�}�<T�UU�P�>�ms�Y���5Bg}�������G<A�H��������k
�'!��	fA���MR�9<wvQY��T�8Ae%��dH�B#���l�R]�}yY7��2W]�1�u��u��I����g<=�h�h���1����]����Q�}�9�����7U`��XWm�sJ�sme��J�\��
�i��@}�����m8�lc�j��6�k��m�\��md8���������l����6>�M^^��E^�����6>��l�|��Ad�l��x�y���o>��?�x����������?;������`���Z��-V_����*���:Xb�������,�D�,�������$��f}�M-���a�zQ�:��h������	����y/(��3�d��k�������C!�P�2������&�,�(�� @�R���4<G���v(:�)�F�d�(��=p��q���R������j�=aU����mX��G:V�	��=�Oj��L����t5-I�#0v8E�0DgY��#_��!*";��Fu9{����#g����hv��o������^���7�f��������f��o�z�	OB{4��tA6������c��#���i��#Uz$��tq��=����v�O�f�3-�cf��=f.��1s�����]�#�����14l#�Z��o�j���|������e-�����B� ��3��4��f�#k
�aL3Lx�1���z&��f���~ZJ��q�S����"��Y���v���s*t\p2t�U�O��cR��I
lk���]z�Z0��G����������d�}U��@�.N��b�:����b��s�i}O.�������2�=�%�P�g����nNZ��>�
7^F�2*��z�ds��:�r�sR��eZ���f����J.x�����kK����|;�����s~��G�xGkd����*����MU��RE���"����3Ee���.�[��*����,1s.���Mf����"]e���+�y��{���g�+����se�����c����U��1.!
�+�J�G�����}v������������1�t�x�C�8��1������E�L7�]��c����>c�<W[�1v���v'��K���M/��!�Y�:DB+Sm�[=G/Te9W7ya��$V��q�������Y$��9�E"5��.����3V�T���9��9��w�{Tj�)����g8���E�;W�$K����V���/ZF	jR����e�oM���-������W�L���??�p��x��o�;Vy��=~G����*��2�A���Y����w�e�\��	�]�Qm�k��#��P���j*�*JN������>UG;�1��~�5���*�9�q}���������i����w{c��+�n���l}*1o��T��"�_@���5�Q��;p������w���i���h����L��q��*c�^s*1��S���*-
��A��>�f$�a5���V�))j����y����t������?�<8l��OYM��"o�o����o�����
^��������{�}5����?��Ts�����}�<�|����w?�����s_)����)����}>��ur?,�����6x���r�n�O��!9!Wk�e�,����6��P8B���]B�]�V�)qqge�����&�eo��iB�0���tq�L���&���`�I��������W���8X�������c3��7G����/^_�F���7�?\��G/�"0~��~�����}���o~y���8{�#�>����w�RV�V���5�?����F@i������7(���^��@e^����Y�E��`<C�m}�G����p<��'�xt���*� ��2E�m����`2D{x�v1��}wAZ>������j�*���k ���=�����KT�#h��j�A���2�4��}���M:Tu#���!)5�4!E�������{?w?\���O@�J���qX��<@NHs�4�("xQDUE/
Kyn�\Q�����(�����l�o��xSj���E���@���J����";��������LFl0�e��%dV���0��Pn� -d�Ku[7��r
���_���?��^i���D������A�`��U��+�o��!d8��U��3�x8������*lmx�� �4���D�	}�����4�/8��x�L�lJ_p�xA_���������2c,7�[��Y8�}�R���`rP%�:M�u�L�y��R���c��v���������;���w����w�rx@��H���c�|�>��>������6����A*����yd���y<�@*p�|�^&�.�$7�&Y����c�E4{;-W-�&4HN��I���jX"��e�O�����hf��
N7��lU����c�U�C(o�
�JT1�-`�1+uDcV����\��c&��8�v��P��a�*�<:�za���x6�xn�r6F)��:�)�+��)�zv�?J����)}
�W���������W��|=����~��+:��kD�J�����N�W��vqH_O�W�jf"�J}f�76�[�3C
0�u���v{~�v*��@+����-���`�"���8��WX�����R��Y�x���8b,Z�1�WW��K�H�jk,W%�����S���s��p��"�&xI�X�U���i�+!��IBe�r~{R��sc�L�K�~���4:��.�s���=�T:9���T�����c��]y9`�*�G�!�_���v��A%?��4��������o�bQ���gT�#���4�+�H]*2td@#�P���������%��QIg{���y3[Y� r���>�<��O��T
��_V�>$���yk�+;��d�sD����+��(}T��'a����^�3���"�K"+���LW����4/���E������$��NQLQ1�NQLQT{�"�)��NQb�Ix�)S���>���)�������N��J��$�B�zd'#�$J"�����F���B��+�F��-��a9�v@)B�r%�6�����H�i�gY������^��&������r����������C����������9��q�|��6�\?n����r����E���],x�M�e���urW�m�p�X!4������R�o���j]9��`�X���xTCLj���ip��i��;ip���a%\9�h�q�����i\,��Z	#�)\5�����zG��0q5�����)*{��zJ�~�t�G��[��4l���n�d�c������t��tpxV�ny�g�u�v2G��1��[�����'M����y�#3�J�'2���Kd�2�X��n��Gf�03I���#5sI�|�L�t{���X����_�z�&/SgHHf*;&/W������%�etE� �~@�0_l�#Xy$��~����4����D���o��1+{J\�6{�j����Xq��7����������DN��9��nq-�6[
.\/�J�������M��>�;!k
6!����t�ql�9up/���V1���(rr�X{nKz�����_��m�o�"k�"'O�����+�BI�id*k?����)cP�#����_���t��\9��f!c
v�M�u9&6c�}�M�	e<&�c�t�M<J�L�G�L�N��^=,�1v�7_/����AIi�+M&�����b�,Q��/L���X���9�M:@2,�5�*l#'68���k�6a���U���d��i,��)wx��R3��:;D?(�RtLey��F9���(�Eg�p������b����<�q���?}��8>�?���stc��I���p	���_������9����d`n\����~�YV0:�K	����m*���4����Z�Wl����p{.�:�����"��@�N4�Y�xKr����r	���\wdX��&�=�>:<:Qc����40:�����&��[����A�����C��b~��T�dPy�89������'���`���dz6�N�p6:G�1��f�Ih&�\Q4>����g����-�,:	�d���&��`�����1��'��rg�$�L2�n��''��rxx��`�c u6��A.�1\�h�<S6y@��4���Vp�>l���!��������C'o%��P/�qa���������'t=��	]O�{Bw�	�'F�nUgJ�2�_d��7���:%�Z��\q�N����-�X������.��AGe����Y�j\)m�r�:JvNet%�y%���������������R&�n4��R�M{��5���<'��D�����K�_���C����r�[5�
P��u�Pf��e��3a��D����.C��.�u�����ZsV!jh�9�����[��[�7��
��e�[����jR��XV_o�*M
�7�`��c[o���T8��B`��]�.�_���_�G3�Qe�p��{3�_��0�D�T��H��l"c,��T�9���
.�$�h*�0����L�3����F:X��]$�O*�4�X*^��	6���E��������T��Wh�h&���I�6�$�x���q$q�c�6ch�����y����O%���P�P�I��bm����RU�����
�Y��d��sgWV���Y�J�.����s���JE���m�g��ca���zwV*��Tn�<+�(�Vj��G�f�2+����'be���i�@���2K��(e ��$be���9�@����2E��e vf��t�R���y�����S���n�"��6���P�E��PN	������K3D>�i{Z�Xe*0;\c%U��:�,$�%k�5g~����`�o>�����
��.�[9%�l�E|�E���3���zs�	`>����Hy5}�����]%�{"���$X��	u��2
Ow�dYc*V��%�!��*z��e��=T��L��AvFc�wN#Y�w�-T'����*rYY2�Q{��;�Q;Y�{�-TG����*�Z2�S{��;�Q;Y�{�-T����"�b�Y�rX����+��5�a���uA
�e��x-Bf�,�B(�!�U��!8g`�*�Z0��f�
�e���+��W�2KB�D�����)�K�;�{�mF)_�HB��}�[2���M�����r9J�`4&�iu���j��CKV{d1�?���WR �U�x��������YP�=���
�y���:e��Rs����,n+������ms�N+?�t'4��?��(ZwU�:gd��:�9R�s�/g���;^����#}9��������C���s�cUg9���<���(g9�:�_�#i9���\���C��.gwI���s����A�����DD������I�u��N���i�]�`9��h��j���As�2
P�}��w�����k�v39��kW�X�5��s���]���+Q�~-���\�j�9���
������n`7����
��
��LN��"u������
�W�3E`���
�7R�_.���i�~}��K��L���#�{_GG�Wj�
���t�wq�2`Dx�
c�]$�����	�]G%�D��`A�F&�)
m]&H���W2��c�TG���M�ul-���
��xY���r���F���r�R���'nN\I��[bT~+��~�\g���IK�k���	<yM�;��I�����<d'N�Xe�d�KW��(b$�`�������LM`�.&�2�	R�r"��L9��C����R{(�oR�A$�C��`h4�r�6q�k�6�86�a�n����Kb�=��XH{���D�����S*��M�6�Sv�AJ'��u��AM���w�D*;� �^�v�NBM�#���#�v~�z�	v||�IJc���+�3��~���b�J�����>$�/�����/I����}��Y�M�$������)."�#T�O��w��<]!��y3
"���O���Ah����B��z�6pW_5*���G3E
�"�H����RF���T�2r�Tu���T��vS�W����W(r���1Aj0���4^EfT��!O�� ��>
��TU�ON��{4-��E)�x�8�A���Cn��3�RS���w�:��������in�3�BS��u`�1x��C�?�O�r�t�] #g��3��`��:C��!O�!�2!�����X��Gb��WKD�./�%L,�I���'�=�]�x�����������[�(g/�Hs�#D�����G���!+���A�W-Ee���2UKQ���=R������Hdg�rKQ���m1@�X��k,YJ�H)�FJ�4R���Rf��r�H)g�Rf��:��R�]�PS��Q�'��q/gjB���>�Z�j�9\)�;��?+_a����T������<�R�]*N*
%�r�[�����B�/��������a����*��l(mF����������5����A��+S�q)���9�^��K{/L?�=�)i�@������=���)��)���=�Ghp���W�=��gp������wjb��a�Z��K^\�������&+;���'3��/6������Cg��|%g�:�,"\Ld*�1k��d	�dC���F:��T����7����P���"���C������������i�6�&@�H�i�@���&�g��@Z����	�6�f@�)�v��a���>�Q��E�39/zqq���6M��H�4A��g�����dMv�g��28���i���r�e��(Q.n�e��#9��Qd�sc<s����Ip?��b�h�r�����7Nc�6�u��\�Z���`���)�?�
�5�6�����:�jw���f������?�`wPv�O�`����P�
w@�(��;�����"P�
y@)(��<��������M�Q�j��b�J��E��>J�*���U��`tN���F�T��`tN���F�8�<R
��1�1�2�$7�&Y��|��z�n��6�,����������oz/�={:�m�����;�2B/�$N�7l@����G����Y�����N���UF9=�%MOPM������K�����R�+	�cW�o^����It�2��S�&%}$}J[���_~�)���o��\Ji�y��R�rq8B-��5ZOy�Iq�z�`����>V?�������{?��J��,��;n�=���N��x5h{�k�KZk�6�y1j*L�C�����,d�O�����)x}��fU�;n+���N�������0�����H��8��r�����O���z��?��3��?=�E��3��?_fV\��j�lC�9!�
����6�����j>lC�96�
����2��m|\ dT%Q����Mr��e��{��Zp�����)a�&�mn��0SJ�)&G!C�Y���"��e�8�~BRi�^������q���(	����#(t��?A�����?��T�{������0v����Z��c
��)�ps����A��/�Vj��Nl
3�B��Z��c�:1���
.���.��-9�����f�H��X��z2��da&q�R�?	${So�^h�����a�&]:���\Hu(��A5F����Cq�)���Pn5Z��a�N�M.�9����G�i�#Dn�I|�^t�
w��#u7�-];Nw��E�n0Bw����XJH���q�tw��P'�&+�)%9�F����y,�P����uRCk�������p�e>��7@�p�����d������8V�L�~)����@�0D%n�[owi�c��\��%l�/LQ��=��B�~D�.�����P%Z $��0��!��!� �0���X,(<`��������J8C�t1e����`9f!1Q�q���aS�����l>�k��r��;�����
4k��)����z�Y�K9�u�G��p����5���l��v���������w%/�H_��o���r�=6�[j��y���.�������G�r��k��5���u�����7��f����X��`�HODW"�ac3����|�a�H?���
N��R�&F`'��KUXj��M5BA�vb
��d��o*�<j(�����Uj(Y�)�2�m���r*��~��NSXs��N��l��0����
j����_a�M�>T��MC?Tf������1�"7����
l�6s;u;s��`�H]AE`�������g��
�73jt+^C��d�f��G���	7���`�>�������~�`�9]��~�O�k:�`6}�_�`�d��wW�U��fm8q�Z%� ^]Kl��~@+'���Pmi�L�[*M!b"2�����D�$�p��eg�6�wD%����m����F�*"y�E5d��7�(��*�����(�2���-ju�S��H=I�m��5������
�BaF�Y�pizz.�I������<�L��}i���2f".�fY�CD����9���e���� �����Q�r	\K��v������0��g���8�.~)�9&�Y�H:p;�8��j����zFN$q���P$����T$���F����4	3r�ME1!�F"aBN��H@�j�T>���!�;;����F��#�a�;�����0���a7X�agG�^�?���O0���r��	��<�LbJ��v��c�?�
���,�?�jt������Gr�5��9O	��e�(�GY��e5:�D�������a����!FK���?�����,�?�jt�������C=���++��5q���bD��V��2PZ��~���{�u�<A\O��X�B (w[f@5�i��0�F�&4��f��c����N�����)K)C���Xz�&,���{�:�9�G����[�=L�QD�j�*+v�-���I��U�f��������N��pE�S����Q��0�2��7���8k��I�qk��R�	�[���n�4rOF�)����S�z����r�G�����/��^�����{�|w���)C���N�U]��;/Q}%�V~������:{��^S���5����X�^p�
X#��5b��.�F�����Z�;����5��8��mav��
k�0�����h��U�R	+�h3�,jJq�cN\c�V(~��w�����O�J�Zz����M)�SJ7�()n��
��(��B���8�?E������O�.�>,���S�I��(��B:��7�?E������O�.�H,(��S�����
�Dp����J{��"�T�����\��v.�&��][&��Nk�]�Rz��4e�WQ�����z�;5�Uj�7��M~����I�����l����Mo^#�7��:�z�*H��"���O5�����L��R+�d��F&�r��������b�����r���L�)�%��������Q~���*x�I`�FW���-
t'm��[ra��ZR�E:�sHVG��JM��
�W�	��_e�����"T�t��������_�0Y;D%e���?A��%{�`�Kw-
$�V�Y��/&����*���Em����v}����Ma�$\���x�������E�����e��e.	*�V�5�"6m@�X���*���^�+�����Z��f5$
�O=�T5.���������mT0Y��KV����5�.��Ik�PEs�����SE�����s�����>]���$���&�!���/���A^���Z�"8�^$�A%!f��sw������H���PQ�*���\i�{�*�-}Z��f5"��O}Z�:�%��'��O�"Z�����iB\+YSh+
jItK�.�_z�����L]��e��O�8���L����9����Y^\��t	��D\�{]���OG����OF�XsM/je�v_���M�.�m�K�(�����>�����:)�$�������4I5����K4ce��W����}r�Q������>
�[6+���!����i
y,}��J����x���p�i�����i���F���6L���0:�!��O���W���/���������\������F�eb�Ef
�:>�f�n@p����m�y��
��5�\���S�|HN�z��dIb���������^�H^j�Z~�������@����?/��F�Y��:MWKTl,��*A�R�vMc���n?�EI)j�w�_�_���"���b��}B~�����m�9
~>���5��h��y���??z	�yn��Z����~���W,�~��������X������&�g������W�/��;��!�� X@����%�\��� x��O(���E��U�A(�2EX>I��A0"��`��f��� 
��`4�
Ng�
 �L]��.���������)}���3A3����s�b�IJ�}����_�\���h�����nd�#p�$�f|��,���<x~���{	��Q=�r�f�<���r�{G�MhfO���\�
�r�rT0������>�`��nG%����V���[��A��b�SK]Sz�I�A��-E�3m�k�uj��S��hvZ:,0��D�R rC�m�����x������ZA��Z�q�����X�������~V�O��d���)�����Y/���O���4u3�����|/�t���laxt��"V@&�����W�{���G�F�,�j(b<3��:�8�&�q�T"��v�L�\�In�M��'E�e��7�G<���xd�kN�	P�����c��0��8�l�R0�8�)�+��0}
��G�EX����W�E���F���J����|�^
l"���C��sR�l�l6@r/�?�tZ��� u���1��)����as+d�1D(#=U��R������3Y!������!L��X��I��b�������^[��N�����Q������v���%5����k��&��C����2�'���F���M��
�����[��Xl9�������Wb��KE��hDeZ%��;;�T��04*	��l��cp����XY� *�*������Kb������I�����e3D�'��% u.
������$l��k�����<�Q<�Y�4�,B�f�L�.��Qx�
ipjCa��*��KA�x�@S��U��"{����k�Y��&�����;���H:���=E��+'y�[� �4�[�Q�i�L�[qX_1�����.;�?�&�$H~[��4��,aC�L�1��JF�
P���S���QUm��.��%M9vk�����>.J�%m*F���?~�$��M1�������<89H��4��p��l�W����������]+���lp�l���kP���c%�9O��V{H+jiE��V^���O
�u�`(�`?<uGd;��@d��W��h1���[zl�7�
;������+�6-��+����oen��[��5���V(�'�m�0�QE�T
('���I��.p9������B`�d�BN�D�B<�
Y��$3^��3e,.���I�n/����:�������7����J~ ��H��\
�U$zQCL�etE� �m@�4����<��e���� M���4=A-�5Y���|��>�b�&�V���2���y!��7��?\��$:h�.v�P}[�K%�M�+y�pv-��Tv�rR�UTsyL��E.5����x���~w }�N�jFuK{�qiI?��w	O�{�~D���OJ������0���~h�D�i��lv�.��+
������[ �����b%��H��j���m��R���eY��.��}���Om�z(��_�Q`�c��V:�I��Ur|��K�Mt/�D�qe(����|\_Uc��WX�w�v~�X}dv����/Z]���x��G�'��d�Nb�a7")��� �ga��mm��%w����@m*���������:�"`t������u�*����O>MR9F���3�
t��dY/V��H����\2���\��"%���������*w��/AD�u�H-j����d8URY�D�N�6N����d�>Xi0:��M��(�����pL�B�Yx�� ���l����i4%N��Yt�� �MB[�"�)j=qO4��f2H�F��V0�(�M
���f2H��1�����J����k����
O�x���/�~q���O�x��0���jX�8e�J�G�";����+��-�U��t�1����_���w$�i�M��c�����%��jbB�]��O�"���(�T��f��OP05
1�Xl7���F�GM����U��w�
���x1���Z���M+I[��c�b��[&��
{Q�i����LCn!#zC�aw���5�%�(��"����������a_������%u�hH��RL8��A&��id2��f���`!?�����d��M�
�K�t�(�i�H�RT�i$�)-��#^�����T�r4�Xz9�$�q��o�rI����f�r|*iO�CK/'#I�8���b�3�&�V[�-m�Q�)�����I�Q6O��x��d?����B���Q�+��z�'��f���QKDWq�����a<�Z@w������S�Jid v*C�X)��u!@��Eb�*����@��� %����Aq��Dx�D��+�r�gt��]r��i��;�L�[9>�\�CD:�wYg{�)�����H��[^�B,D��Y��AF��n)�;�*�$�*%U���
��>����<��n:��!����Z�j�t�����1��#I�����K������r����U"Q@�:�!d��PhB&G@~F[��t
��(�qA?������������gq�h�t��+9_���v9�-����DmUIR_��8
���`ZM�SK��{IL�0J/�����J��:��[��xCt�@z���:���fI��Y�j@���(�m
����l�>�E����V���_�B����r��u��@���c�i������L�X��&*k�!L����n mW��~d:��\�sNY����V��6��7���p�`9R.
4O����H�4�<����ed��&���dh�����t�a��#0SgM�=�����&n`S7���������E��(+^m�������
���/+^
=������W��\�;������S���T�3���tX�wq1K!"^����!v��	.LP	��Y�.v�X
��u��7c-�W��;��OCU�t&��D/A�k3Y�q9A�cD��5��3���_���H����,�9]<�(�N��X%$e�Q���.0��Q(�J��$�h&hJR���)�4�J��I�@#w���Y�Vm",pl�����9��4���SR�����o���pW��������3H~�pnx9)e�]������2�����l,����
�^����is��z~��i�r���j�&"*�\������QCL:�;��$�Q�}�IJ}���+��1��~���b�J�����>$�/���.L_��w��	��Ac}��I�ql�.#�S\D2G�B�q�>�q
t%7
��C4�zH�RrC4@z��b����AQ�:{�,z*]d����@�]�T�Nb��J�<�*���]P��S��D����{"�c|*����B��y���q��g��Uz��p:��:����P3P��Ce�<� �$�!�$&�gq���!����U�r�i<�B���m�g�����)E������m�%������=Wn��{�S+�Z����V�+k�Zyr8Q�c�{j��8�����&��Fm�?�c�GUKQ������U�RT��j)H��~&C�_�R�FJ7R���R���2k���FJ9�����t���0�g��������&�`�$�y�������vu��]�l�o�h�3���
~���|��[]�N�v%n.c��4n�o�1j�*$�w��!����%5l7�au72���P��3�`��{��sp����{�ms0d�V�=�D����>��1[,S��������$����p>:.V������V�[�"`��!?��{���x=�����(�x;��q*���9*d������Hi m
����S �LO��3�l -��@�H�i3 �H;����9���5a)0�������i��Tt�%�zN���w_��d�%�b�9H83�j���
?��gd7��lZ�a�,�b^��i�X�h����dV�J�����`�?����e�l�g���i���k���m���o�W`������Wi���h���VI��qm�)�=Q%W�F�T��`tN%a�F�Tb�`t��5#U�p��`�e�In�M��'����"�.V�m�)���{/�={X��������}�����!�JI���"x�J�������E_N����������u~�����$7WW�S�(�(�����9����aC���(�Y�h�~��%$�c��)���v8�P��r7��Q�f�;n+���N�`���a��d�^fi�����g�$�?=�P����OFB��9���l��L!l���#l���&l��,),�T.�`^��Di&�������*d������*�.Z��)a��mnN[4FD�o��!,����'h)����tuiWm1���[9�L��=����G��[���I��0k9>	'C��������:U�s���gi�Z�
]�x�QQ��WU�~*r�hj�[�S�C�.!�qF��?	${SE�/�?��_�/�G(�J���P�
�E�v����Zcy(Nf��X��F+������>.@��>.������]2�������Q.�����q����� ����9��pY
��h����������|\�M:�����RS�M�8��&��I{o�O��4������bo�n#N\�V�&�W�������8O`9y�d����u$<#�o6��l�i��.�G�LQa0j�yGP�>�&X���^P~��K�T�7�(�������n�k�*j����:�)�|��o����&
~^�hl��f�����v;:2t�Oi�p���v���A?�W��`��s����d	����2'���NNM������nW[+�}��Yq��8�b'�yI��9�c��T$���F�0&��T$��h$�A1��	q$l������:(��
�PP�����C�5���S�����P�k�g}������	�;��^�8$.�an��<����l�?���R�������.�1��K����?��`��CG���W'N��k�X�%���P����O�*w"�ou�0�����������0�j�9PU��h�~���{�
FxE-G�!���"�P
c�aD3�y����	�0�&<��g`
o�n�k�=�\���S�)u���y��U���
5y�TF��h^U��+���W���LU;Rh����a��*����R�^U�&W���\T��*w9��!�I�h����6���mwha|�h���C}�0����Q)���Ng�Ay���,P�l%]������S�:�yU�tA�3���O�.�u�QY�)�
��)�?E�J���\W�r���4�D�S��q�l�<�N�kv��<P�����Kc[��$&�%a�aWp��UU��n����2�P-�i��,C�4[�35S�Z'�F�:O��R�����#��g��|�)E��F��B�����.L��'�����&8�5��JN��XSX!?UG;�1%�����r�$�g\�����(~I�K�i;���jZXe����]C�\M3,Mk��v�<�}EQB\Xx2T$����JDu����=K�]D~�H���|�dTK���W�P�+��;u�4��V�X�Q-C�Z��m�Q-O�v�Jc���<��
c�t(6���F�U$|�����p
7�|�A��`���G�x��J��4i��/=��c/=.��&����[�zM��a�3�E�R��q���Cg�C�d������*S��|����P
��%.�l3}�
D��o �*�7�5��@�T�s�t�6�(7���spq������r�i��B������������
�q������6�z��~�l6����
�"I�A���E�>$�h	|�����+>����q�Xmh���y�)��o��Q��������~@�����_����@�	�w?��x�>���7����b0G�Qp��-�DN�������\���2�E�{A=���=:��be�����)|����U�Q�RTr5r��-�>����N�
F>g-6�N#|i"�sI�HX�X'�5W2��*��r�
@�p`��G����\~���w!�;����s)�.5(��wp��K6t�e�%p	
<4e��2��A�L�K��S��4e���{TaAp��?�������|1���?��!Lm�hC�~nG�=���Mj��8;)g2��3r)g'�Li9��~x��MJ#9��/���NC���#�GI��	�C��c�J6h�^�h�&i8&gg� :
�����o����fh��F�$�E�A���M�dMB�M��E�Ix1��p�G�TDn$gJc�)���s, ��0��P�WG"�s^W��;I��������`u�b�4D[��=�3�*Y�=$���G8�ZV�
�����l�����$�@��)5�KUDJ�1n��r"o�c�%4B�U69Y���EW��]��v�����Hd��X��������i�8���_����BA=��pH�?{<~����bI�3Y�?{<2�p"��b��k���fhYj��$��'�?��	�OhB?�z��q�e�N����*����_�)wbm|���W�,��]�������p�n$��(\��]��.�2�����t�+��E2\d��p#
�	��������+�� _WG_GG��K@T��(�pC%��@e	�Q	-p�P$\E���	x�_�C����\�K���k����Q�����Be����E?R!���$@_��\�,d����R�P����G��>I���3h�.�G�'Uj���d<�G��*��zn����_��4��#��������K����w�u@x���\^�
|a���"��]�![y](�N�`2
��V�J�X_��.�����AMl]pL������Qoh]8���T	�K�w����D����������_��s@��d�f����.N�a�u�����p��_���H�B�	8�I��B���%F�' +{di�T���<��5bl)�T�B��6�e���Dya���'p�a63�02/C&�"�U��������)�=�H��\d����9k�+
�����Co>��2/��y��N~��=���GX.RDv�R�������K1����[�b��|���*�*&������|Gb�M���W;�T����oUFYL&��e���;�Vwr�zr���
��SGr�����e���iP"�6}���@~%���u���6���-���Yg=k$��;�#��4��G@a@;�-�L�.<r�1��������@����j�~�Kd[w�~��ZX���wF��!bT%�3���x�-��,v�le�#��I�G���P����(�6N�MXjau��B�E��v��J��o�- �/O����y�2�c!�j,�H[����
)V}YG������R]��:��u��������@����K�N��z�-����
,����zd]���Z_�#iY��-k��N�uIS{�����e��V��U|�L.oQ+��A/�������U�n�XP�}�$���J��S��m�����Z�����J����VC�%E
&��w���lY9}��iF��z]8�F=���V�6{t�����������B�/6���_@�g��p�����i���\�a�o�hW
��)�TgC��92*��C�]o��)6j*w]V~CU;�2S�I�������������S�v���:@{(�m����x�]lhCm�t���y���E���^���:M�q����)����*���:	�
�o�\N�As���������@��z�B��A0��"�o�4Y&���W���f}�pB|�MPGW�-�E�M�>�fp�xV�(��s����d�7�h����qOX������8��F��(�a	��$��0���-�)��$	4�t�y}�
�����,�0��
$5���.����&�.f�?�~K�s<d>�K�sk�d[��};�F�>��u����	N�������A/��\����A��-J�4��r&.���r�a���+�#p��+;�mN������ukP��;k
�o�H%�%��� ��%W��=�T�1�6���'Jr38��23������E"����T$��������L�eL��S>�BO�`F���<����`�#�m�M,_F�/���k����	S[����V�pb�Y��(<3��pXj���5,�u���I:O���CX���F�Y����E�E�9/�
-u��sq,���Y��������P�$���S�[H�'%x���R��Mb
%x�)�S�[S��$x���R��xJ��*<cUgj����Ao_�K���9���;\����;	?�g����h"�8����
��r���Q����P�����G�rJaNw�e,gpFs1�[��s���;����_�\�39[��S����9@�u�#��."�]<4�������/]GB3!���'>�+UsA�!��	������x@����r42�@�}�����Nsi�����zW��2,������q������B��:pe�5�?����n�2~U��(
���g%�:��%i^f�P�y�y(�~4�����X����V�#���G�_�N����>GM��%���o�_���	���K99B�*��tR��/��H]��$����Kx6R��/��e��K�I]���?#u���tR��/N?%u����3Fm���t�N�������I�)�[����.�_���(�_�>%+����K������q�������W���$��������.�_�N�R���3R��/N?%u���tR��/N?#u���tR��o�������8=$7|��$}J�V����K�������q���4T���cR��/I'u����	�K������q�������g�FS������q�)�K������q:	����z��$}J�V���Gi���S���/N'8J����K�I]��8��(�_�N�R���Gi��t���I:�K���hTK�_�N�R���Gi��t���I:�K����/p]��$�������	�����Gi���)�[����K�I]��8��(��W
��%��.�_�Np��/I'u���t���q:�Q��$������	���%��.���a�ui��t���I�������	���%���o�_�Np��/N'8J������q:�Q��$������	�����Gi��tR��/N'8J������q:�Q��D~VU�6����'�@�gN������9_��t��r#k�=���tmS:g]g����[�7w�e��6����)������R>X�;��*��*���*�+�yY�����%����}���Y[��htnd�bWKD��1d���G�<��T~��B���YCP�� �����'���,w�}���EP5�2�0��Z'#3��Y.�8�u��#�C�D����:c3R�Q.�� !{
%&�V���z�^�'��e��<Y=����OVE����O��A���Q2���!��a>R�z�o�Goo�]������Wo����h���3�`����������r��b������]���W����[k]����:���i5�)������\R���'*�Y�J�~!�N;�~D0��Ck�j��1����GF5�a���j��
��:�G�Nt�Ox�|�&u����U�����yuK�CUq7u`�U-�cU�]�[�_{��y�R��.���3���������[j���+���|1�]P�98fj��v����T4}��#��7>���GE9X��X��c�k��D�nG����xb��vb���Lhp��E��o���]OO��Akp�,���t�����-#�1�'������E�:� NK����r���-EX�"K}�$\����a�7���X����0EMoa��*�`������q*�>��C���D
F*�b������B���Ni�P��.�p}�7N�	��
�\,�gbi8��#�n���N���~�9h���,����J!I�5#�g4�I���3��G��1�3��S��TIOx{����gB�L���n*��$MY{�4�T�3�yfr�3
7��C�f�=��t;��T�������.�s��XB4�R�V�i������D�g��R�NDYJ�b&�gj+M'�������
)���uf+K'�,-�b��s��Z��x&R�=�9���u%�R��H�qP!�]a�������� �:6f�	6�]�r�	f���a
De�4���M|:8���t��	pf�k ��*I��0w�����E
����f���)@�[JC����0��j��Z��t���X��X���5�]���
a�0�K@_n	��m����r]n7������9�-���������|�<$��Y���&���j�9k���,�M@d��YU%���;�p-bi��k�fy���Gr�r4������F"�����J�90�Kv�6����5;������t��"�b��<����_~f��Z�L+_�,�K8�o�����f�V����7�G�d����Y?�+�"%/}�
=B���O������kN@���j����Z���[r�B6��)8����1�:i*~>t��o��z)��ze�@TKa	M�V_cLP!����� :�3r�H?������Q��0�&��\KQ��a|89X������P}l�y2�*��NOO��d�G�!���'#�!�������SH�g�C�BK������,��Rphi����$�x����#K���
��������V�(b��2	g�8��F��O�����?������x|�&��h8==�R#�!�����?��V�)]o���rN\M�LU�s�tW2�k��v^����6.b[�d�tJ���0w}�I����o�"� 3t��d�)%�S���0�����U�����:?�OV��^n���Y{�\�V�@�p5�+�K���y�����Lt^v�J������^0P������
J@���y9�_�+��!�j{%���z���WL�@`��S�(i{��{��W�+Y�����.�_�+�.�_�+Q\=�]`��W
���~���S��k��La���_	��v���_	��u���_)0�����	�����r��W<f]���W<b]���W
��@��8b]���W
��@��8d]���W<d]���W
��@�����.��2�+��!�j{%���C�5��J�g�C�z��J�#����W<e]�N����u�z����	�uGg%�c�����J�Y�c<�+�.P�z�W
��@]��_	p��@���_	p��@��_)0��8h�T4����-��������D?Z^	lH�h��|��RX*�A?�����FT��~�G�+����Xgl���~�:c{%�c��1��������Ygl�vB[?a����)m��u��Jai���3�W;������^),m��u��J`Oi�OYgl������u��Jai��Xgl���r���G�+�=�o���������E�����F�V'b�;�W
;�oS���J`G��#��+������^	���~�:c{���%VJ��!��hy%�S��)�������OYgl�vF[?c����S��S��+���?e����3��3��+���?c���bXJ�
�����_ig��vJ���������03��������S���J`G��#��+����Ygl���~�:c{%���	�������OXgl�vJ[?e����m��u��Jai�g�3�W{J[�:c{������3�W{F[�:c{��j��+����)�hy%���1B��JaO��)�hy%���1B��J`)�1b�������3B��J`)�1b�������3B��J`)�1b�����R�c��+���g�����R�c��+���g�����R�c��+�����"�W
K[��+���b�cF�X_)�)};e-��RcF�X_	,�<����R�)}����WK)�1#D�����"�WK)�1#D���f��RX�zF�X_	,�<����RX�zF�X_	,�<����XJy�!b}�������XJy�!b}��������F������\��8��E�����3������zc����n���g�p�5����nj�3����mC�������r/�;��r��(�q�?���R�o���&qx)����!�#
?��#/_h�
&4��f��c��i�M�{I�e�t�����*>Y_z�Kg�In,�����{����c#7;�������*�k��v��
;�����J-N��V��
��o�����#[{U���q�����K�F��[��F���0��d�`R����'?mZKv�|%���M��kf�c��)��?E::cZ�SD�W������C���=����������x�(�� �����p�#v	 ���p !�1?@�O���f���"1u�tU�A��zqs\l���t3�z����7��M��_#�#^.��e:�M�P�+��bu����g������l���S�ty:��t:��{���[}���X�d6�/��,@	8��P'��<Fp�rq��x�����s���_��H�����]���M����<��ip�IPI�y��]?l/���`����j��M6�Gb���	xSHm)�H}���u�BL��@0s4�b�z�yX%�&Y���{�"�Kx����b����_B�C\�%��p��'T[��,>~L6i�P�3�*@]^�6�y��t-�W_H��mL`������v��.YI�
��9M;�C��C��*��,���[_'|	���GN���<�����:����a��-�J���h9
FggS��0���x�X~1���?��1����$��7��;.�e��c��r�g�5�u ����:�z�8"P�e���$����g�W�V~gBi�(FN[�/���1�7�|���'��:�R4�h��l�m�������_!����-���a��f�����9��W���.�"\���r>�#X l�N�>n��a3�i�x��&b��}�>�b� �hXe��p�*[]��J��H����7�j��f�*��{�UXQ]t5�]�4�0E������!���y��Y?c���H��Xj�a��=W>����1�5�z� �r���%iH:� ����+&��!�{1n���@b����N#����A����o6O�"�F�������`&����� �J�m"���.����Gx��D����G��U�PP~�b��EY����P�F����\c�Z��ip�Y��������$y�5�L�K2��_H �]������Q�U�Zs��F�w�5���jewK���nq����R����V��EC�J�v�}%��um%�)������q�4���qe�r{��9?_m��3�w�V����;�����^���#h������x�K�}	���P����M�D40�x�J~e#�� ���u"�?����}�/#��L�tK�8 D�>���D�d�&1�4�::
�TBiz���E	/��d��|�\p�����a�����7������'�t*Q7v���u������q��������6�����~vstk���W����6���(�P����g�������I�5��
���uv���;�=�\;l���v���.���������v��a����6�~x��rw�������N�����3��e�\�U�D��l��w�Hw�-��������;�=#������|}w�XQ=����$X1}�������s������3���kH9�.��:��NJ�]F�@:���u���/�����>��vr_���~����eO;)c�I�����k|�N�OE�`*�m��es�����D�T��	�k]!���.�l���[�IU��Ey3�G/���t�VG��*���M���T��������8�5�[�xo��-p$T�-p�����\������P����o
o�����8j���9���[��sh�}nyo��-p�7����[����`�������z����@������U�S�k'����[�tig`O;���������>����i8�B��8g2O�Cf�C��5c���7�i;���5�k�"o��Mz�I�7��&=���Qv���x�o��Mz�I�7���Pmb�������tKq���"��g���o�o�����&=j�7�9������sh�}ny�o��Mz�7�����e�����`��7����z�����@�I�7�����S�k'�������tig`O;���I�7����>��>
�b��1�OD�;�@6�Y����'N��*�
X%�u�]B]L��
����g���y��u
f���YMg�2���g���[����.~���o�^�e�������Zo���w�a��;�xx�o���wk�7���;�x��H���x�o���z�o��}y~Gw��!�xg����w:�,o�s0G�7���������;�x��tox���!�B�����]o������;���9�&z�o�������NJ����7����>��vr_{�o���]}�==���5�G���������6�Y,��up�0�Y�I�`PY���3�v�y�&��G7�!�{�!��(���>����@����s|F<���ZN�>@9��Jh�Qi��h
�A6*\_��i��u���M���;c%(^���I���������=�'���[Di�
o�-��E����Q�"��Pv���"�[Dy�����E��4�[Dy��n�=tT��[D�yp��q�-��=��"�C��Qsty��}���-o�-��ET���[D���!��[D��Q�\o�}s�h���j�z����:��NJ��E������>��vr_{�(o��]}�==���j)l�)&;����j���&VQ�d�k��pF���V��F�DjS���^�=�m�����JX"�e�3A8<� C!3p���?� �[��e���#�\����F��Jo���Q�5��F��(���Pv����[�xk����5��4�[�xk�n��wT��[��yp�����Q�=���C���(styk�}���-o���Q�5J���[����!��[����(�\o��}S�h��F��(��xJ}��4�[�xk�.���i'���F��(�������FQ�Kh��8�{Xn��B�{k�P^�����!�����Bf� ����6����&�����$=�e}�6��lhn�� ^.�y��H����@?�����@e�f3�h���@�2����A%�"x`��58�h�����1]��.�L TT&/F1������B�t�!�(�qk��b�B"�L��A�e�/�-Bl�MG��i6O�w��S��5M�"x�G�W�~�d����,�u�G���,��#x�Gy�?jb���
�]v�;��6@�����_�l���.^��	J�0r@3����Y`Z�%���}^LU��M�I]��kM��<�MU�@�E�B����
r�&Z����:j~�����L��:�c�vy�nyF`
vF7���S���b(���o��
�]��
ne���C�r�X"�c�������1�������im�'�����X�(=���H�r<��hF�A���]���z��)��yK���4���:�t�f�wp�sm�BGT'Y1����'?����|�s��;��*����t�X��r���K,������z�3HY�b������El�A��D�K����5�JV~��bb]��Os�r��<_�a�8lxi����O�9U����.h���fd�v��\���s����~�z��73�m��;�.ul��{��0w���cS�i�<��k�K�<U'�1�]{=j�	��V9v����u�:������_,B��1���nM]�)��u�[�i��k=���u���X�Z�8�E�t��R�&��G]�z�����a��.ul��|�u�G�M����������[S����c���u���Z��5u]>�:������c��69�)I ��KG�.�0���R���S9�kLK��]I~��E5-�g����p��p����������u���	�a��j�i�B]X�Ox�ZX�;������b�'<l�'
b���� i������sH~�nb����b~K��@�o��@��`��N��MfD�]>��j�|h����{)�������g��I�,������~*o3�l.+���2��n+m!(m>���b'S�=�b��k/�}}�f����"{$eo�x�~d��������Yk��w�i�����)��w�z�\��wg�����������j�u�]�c�!p�Y������f�^�Dm,�-�B�9wQ���}}��(mq.��!�������V�>rP=��T��
x���D����z���j]���;�4��s�������5S�G��NLT�������E*�k��Du��X?����Y8?��tc�:xDu�#���.R��I7����T�:��Du�4��t���C�c���4u���ZG:1Q<�:��=D-���Z�,~��5o����|��~�:�������/�'l���l���	���1{"���s���CC��'���t8dF��x�1i�*:8��Wo6`
�iZ��e*���L���[[C�{skon������7��G+���7�v��7�����uG;������7��@{���Av��[w�����P{�����z`�:������`�z������`�:������A�!�13����NLS���u��A�c����W����S��#�k��Du���\O�1U<����NXv���ZG:1Q<�:��NLS���u����c�����`�m���uc	>�a?hX�O�t������6�c�����M���=iskbg�1�O�x����VXY�i�^%R�$���@h�fM��Djb1z�Y�����>�6Qe��'x(�T�a�r���0��o=
�	~�/�����.����[�8��-P��a!��J�&�%+�&��*�&��Z�M��	�K��	����M�;�oB�M��	u��M�����{��&����nLUU����nLT��:��nLU����oB�M��������?�~tb��Ht�#�����G7���x@u�'���Q]�H7&���T�z�����1���t�����T�:�����!��~tb��xDu�#���P��7��PcB��G�K�qX�A��|���~������9���/�'l�����P_�������?�.����b�L���/����&Hb��J6��������4�f���5	m����D�V6�&ef��&���z4F<��9�)�ftk�L�F�i����OAp��5an��F�4��P����������!9Q�l%n��x���8�Xx������q�-���z��z��j�����r����r}�M������\����r������	o���F{��C�I7�����]�H7&������I7�����]���\���e�y@�@�,�K?:1M]����NLTi����jt]<�:��nLU���u���C�s=��Tu���ZG:a���c�k��Du���X?:1M]<����NLT����[�c���\z����%�8����u`>a+l?f�_~O�
���������c��-�����E_����[�������X��vnBCCg��*]6I��e�������-xS6��x7�f;Bf#�����d��jn�Z5���������|����T����Z+�U���u�������UoG;��z�U����@{�U�Av�[�v�����P{�����Z�:�����Z�z�����&Z�:��z�U�AY�!�1k����NLS���u��A�c����Q����S��#�k��Du���\O�1U<����N�u���ZG:1Q<�:��NLS���u����c��V��`�:���uc	>U?hX�O�B�������P�c�����-T��=r�^�L���^.�m�$����v�I��������/h��	��h���/�z#�$��B���'���-�`����6IOh9���M����7�M/��<N	�n���5n?���m���n���p���V��3v0.W���(��p8��ODF�-��������	���J�#��,����d��!���X_�����0��	�7|~��B�aH��q���1���@	C�uP{f"hf��3A3��2343�}f�}�L�Lx�����v����s����#x�&�B�T�E	����vR�%T�}7����O�	����������E��YJt�g��Ey��g�"6o�}Pj�Su�k,�#�m��psA]/F��)���J������U�f���2I�I���h.kK�7��!s����Im�S�f��#��R4/d�WzTv5\�4J��C3�$qwe��0��
���5��{t�~����������v@��x��g��N����X�T$v�yi������Gbp�A��q���t��-��L�,��V,�+&w!�.���|��3�j�}�%�`�bUd�=�G��z��b���^v�(rh[����d=��mS�	{l���j�!��Un�v���mH����z> ��X�f!�_s�+������C�/#*��_FW����|@$��+��c=�<���y.{�i���BI���8��Z��xSt��V�grpc�9�3�2[���/���\@L������L�I�I*����{�"T�����*]r�B���|���^�HT��-HyY8�pyZ`"�������Q?����j?YE��//I'�u����Y��������8������+�@��^����wU�5��v�{�����O�yS�3���&���ve�U!]��>��L����L�U�ZeJ.���!u����S��O�i��I��?�H��zS�h���W��2Q[Ur�����h0L�����.[���\��P"�#N����Xk#����v�h��ii�,p��a6��J�sIeC����5�������z%=BY�DI����l�(�a�M��cp���O��+I���:��\�0DE$�mr��|�vQbM�����,���;�������k���HoKN9K@m=2F$K�_������sx����8�p��a?�M��@�H�<�\�"j�-lbh5�������������\��"���Z�����aL��mN�sU�B��v��o��3.�z/�={X���?�������~���	|�����m$r�f[��I|'^�d����m��.�mz���?m���F���oMHd�~�aC�����K�2�uC�%
�q~j:35^�������^�������=I'���.
'R�(�������d��%�h�
#�E�W��j�F��(�D�2
��d��b;��2�
{�������Cmy���:5�-�Nq�x���k��x��8<���ix���kK}�M&��E��������t����������N��>��k�����5tc�L��! ~���!����c�ad�!s�C2Lx��j��=�U/a�5�F������-�&�8�uu������L5C�uX��������������z*�������
������0�S������R��&5-2STG|.F��b��v�>o�[�G�p�:YD��
���@(s�.��`}l�%a����q@���s��y!#?3�sJ
������p��3���s�;]l��b��)�{�����W>h��?���T��b�
��+z��y\��Y�����������H�]�{W��;�������w�n�+�]�{W���w�.4�{W|2���o��]����+����OtI������;������wEo��
p�Fs��~��w��5��/{���<N�O����b������{Wl����b+���Y��wE�]�{W,s�?J~~j:;5^�������^������I�]�{W��w�:�]q�����wE�]�{W���y������+����rY=��~x����@?<����� Y���������"M.	i�]7�2��v�F�����)Z|��C��6��1�I��|O�w���r�n�O��!I�f�7��_��)��KZ��������mp���x�1	�;6�g0G���X������K��f�GK�����}y�<���F�L�������."�����R��V�����5Y����&��%�w�<���,�]�{�f�{�th��.��Kz��q���Fx����d7���.��Kz����d���K>�����G�������%[�h�'�����Nt���D{�����7@��y��Z?|��������wI�v�����[�]��bxj���K��B�]���y����s����.��K�9��C?5��/�}B]��]/����y���.��Kz��;h��.���y������.��Kz���<��������~|v�����D?<�	Wu�}��.����O��zl� ���%	72M%������U<?~�]�o���r�\c������	�����
��g�����|w��8�\�{D.�W	*~�f��rg/����;� �y��b�e\?Z]����q68��s�z^3�~���q4�7��n����}�����c:����k����M���~�.�KF����d�i��w��d��K���d�.�Fx���%c7��w��]2z���%c��]2>����G���8p�.[�h�'�����Nt���D{���%����V�y���S?|�%������KF�v�����[����bxj��]2��B�����y����s�%�w��]2�9���??5��/�}B]��]/����y��w��]2z��;h�w����y���%�w��]2z���q���rs���~|v�����A?<�	Wu������+�����s��^-�_�E��;��\�V��.��l�$K���,���E��(��4+�xsD�f����xP���NzQ��q�>-/����{R��'E����7������)Z�����=*z�����N��G�&[�=*F�����h6�{T�����GE�Q�{T���L�Q��8�;��=�Nt������rG;?�%m�w�[�hW&�{T����7����[������o��=*r��8}>��z����S�Q��z�����{T|d���GE�Q����(�������x�����z�n���K'�GE�Q�{T�A��G��6�{T��GE�Q�{T<@��~�����x����e�T\��yL��#.��x��]YTOex�GE�����Q&N?�����@(.���SB����w	��<1b�2���lX�3g�7k��"^l�/�O��~q��\��������{���p8�UM�E�Zp
���y,�vw_�.��=���i��nd���N2g���)����Z��	�����t\���x�t��tbi��-�"����.2�����z�M�������L�"�l�w��]dvs]y��E�w��]dv���E����x�}{���C=�"���v~�K!�D����L�w��]dz+joE�7��7���]d��1�����h�q:�|Z��.2+-��6�Efk-�.2[i�w���:�]dz��Ef��Q�a�S������'�U/�����
��Nz��E�w����y�;m�w��]dz��E�w�y� �=.�s]q��g����������pUG|@���.2������<E��Ub���9H^�{�El�Z��_Jx��V����8
�`�$�j��5��Hk�����'�OsO�1������:���1���c����}W?]������u�}]z_�����u�}]z_�����u�}]z_�����u���x_�'����u��fz_�O���a��Qt�+������;���.iMx��rG�2�����u����9t�h������u���xkr���������i��������8x_������l�}���#���u�}]z_�e�G�P�OMg��w�PW�p�w;7|^:�}]z_����Z�}]��q����u�}]z_�����r�����u����].������c�Uq���������z*���|]
C����z��~�l6����M��&)�{{i�������������?�����
��,�4�w#�Qx\�C�CK9�w4����3<|�m���i�����q����w���}:�8_���@�T6�I��.U�F�{J�V�@p��Y�wl���]��~9��hRk������.Gw��C���r�����:��&�	��	��� �������fs��}�?
���h�`9�4L&�4_�l�������1�����b�l��������)_I�0L.���:�L��F���L&�=���=���Y�lw�l����:c�<X_����6d�1�.`/��*p���F2���u�L�<�C��5<r�Wl���2�
\$�Ev��7�p�ahn�k4��@��5��������1�Fw�������	F�S���"�Q����CU<�Lpe{'�A~���+�sE.�Fz���K���t�������
a:�P<���9�jXG�C]G�d_��=���2�nL�� fjZ��C�i�-�F��n�T���6Bw��PH����!�n�:]���'
L��A���>7^]�
��\���-��@�~}s�R��J��r�������t�L�����u�����&��%n�����g%���Y�"�u��EM��*UKL�Io����3��h���:���$E�����A0���Mh��}v�u�>KA��n�d�g)���}��+`������tf�5�,�]'g���C���o��$���7P��;�r��q�>�h���
��N��l�isQ�F�a��>p��ec�F���r&.���r�a��#p��+;�mNv�Y���+�-+���Ch"\�p�^LKU�K�-5#���������	���|#j	���19�����]{:�s�X��u�8�E���R�m@a�%i�&T��A�e*�l��t�dE`F��1.G�}����4�y\S�<A
��A5NWFy�������]�jd��]�n��t�M����t5��O|z����h4�t U`�<�t�TD���hvv������*�l��W{��m����������A����>Gwu���`Fx�@�����w����r��]f\��KB-o��M�L1v������������h�y�
8�������q|���v��E��o0z��V����~`m�~�Bj�������m�g�TC/V{z��
���Ss�.�����.�ss�-!��T��#{@z���sKi����6��L���V���������
)���uf+K�f��
I1���������
����s������!�qP!Z�a�������� �:6�Q���/��`��<LA��^N���u�F�������+n�jN��qb_���T�%���e
��b\�)pXJ�=/^��y`)
-T���Z�Q����![�b��r�Z,�0lw���a�@KP:������1�`�;J��DA�lr�K��!Z@:i-k����_����2OYY6t��.������;��p-ci��Z��N�����A���B/)������>�����SS"#������R����(g3Yd����*��tL@90}���U�����x����]����Hrc R�	>�<��U=N8�-
�P����CaF
g�p�=n�����v9���Yp����L��g����q�����8��'����I����s`���������=A'�l��I�5�����qF}|����C���8�I�w8��������t:���l2��ig���
G�h���1JG?���
����to������v����������=N���h:�L�7�����x~=����<���NO��p6
~Z���=��"��w�a�����Q�m��A�g>f'W��_��6��o�Z��:�������
��\�����#*F
�4/���������|����~>������������hm����O�n�O�0��ty�>��z���r�l�O����7P6���_&w����zf�	�^%��I�N���`�,��vb�����/A�%R{�����$��]�����bKkH���c�*	��{���eM6��5�G]B�W�����p1��J6��&�I\-�v����;��\;V��}\|B�e��'������W���,#��ww��d�mZ��A��$9�Zp�=L���5��e|Ok�������R�\!)�$x}C����1�E�	:�wZ1s���/A������	�������{Xn�K�21��������k����@9�I���]cX<��t������%'x�����#�����8���� H�w�"����M*�������6�QHV��'Ap�&]hH��m/&��Ap��!�J��'��e}�iF�^?���g�-�����5N�/b����4jfjf��?�r�O8h��Hd	����V�%��,�����&,�f��U���D��,�q�p$��%���M��^����j�M�����n����/G�2G"�R��hO�y��LnJD���u���o{��}��hY��}	���h4 4������p�RR�gZ]V�	���&T-�7W����e�e��lD��`����1
n�D�Iz�3"��
�p6,lQ��'������>O:|H�%���'w�>�������C[����&�-�M����&�F��^�>�Q�k ��g2�=c�OB�_�s��qC]���K��x_�8j|{�9�BJ���%$B(���W�<	d�-I�����t�
!�0Yx_�r�������4W�b��'>WpI��XP���X�����{p�P4c����9�b�2Qo����'�6x���.�/��:o�^�������w'	���%����m2O���D/�G��o���+����
���G�!M����F)���qT���"���I�t���\��2t�8���m!�[30��V��4����w�yz&�=���������������X�����h"0����w��X^^e�$"����tdE,�����L����.Nz�x9��a2�����h�{hSA�>�FC�9���q��H-��
�T;����n�V���8��b�G�
p�������59y'�F���U�PO�v� �������1������)����������&F���p��t(��n�{��+���e���PS:C{�M9`�����D��d�h���K>���21����BF�&f�'=J��s��w�}'��/��FC�Kw�<K�3Z�ht��L��`��3���C��������7=>��zTFV"R=j�SL�}^���
#��HydS!j|�T$�L���c�	����I��A������k�����-����k��-�$��	RgS�n�yh�LG���ek����1�����:I�
&������Pn�����#���)��G`�Z�UG�Z����jc�N��s�������R�N
�<b8ndQ�n� A
��XK�U'�o���J",},��y
��&\mg$]���F}B	�'�AtH3Zi�a��������I��������*�7���Op��u�"
�IE0��2 �}�����R��'d��$�K�,�(�&�qF���M�i�~H{}[�USb�6�T��H�����m��-(?�O�R�NS���
Qr8\�)�I�ElC���{L�c�(����b��Ym����/�O�l�IET^�$�"��y�}����&��Rv>�C���K��@�#�J(��<f�J�������:]���.�"'2T-��%�p�����[B���x)za��8�<��BA"�T����"�f�FD$��H$��t3��*���������x������y@���pz6	^=jy��-bs�;����_3f7E����<;^�2<l�����
�	�gPr�Q���u��G��*�H��f=D��W��qq���9��9|��������-^	�
gh��4!'m������eB��z=,���?�k���&����������f2�LON�Qxs}�p8��L��C��P���kp�E4V�g2	PJ������gh{]~�$1Pvy��~X>��L��-�3��{�|(�tL��h���������������]����4����E�zX.�I���R��e���x	�&��g����+�.�"��Ux�l>&�����U�����VM����
��"SnVs��w�����z�p�
�
��Eq!�p�+\K:f��L��	�f/��SD�E�7�7��+V���<2��r1.�:69CSed�LK�B�{�q�>���_�1@�P��W��Fi��LD:n���<�}xH�*��>�"�7i�>�Q_2�>���K"%C����^�dy�+@���U3�nU�b*<'�_K�h:�}�������/R�������,���$�f�T8��S�c�i9����Q���F��m����l���x}"Hyb"9K4��d��,���7�`�{9���6O��i���|�&�|E��z�%/
��M�mdv���H$�FCa`���K%GeJ�S�p����l93���_����B��W%F�`FB����r��"��f��,��7s��[���<�����>��|�J�@7���q4c�|������a��h2)NN���q���Yf�:O��c�m=*�8�����c	�C����
>�����K�@��)[��`����rF���������|��r����{��?^�����������?�������NoIJ	��T5&�G��h�X��t
������<}����?~8���R��������o��{�G������!`A��7�K��q�7����������"�*�	�:a\u%5������Hv�u������j!-J������t��R�Z�$*L��b��)�����*������[.���q	Yvk�z��J��|B���"-��8������	PYy�A.���\�1..�����2�b��)I�R.^�\��Fk�J��B��r�
�����Li��}q��q������h�$h���{�SQ ����U����B�B�V
=��
#qVP>;�����>2�D������
`�+�]�a���3Nu��g�|/��_X��
1^�r�P%�L)��L�3���s�^l�/�d]�G��.���x�q>lSqI�7�����@�.��G\������^����^��e���ehu~��z����A����O�X,L�e8���� +���y&�Q�M��5��3yrDj6<�����"��GO�3r��YR]~�x~������Vb"+C�hTA�-m��d�?�Gl	h;d�����=b�k^+,�Q��&��3a$b^\4J��`�9��7lA���T���2B&9�}��3�[�������������6���~L���ytxl�F��	����'��	���<�2��:����U�D���*V"�G����X&���[��#�|������Z���/�v�����Gr�����fv��G�LJ�E�����(�	)S7��:�V(;���M�%##)&��K�b3�/���T����Q��S>B�!�:z�A	6#2w��zr�z7n��X�O��"���!c����RQ��[o��y���'#��cm���q�F�8���.T�^��V����x��v�G�a$\���+��E���h���/���6������r��Y!���@��G��
7�F��^E�N�W����v�b+)S�#�����_q;����'0�A�>����/P�f��R c\�q����j���gK�(?Z�e[����M�������
��t�����j�<�>��������j�%�@�Y3���"q#�
�X�8C
S���$rVDjI��GFJ�6�F���������k�+����X�K�
|�?`����6�<L�~��:^�"ix�8Xf'�;�!�v������lP�XBFzd0Df�7dQ��s�����y=��d����L��A��9.��H��7�f���A�E
���w��F6�Z�P�XU�[8%huq�{���C7�_p����`q���{�+��F��"%�xg`����[�!+,M����4��P��2<6�M��[��H���C��1NM�����M�K#M���=;�6	4�)�TT�����EZ�D��msc�n	�'/?��!��!�[��7B�m��x�'GV=R���W�\Bs�D��,�0��i1O���zL���e��p���E)�����4pO������a.:�Z	��&m^����SO�9�*����u�w^vrr�XA9>{&&;!�IK=�A�m�\��
���%[4�X?���8���T��� uJ?���Z������[^4C�!�**{9��#,}�s�}�q���@���������B)���Fd\���k�R��q�����������W	9���"�ci�n "�2H�eH�'��� c���!��HT�\k����Y^�@=AQ����(Xai����<�p�1i
�s���4��1w��C�����l5���j��������M5�Q��(j�5��-n�\��8��9�N��+9��b�Jd,����L�Na�2�����D��Y��3�;��|�����IM��5TH*f�1�v^*_�����EJ-���z�u����$c>I�K�A�5{$��^f��
�a�l��K�����h��
��K������r���s57��j`n�n/r�J:@;����s�����<|�������H^!���u���y@��N>�	����j��a�87�#^O0u�����<��s���k�5�?����5dG�'��R���@3��s�0�\��[lH�v�	��X����Re2x�4S���� *�&��M}��+�\��G�@ir�F�LQD'Br�f	}�.!Dp�\���\c��&�.X2�,������		���P�!7Y���06�Da�|�JR���Q���s��P`�M�l������.$`�+B��q(<��J�4'2�HZ�`�y1���6Z�&yH����<d��N�ob�w����\��u3/E7��V3re�,+J{�k���59P������T�.EC������b��Ze�<��EJ�>���k������^�0j�x�������&�xHiW���d��-.{�����r�\2��X`���������1��4m.��4�hg1n���re���S�f����\�xG�,,�P4Z��c��Q�yX�h���"���Tq#�*0��,4C��[J�2�t�������83R����[��C��q�+S������Kq�f�\�=��[������A�����r��;���=�<�[O+[���rUFwT&~�/��Q���u��fj��/��v�QB���4��L����N<��@?���'k��7�F�&�����U��2)�.`g���5����f��}��h�g�a<�������>������c�}���
�e�Z�Uz�E�rjd�GK��S)�����}��#Q����&U�P(�R(�cp�.hy����X.������[�N����x
#l�'!�nO���$.���\��x�����X�-W:���EYet"eu�U��2�s��7:�(����4�&�x!.d��/����
F�_��?RMQV&O�&�D�&1(��H�I�Y�BIFg�`9���_�����gi�r]��g�X+������X 6��e>���7� �E��1s.��I���*�t���:�|���J�y�6mz����v�*P��w��G��d+�
:��*�m��@�u)h����U�
�O��������k�-w[�����r]���r���/8[��LI-�"%�w������(�Y���O����iPH�����V`oQ�:-K@�"<G1����CJ�L���cs/�BQ�R�C���u\m?3/������ympJTp�p�U����T��a�����c����@�J�G�9�����P $�Nr�F��q��qQ��S������!�����T��f�-��Z���dUH����������lA+���G����/T�g�t��/���&��3v��M/I,����=S�$�	�w��Tb����|2��y)�n��m��~�x9�X�X�����bn�PQ�?;�������r�^!7 �����`7�1u���*2ufM��hD���8ac�O~r�e�g~�#����
�w\'�-���>I?�-�+�M�z�����_y��x9lVdV��N������$y)��4}�:[�}I�1�;�%;b���g�_(��|U��%zK��EK����.���9�5�I�
���$�/2,�2���O�O��!�U�]R��CoP�:�6��6|����/%
�T%V�?�S8z)
������Q�_Z�x�{��6&+�O|�}�]/I4����P~a�B�=�Fb���W�uY���A��Z&ij�F��
8���������
����'JG/����v6����xe t��k{c2#����x��kq��,��"3��C��y7���7w��|�J�Y/��oJvYW���W���S��X/����Ur-MQr
�;#������'�@�\T�]e���DT���V�P��r��U�6H�{d)i�^�6E�9�z2�bM5�j���"]��"���=H��"N��G>���);$U����D���$�{�W����f��K��)�+%�`�	r�6P�W����ly`x����J���K�a�c��W�krOM�q����=� {X�<���\\��H����2]b�,%��]d�,qH���8�%�������N9���IX��	)����>��'P��s��pC�V&�t��X���[c��D��3,H�)��<���'�����P��.7���
�O�ZlI�+�����g�&H�����l�y5`gX���6r�J�V�04,�bg��"�Q(g�r��g�@���qf��g��H�Vp�A����L�k+S����0sDy9����!(��'z�I��b�Cg���,���g��0r1F��Lg`��s#�)�v*� *���t�e]u+v�6�sq�w��Q�k�%��R����}�085N���"�j�G�-U��I�T�?h=���A6��b+�7���d�����g&_�����%�?g��,�s ��3������L������J�b���k���l���B�:[/6�XM56�.�Y�BF��U;"e�����i���z�y���u�!�?��4J���pQ��S�`Av���l��r)&�XUa�h���M���	��eJ�������_R����&
�b��"U�(r���~iU�@���2�
x$�e�g7�����&Bb��l�WTY�A������"�����0Hk]I0���@fz�r����/+�fM�}>�Gv�)��F��&I���]r�F�DD���<���j�l�����<*�!�d�xO>�}+%��i@�A�<�W�e�U���t7��\Z�}),T��5)N��v�������YZ!�j�������5��V�|)m���\#Z��������k�Y��L��-k��J�#@[�i#fVI[g�um��9��3rT����/��~�hh����U���&I�<��4X�����6-�I�(ai/���B�N�.��]ZE���vJ��]h|c���{�)W-a����iqr��%�WTJlB���h#��{^W<_W�����J��L�{
������k�:�V'_d5- u��U�|aU'_����CYm�LG\����6	�'6�eN������0�$�d�C?q�8�
	�P��%��r��O��5����$�!C��o�1��
�:3��8S��9����HI�*��P3��J�^�rRH ���r�{	�YkM�T��4�\�������)�'Y/��pX@����?c����!�3����T������G9^����/M��wgWl��z7s%�G�q�~7����b���c)006@%|����&�=�RoJlS��L����X������Z����C���~��f�kM��O	b)���q9$���WeM��s�a2�T�)���p6��BW+U��������l_c��������;�"��(��5���d��G�5
��A�gX!Q�_��g����pW��T�U�x	�	u4@����0����)���-��J��_lU
TScd%�M����GN����H���vR���_:�%�{���1%������V��E��2��/�=3�:�D�pC�!
��j�%�����*V�5.\nr��%I�L>en/��������$�+k)J��o�t���'����ie��G���f�`�x��F��,�3U������ J��$��A�:���Zwx�
�`a.cy�'�����t>e�"	Q���uRY�R�
 ��	�\)���u��Et���v�r����)���n�]�R�{i>�Q�@�qM3TT��u���:������x?�"�F4�8�����i��yY��V��j^��>�H.G�K�qy[����(NV���U2�2��-9��7��
62b��w4��MwXI<��d�+9w�fP���C��|c�++���T�!���<��	��V2�~��j���_|s��!�%&��
6z�2�5A9����]�^�X�T�
R�yD*C�X����{4���z
N������T�����C�/�(9����:�����/a�CW��z���������"R��ZD��P�w����o�J*��������<
*��&��[|�<��l�I�9�b����������Xf/y$�
m�4I� y��7)��k_,N�o�L��&���������!v���g����R�Bf\�'�,��]?�$%�])M�0�Z���pM�P���r1_l��N�������iYX���������q����u�O]����Q���@��,u�C����+�wP�z��W��T������U���'��p��Xh�s�lso��&:h��]B$h��W|qvDt�
��f�����
B��
�u.�_�1�a:���TzB�uM2��I����6q��iec�l�_�:�J����Kb#��m6��~p�#�^�P�i�����L?	��PwGd�����m!��Cqf� x�V��vy�I�I�Hs��Z���d1�`�`
N2;Vq�����������j>8���$L+����2��
OP������D������xh�/��{�����q(�m\6�&�������j#�`e�?S&����jwY���P��x�|���"Q9iZ^����VEQP9gu�F�*�V�������M�AG��d��+
����$�P���l���������&�+��%�d��2"[��Dc��9������{��!$mr�H�IY"-������&K�)��3��-y��!l��]c����T$�|=���X��c���x�h��PK0�������W��}�T�4����W��1��7�&��i�z"��������]�_C0J�m6�Ld����A�1���
�����$z���n��5������b�L�\��K<�c ����:Yn���Aa$�������+E�\~{��n>F�������e�aD�Q����q*-�T�z���{��R�(�s���
Q������&���B�=j]��&�9�\#�h@]�mE��{�����?�����u$g�m�
e��4k��Y���hK!�e��-����bK���:��e�%���a��,�8skp*�������!#8&Rr���Y^�-:��	������f�����"3>H�����i�����D��4b��d��~�eG�B��������O�Cv������J*3Z�a���`�G��+��]��d�
��X�H�$����d(	��
$3D�=��
��,�����'6z����xJ~k�<_���������kV����ts[)�$�I���F<\#�tI+�A.�UZ��g6?R2�t~�"���]���<�L<��i��b���g(����l���aSMb��>���
 �j�vh�pw�lL3Q���������L�Z��Ye��|�Iq�9>qv����!�.M&��x{�-�9�7�����=$[�^eK*������3��XS�2�o_��$���A�|�����TI}��fX^��)��kR%�M��-�=PE�f����g�Z�����*�!��l�5�{���d��
K��RP�4�X�-I"
��D1��1�k�!|d�!��(�T��P�B�y*=Hk���X���<�P?���g*Sa+���"56bS����������q��q��b�l@(���<�6����f�s�UP�����^(���I�6q*��5��n%�7�$U>�)�19�1vM�eI�DN��Y�8�rF�y����dIg���%?c/�I����������M�k���?/c�xL��O��O��	L��cr'��m�!�y�1��']8���\D����;�*z��R�.�Ro��7L@Gn��#-
:�n��z!�)�$(���B�\	���*ddB��R��9w}/9���Kf^E	4�6��B��{@+��&(�:��P�s3�9=�$nY�y4rir���K�L.{��3Ii�����d,�7�%��i�����p��[�������5�v��}����=&o���dgaP0z0�Y���J+��sM��JD�����H��/U��[��A���:��p�������D�v��&Nm���S:�;�B,;�M��BF�EU16��blA�7I%M��2B����6VO��>H4�A������=��!�	25"F+Zt��s�:���	�*��wU�H���r
��ap��%h�"�u9����F���tU+X��by�-����(����/Y�KUJ���4�$�t�. 4�N�������(����$P+�wj��X�N�
{jT���D���u����A'+H�d� ���GA��N����z�7�[Cb9�ts)�,����_�"S�K"�@9e�D��\p��yfK��)9��8�) <*�A�3�+8�����q
4[>}2����t}���G����l�}�5p1Pyl�����,�r
4}!��m'Sgk�@�X�_q-��f��{�1���*�OD�0�
.�o��>P�*�����rm�����X%���7_��GC(6�Y<�
��KE6�z�F�"�=�#���:�tH��=�kY�����2��*���/K�����h7L������b����|�ag���AM����[�,k���ZEtP���3���V�}�0������3MAA�Z�1�.i.U�+;*����u"Xf;����pYd��R6['�^�����PG�p./�y�������Y��m�(���`���P�CV��0\�~�X� ���
��)�������^/%�6���d8E���rw�dmC�Y��S}�4�q�8�oX���CT3d���~)��B`��!vF�\�������1^F]Pb�h��$��N��b�UK�����0�_�JT��W��@�
�J���K��63��=n����$��=L�a�4�~��������k���
�2
�] �H)��R��3��BW��u��=���j%S�-J��[T�=�hD�5�.�6�i�b)�b��8�U&��Da�8���zpK6���<`-}H�����R�����_*�~S�8�����	-z���b��������&��j�/�D�u(��B��"0��	5�	 �,�"��T@���X����B��!/��Pe���`�dv1�K�D%Eq9*�R�~z3�9EHM��5X~�s$I�����E�9����0��qb��$�P���94�b��)�c��v�bR~SU���D�P�0����Z\z�j&��y��<�7��I���	kyA�T5b+�v��t������jQ�_�#��*_Q��(�L�%����pI1a ��
�������:[���! T5�J��
T�Q���H�
Usp�R�a���z�����UY�RC+���� 8>��9�������x�k���z}�E��'�������x���LO_���m���������8���Jn�p8������c�{/^�p��_�!���tE���i��DK
X����Kza�W��!n�/�R�eh�D�^@�D��1>�'�E��\Y��!Ym/�,�k��j*�o_�_�����;��!B'A��f������k����mQ\����n�\ N�������
�����H��~|�7���/V)v���D��Y�?��b�D���q�{���] n|3��Bva������t��oo�GY-o�'��5Y�$�<�)OVAB��S����L������_���)$&�?����H�q%q�������m���U���d
���${Z�l%��+��n������,HpP�n��~��1���)cl�LwM?�������J4�@&+�k�"o7�#5�+b�kF�-8������GS�NU������El�1;A��\�4x��n5(4Mt���{��������6�G����6�dh��s��e��� ��?����5k�6D�wV�UQ�U�4�7���g'����g?�z����9���I|�����nJ��t�����0��mw�&TWo�����<���
Vn2s2����
�?P$�n�"��,��%}���� Y}:��TK�uA(+3Pe,�M�D��4�	��b�D����g�����3*����a��w<!p�4�M�)�)�^^/A�:����y���������K�����?��@��XK�C	�QmH��1��c������,����������!���c�4�>N�
���,6e?X(%�d��e	?�(��O?�����>��TY�����wk��<PyS�*\��#3�F#������RKic/&CwU�E�����
S����
�?
k�?V��o!���*��JcC������������)V��@%	�,���bS'��d����[#�h����4���9��jD���w&� �z\E�Isu�8����1�Z��-O�_$��h�������U����%2����G]�,9�Cf2i7�|+W8�'
�%�D:
0;�X�m}��/?�]��t���E!�������?S>U:u��m�����������Zl� ���vW��G{����\
�mK�9z�*��5���3L�;�@�b~)7"M;-�>��
B�7�1&D����<�$�Hl����t�d�F���n��1�#��5Z�'y�t�II���r�"d��^�'������|�����0R�]�^�������Mu�RJpw	{���@|~�����l��%J�\�����	���c.��!�R�������9����_������*1�VC��-�n&�������lsM����@���W�+���BPd������UJUML+�'�����mU
�AU�R��Z>Ksq����"�
IL�d�M@�����=���%Z" paU�=����e=gw����;1�"�qH�4��V��l�����������
�S-l�,��KH"E7K��r��h-��x�F	�z0��$YLY���r%��AU�z��yT�F7�r�-�d��G�M�Z�4���^��>���[�K���#���v�d*�e,�2�:�Vx.v�����D*���@�>�[[�afE2�����B���A�2H�V�f#�*@����J���x@�!�g�&����!8���G���z�a�'���������y�]�S�kE�r2MYM�3I`5�q�r��Y�Z��N l��������R��0E��u�i�8��-m�;��x��!.����*���z#��@;�YTm�,�V�uBgjl�t�T��0��������"����]Y��#��d����E���8�f�p����d�^��Q0�V�j��*J�t�L�#�����b^�b9���EZ,�&;����Y>/X�p�����>�&L�
�$�����?��;��@T�uz��O�t�>/6�|!1�.�v� ����Gr�c���3'Ae-W	 <�2v���U�8n=�������V��$���$6�xM�~�v��[T]������?9�����,�����.��zt��hD���$�Z��T��e�XS;�S�;t�z��c��_JY]�"6{(0W4=f�|��GO���n)��������VI�"5V���ZT
�������B{�#|DOR�Qbo	����EdQ���3)T��di8FH+"Rh5���]c��;�9A
��NW������S����y�@��Z�`Q�-� 34bhh�W��� �^���e��C��E>sv���n[�X�/�HD*zF���"n��mq����~�����TU�`��,����e���B�
M�a�l�]�����CC@�"�F���f�R(�1��!�����4��\����gWa��1_����8����z���W����23������J�����L�f$��Eo��J1Y�������A�S���>$�D�X�}XN��;��yJpV_��Y~|<��fEj��>�+k���9C�;!����^�0z<����{F�^�������w���~&��������5�E��l�i����$?���7�	4���)��3/������4|������d�������=�x@�I}!E	�B�f������Y��Tu,oU�l��5�zr�4-�MU����
�DUyK*��L@���
X��T�D�2>��:+��M���PA��]��Y0�����@&Z?�B�s*�&�a�����Eq1�*3�$f��S9�pv��������>?��\~�1�'���sr���|�:~��G����!�%^�I��,��C�2���������C[�A�M��4Ni/�tWa3��P8L�B86��*}n
���zA�����c�
����c�,��}G�M9X�I�t�JEAJK�(F�w}��$G%���������\�o����g��to�D�~�����OK��{t�����S�50;�EZ�6���]����)�X8��3
��'|�F��I��N�YF����?��o�����,�+������l��w�ru�����e�����l:�����^��2�����?��d�f���D��uq��x��?��#L���<B'|��w9A����)��W��W�����1��?�.�������n�d���#����������#���O�<�%��"���?Fn��;=|e��8n�\��y��iY�Z�W��pvy�j�1��G4��(�*�xHI*C���b>�lJ>�?i ��%�x��r�I��KW\q,���)�+.�k����

�-m*��m�G�5��I_\A����t)6�B�;R����#$�Mno�>�Us�������J���(�����!Y�A
@��;D($�t^,
v��>r�Wm�����{t������q>�F�R�����Jq��
w�'e��X~4�e�������3"b��a��U��lyR���r�I���f��hFU&�K4��"�G�4��h�����X���iyB�����5U��#����']ml�`o�,)�U�4������Z��u�;�[�����Ek�@����<5������M���(�����y��)]���l{	��<��Q��W��cB\I2�N���D($/I�b�0����f[��C�
��t,��\���
�'�g���z���)��A^�_�����\_�ol�2$QZ
��tNA��x�O:j��\����5�*����F��p�T�Z�iJq��j�����bU\/WW�<Xn���Hp����O���������?{����>ys����g/������0�#�d��f8[��x�R���@./.�����:F�������ZRP��A4~#*���	F�^0���Vt�S����V�P'H���!������}���
�0�x�d0�x������^�d)_�,��0���y�?�]q���X'a��nY2h_���6^�)������o�sA��N����;�����-�u�B��F��&TU���l�p
�=�	'��LYK_�Su��%_|�������9���[��[��%�#���&��l�l���7]�m��B�#��cP���,�m)j�J
������j>����D�aa�/L�X�D�H����&�I�x�Z$���R�i���X��	"����ZV�]���mQ�6p�@��
�M)<�1��$��F������mvdL�j(Bh,�+y+$@�IF����K8��9da���]0s���gz!b}�?rt$hH��,e��]a#�)����Y�Hr��"�C��>�r�k� ����Es��ZG
��l�3>�>��}�`�9�O1[�J���*���`�+���3��9�D����c��z1�b	=c-����(&�<u��Kz�zsE{
��l��d>HC,*���8j�J{@#����OG��b���%"O��'Z�7���UF+gGub^��Aa�	\��*�|��n�+�
�@I6��a
��N&�$�`�<�]F,d�R�m(�P#df�4e�
[&�mE�U��KO0Z�8Se�:SE���L�oW���w��e��?��-��	9�\	'������v"��I�Cxkh�:9>?
�����_~:����D������/L�����A�3y.������U����!�k[(�>=�E�����d��bI��ws����!���Ip���n-���4$=�DR>`���:�P4K�$[(�EZ&��}G}������f�pK#^Q^��+e=�����];�J�h?x ��m��������Xo�=�������G�s��_-�=a�h;����o�%�t�����(���n�����������b4�������(�N�����?��}���jo@������ y�%�����(nR������#��3y�������o7�d�_�o�������X�V�a�L6W��X<�LA�yu,C���>�t<�8N�zvx��/tP���'��!Y�:�"��\k���?����']���Fg`iz1I����$M���y�:�[�����C>�l���T\��|�#��i�U2H����#B�Y���'�NO���7'�=?E��	������)�������v�k����(���A�f�8��K�_>e���|�(W�=}��'�����	����D-p�4�sv����3x����@�9�a����fP8
������Wo����������7,��?#���;�An�[_=�	����Z�T����X��dE�L����x���A����g!
���i�;;mp�G�j������Fs�>g9�����e�V.��%�rX.���R�\�`�R.��e������/6����Sc�/vdzL&P���d��5��6�\1� e�`c�&�#���p��p��.���d�mDk�'Zl#6��lpL�����_'Z\�'���D�:�������hKi����Z����E��?gSN����Ii� r��]����\*�K�V�er����(3�-.������dI�	h�-H�kqsb��-F�jX��)n�X|��������Z�v���K��v)������2��C�
K���|�|������'�^�������&���}$0`��]�;#��������k�Bb����dC��NJ�A�~Oa���W������%� ��~9}�DZ�����ul�B�&^�����^�6G� "�?�CAo���GqT�n�H�j�}�7�>y��:�Zt������������k�V�����1,��}(��^4�>�6H�/�$���W�C�� �t� b�d���w���;�A@�3�;��>3��h1mh�8&��-���D�%x$S�Pl���NF2�
�I����d���K�-�	�� R���/��!�
N
��k�������
�j�a�]oa����Na���U���&��@��@y�h-Lq fB3�n�al+�|B����9U�����?����?�J}Rx6jW,~�'���`��$������A�d���~$�*�Y���?��Vj��;u�����%�d�Tv��:DJ=��R��C�ST�Tu�<������CVt�kV?������O�'�
M�L�`������rQ�H9�*.�8��qe�
f!��B��(F��J����P���'+��w1gh�o��KcCT�j�����G\���y��=�o��
f�*��������S���T?���V�������
��F��r7;���`
�.4{/����&�ko��g�mw3��*VR�B��������%�,,&6X��M����&de�j��b	�mr���	e���26L��c���ps�07���������0���i�/+A��������/h�7���w�}PWB��'<�_��%��N���(���B���!��
�	O1+��������N���}��HyRV>��g�|V��X��l��
D/��AU~���V��O1���Z8 ������n��(������������b7X	^+�j�J��t������N��!OfyO�N�OhG�PQG�RqO�T�OkG�VQG����4�H���:���<�����J�����=CVg(��:#���I���F�=��	"TQ,I"vy�yM��<-��A�N�C�]���?�.%{e��WE0;VT6e$���������r��w��Q���:p�q��k��c���}������G\����S�o���-�R�i�������YV��hL��-�2��-��*�<�V�s�VD����
�#t&:�Eh/��B{�{�2�����h3��Q��T"�!��-����������?IY���3`uP�5>(�ue5��:��GG�x�z#�f�|*M��P�=��4��Ci��CVs5[I�p�`a�����x�\@���j�������4�
n�2d�������[I��V���C��\azp����P��T*p�jA_9�s<�3��WE����m��&���������m����U��d9$�G��!�� lq�w"��/�^ip��1�+�c�HcJ�u���1M�c��1
�&@`��]��o���9��������Zxd=��F�L�+�����:=Y������|��{�Ceq<T�����%���E������js�L�;T�RR���������nVfM��$V�:���<�{�M���c�?G� ����n������j� ������@\{�C�i�Q����ND����A���U�{3�,���<��5��k��l�u����!�H�f���:)��,T����o�7	�=���r�h�����Yi=3��)�����'>��,��b�I��M�,��e	���YY����,V��%���O��+�)�[�L��'Q���0�*t�o��}~���W�O�>/���Q������!����D(���P��Bmy_��~5!�}U�U�_iR�:��H�Q���{��l�vdV��q�8�����E���^�X�0��g�m��4��;y����DA������0�������y���9�5��
;��������22���j��u"��#_I�[�n}�
rwj��s>��pk���~E7/[����H���L��L=�P�����i��L�����{�Wt����y/���k��M�����FT�I��Qj�����0�G��w3�yO�aO�IO���g3���iSDt�MO�5O;����-���V ��J5�z�i2��a�6���e�}�i�!Qv�n���������+va���<"�^�r��5������}�*���;Z��jys�WPj\��v���bx{{�`K!�Q�QS<*�M�������bP4��P�����d��w1�CH7�d��*��-��R�,��96k����M��e�I�SY�H���*�:~v�:6��c�'���T��\�Oi���<�b*P��T�s6`��{(��*�6�TD��� d����}��nS�P�pN8WD�;X��p0w��I|6�{���]
��%^���
�� �������d*F'
������r�n�d��Tl�1�%6�]bC��_9�I��-o(9�������X�.��azG1�w]��h��E_D�x�/�\m�"�Y�����}����G�3H'�1��-�7�T������[����Vw�m�����g�U�h��T�|5G��|6��gSJ��N������M��/[���U-R�V����dW�h��]#)��;���Cqx+A�:E���]-��f�XE\i���,yAu���%U'>����y����=%o�#@��������4&��C��V+��|ZbM�;�>�G�����jx���Q�U�P�:��|�I�Sv
lcg���(�\��,�
�hB1�BH�=!���a����q�x��`Z�����T��a�m{���K��]�E�(���"|�n��p�[����.1Z����7��c���3���&��:�n����Eg�p�v�����J�C�����Ai����>~�S�����G��c^�q{�Eh�;M�]�5??ys���9�����MPg�G�Q&�l������w��#��#�zZ_'�Kjh����[�����mN�@bz;��Tj�q���� �A�Rdg*�����c�mG��`;,4���(qh"���o��jX{�A��
���.������c��Dg���)F����+��-���]vFq�������
���.��>�����>���,�y��ng�C�-!����C��g�I�o'���`�\a^vIc�Ts��F7C9vke�[c=;��c�GV'��'X���GJ~,�|��z�.�nRM7���9)s��*o(�F�5Y�
�^�7y]-0��y�7���a�����b)�D���|V���\�gW%waX�(�� �G�����
vB�$�l�jYS����)^�*�)� e�6�f;�I�xsBT�NH�
���&��Be���B�
����QDrf��F��
���YN3��i����1l/��%nx�Bw�0�i���`�����Nx��������f�;H�v'<
���4�G������������r*����\F�����������jG
��r{�ao��7���|�S������_���x���B4OY:�^����;1���D�%k�a���=��m�����)�2�� ��:�����������K]����pW��@�q����^uE������N�S-f�S�h�������RK��O��Mb��*�jlj����U������0<I}qp�m
W^9O��Fa���@��{�N�G\`�����U���5g�8l5�\��k�������j����B�T������0}E��I}�����=�D�=�i}=������[���r��
���t��?�l������g\`l�l�/���h�������N�����;��|���fd\�k~����_����-qtj��R�"�r���cGeAT@*%N��5n/��q�Q�QJ����r�T����85x��(�7[Ha6wx�T���B���>�K
t8\R���z����0�o����������s4�����W�zM~�f�����M>�9�!dYM�q���������W��E�rOp
��n��N!��������SXf>p+,-#�}��T�VX�d�HK�
E�n���%��e�"�Fs3�w(����X��2����$����).�L@����M3�7+���k��Z�^+Uj9����PJ5N[��G4l�������qC��v����4��RAc;A[a�4��VA�&�Z�0
TI�;�U4&�0�(�M�y��J�b�����Z�>�\7�N�f���4sGr��?��u��X��g<
m�.�������r~��}������ELB��a�����:����e�����.-��av8Sy�X���0g���+B�i��7�r��wF�K���2�P
JG���_��J�x'�r\#Q)��g���c5T2��D1�&��S�C�g�=oO�����i�:�5��x���T�[��kgSG�S��!o���3���x{?�\�d�So�WX8�K�B9_U�I{�	�fh���}3,::���XY�o�)���+d���D4���� �����v����g�e��w����QN���(g�e��b�U=�_	LM�����#�XIY�l�M�#��
�n2�2k\5��ZD�CUsv��wK���O�����T�I���.���g��v��/����e���7�a�-�z��WG���C��eB���*f�gW]���E�@]VmPp�A�
,'�&�d�3�A�B+���/�����B���2g���	�4�3`�IJ}M��4��_}����y�j?�!�����w��!�3k�����+D����]�!a�[}�~0_��R_=@j��] ������%��'�M���U��M�-��J��OL�yWU������
�W��_4p����4���D��p�[i�
�)���A(ze�2��
s�p�u>����y��W�wL��I�����5h�Z�Baj�����)�i�dh�<����rqsy�/��A�M����=i�020t���(X))�"D�����-SD��(h�8����Q��e�G����7��3k3��.��3�sYXj�3tYxhh[���e�IU��"ZN���1SF���pk�M�-'!�L+�kk
��Z�����S��v@f;�U��
����QF��_��C��m��e��bLU��X�F7�W����+��b�>��Y�F���^����~�6������r`$�����p��D���]��*�Q����� y���
�����#4g�����|�������5j> �z5v��M�S���s\��b8���"�B�k���	����%W��%.z��������!b��:tU=�b,�,g7�I3'�|��R2��_�}L#ML]����'n>�Rrv;�E<�nV�������6�����VYt�d���M�d0�?�RIdIj*��`�~���T����U�L[�AY0X�e��*�*���G.�h�h��D�g�UAD�b"
�3S��uwj93Ik�N\-W��l�[��YU8F�g���C��af�W��i\�U��OVk��a&��K�RR��hu��T��:G�j����f&[��*l������1�U���Kb,�-U��U���*fh���4�|`��*eaic��nm��g?��d�M��Q��
�nt�L���*���s8H��u�>���fg���>j�*K&��[�p-MB��*=����+�C�FQf�aq�|Uh��hk�������,dt�o7�,$����j6�d�K�	tj�'_�c�K��P*�c�K<���Zc��(HP������5 �@p�evm������jb}������&;��#�r�^�]u^������-_F0���Rw+���l����m�����vo[��0+D�
���Y��LS�d��W�V�<(yJ^]�tP���%�A��>����CO�u�f����W����"�.
)(�����f���b���Zo��V���V�G+�Y�+�z�D�%(_�bF�3���.NA8�t��Mx������_��Nn��]��$}�%�FO#���+B�j�`(>�%t��T�Y�i�1����-�tD�_S������
��@�oo����.��e�(7���WYP[W�
 ���W�%l���h����N�p��f?�Z?����z?��~���p������'�?}�(7��o����Db�,0a����l?��Iw��(���V�����n%j6��Cj�%D1O�_J4��[Z��MA�V ���n��+`i�An�c�Y�q[��@*�6����6K�; s���]ak��H��H��H�x]���r���;"i2����i���U��`�������^�SC�L��Z����2'28^�F,@�n��k������v��WA��� ����yIk8q+��������|�+3�m�n��/��3���:�UqU�-zP��e��n[�Y�qVH-9+��l���lC�	I	�B,n�������Wqs
7���������l��p#�$�Au�rMJ�����C5���?�"
Z����|}A����{�W��+"�������m�)|������
��A*j9�����k��~rR�f4�O�
e��A������^R=�uz���5��l�P]'�z�t{�5W��\�c�f;q�'���4D�X>+������wG�Y �@�����YSBp�
n��)9�R2���;��`x+���weT^�Ck%j���ace��%���hw�=$�������Ve����oD�@�Q����L�UqN�`VO��(dbH����7����]���+�Z�Z��{��,�������ff���^K�������*�����I�u���:�o#�z���U�dmh�6��oo�h��T`[��[Y�b�@���Pl�E�F��B�������;�������pg�&�e2�y�-��g�o0���C�2j7*�R����*�h{�!+K�)��Fz57��Y��Y1�4<�����,�����L� �+��$,�����Az[:HoLGOv/�pk2s�`�f�)����y/�<U%������+��e�~��,�����y��tz>�k�f�_�W���..�����c���~�2UR�qJ�j�Bk���}���Y��i�7���9g��U	P3U8��������@�ac��0�������U��d4P���������fw�q�c{�mt���
';�W�ZU�^s�Z`���9yfm�i��!������D�e6����0�~-�n���-���)ew���
'P�1I��)�@��Xm�/W���J*��]��z/K������G2���X��8���F4R'[�3�H��v��ju4��`xE�F/L�]��M�N�VuO�?��Q�[b�tw����De����Gq�rm7���[����	MhM����~[r��N{w;���6�V����
Y�e�z[���k��7����w�*�.z�-/.+�H�(~��� ����@��������,
?Qd��xE+gc��5��)�:Ji��%���VkU@^k��VT�0?��k�K-�C
dn:������[ox�y�7Mkp�:%T��e�����Lq�����h��t��J���uE�����Yn��T����c�Em��I�fQ��P&X��u;��"	���R #	��
�$�p�[#m�A�z�m�������Yn��`q0�:��<������`�
����'��L�6=�v;�$X��I�<�$X���	��Uu������h~�)�e;��������U���7����4�q��X�k��wSgl��Q�:��~1�e*�QY0Fy?�����Q6��v�W&:5z���LE�@#������:%���,^��R�7tx_��K�I���l�M2�I��r1��KYm�W������<��RW��X��������
b���04��7:�����[N��7�?�)28����w���`�Q���?b�ou�A������(�K��-s������v1�N]`�|�L~0���u���U�}�gHf�424�N�sSv�*��7_ �X p�1�tS�x������U�q�An�@���0^�Mj?��
��������i?�B������ |��<���Omn��������v;im���m��f�u����'!�����u��Mgva���C�b�}H�o���)6�����9��hr6��=�9Ae��<q��	���l�^g�])��sdGw��Mg�&���%��m���b�Mq:�x�r��8!�gi����V�v��r��R}2�2�2�2�(�?C���1[c���L+l��n��'�'�;A�d0������`q^)N&���9�f����ey�8`��U���&���&'��1���
G]�E�d�A��%���k��*Rv9�p��-�h���'r����U�{3�,�^����U����7�G�c�N@)!�QHh2�b<��M�FDJ�Xx{�r�T#�AL��`{����~��u'�����&�/�M8���5E�T�G	��H�'����'�c���RG���� �����-�?��:�$���������v_�=�@'��� @��f���|��o�6K6�]�����0_3_&4_���z=�
�yQHe�5�����X����@���[,�������^����|��T���O^���9Uo��O^�y�����OYc��|�����/��9^�
�
^R-�A��p!p�0��������g/�MZ�c��3��.��Z�����������o�����-H��Mq�.V�b
����d�&�������������k.�6t�Mw�6����g����/x$d/�<���a\��N����fv l�~���8w�k,Go���9���x�z���b��ah�����^&�3�F���J�iO�l�S���zN�{��9��:���@W �O��xCO��,Q���Y�j���EK��Y��maL��
����j����M?dZ
%M��T�����B|7��)�����-�����.hv�V�N7;e�*���(=Y����2��_����/i���a��&Z�_�.���{�J�_�������'I�P�|�4	�]�8�$)��O����C�'�$�Cu$�IRD�vs�������N>Iz
����|�������t5?+����]7G�k�Yy]�����(��\�<���X�M��S���b��,����b�����x��,��P��`;�:�py�:/Vh��x��1A���w������0*n�:5�U5P�[�����VF�/���f��I/S��n,Q|GU����z���t���t������$/�oM��S�-�Ns�&F�ii�$s$wdM�_����$�DL�]4/3�6�i��x��a�_mZ�Zc	V��t����a�_�n����>?���;�j����z�ew�1[W�n��Q6��Z6p��G����a[`����O������[x�U����#���H�#��R�\����Jz����Jw*qV��DSq�3
�v~���dy����0��F�K�.C;x�����~!����_�t<���r��z���%	$��R�}T*�K�Q��.��G���+��*�5W^U�i��>���\5}T)�����R:s���V��eI�������+�%�1Wk�w��d.�������@��5%���������������8s�:�������l��Ms3��W��l��}��u���������9p�r�F��p��u�/_a�����w> ��6��o> I�6���a���Y�N/b�z����k�l�/���h���F���o;����ik���=�Y,�t�����'�O_?9�������m��8�����|��l��5X��R
V�j�J&���{���h�o��1��N)G#�����w{����{�b�A[��������� w���k�}�zVw|�\���3I���/D\�y7i�Hf��<$+@,$���Dp�	m�e���+��-
]�,6��E��F�%�#d=�O�qs�G��v�R�[q��������7���o��v��&{Fc����2$�
�e� �a=mj�]|+��z�[h�������od�����l��V�z��a�	)|X<�w���t�����(hS�:+c�!x EE�IAzRL���'ED�IzR<����*ud�^�ui7+�V�)�%��^�-��m���v��F �q 1��e������f�F�-
7�������4�X#��6����*,�Xa�.@���� ������rPw2q�|���!��jH�R��h�\����r�p�]�n�����
��V�+��j��{0������L���>�!YGt�\�>�Ag��s8�l�p�i��{=���%��G
��8�H�Ax��t0;���<�Iq |R$������N�V\,w����M���K��������cm������W'/~8m�u��w�9d��"�`�t�,x�a���F�`���;��� �0�%
uvY�����A-�P���kh5#�������1s2����1l��0V[���j�@���,��v�9�D����e�>Q�>QV����L����NT���T>�����f16��`�������(\���jX�������f��.�H�n�4��+�W�O}����m/;'T.0f%�I.�n~���d��%���%��L_�+�i
�^Gj'OHH`I�\�R��J����T�����:�{]�K�3��+������a��a�Q�\�9�v���v6��6��2��s�Se�S�0�Q�\�M�v����K����^�%��[cI�+�}��
'��T��E�Wy�K��r���-��^^�J��Z�exU ��|��Rn`�����;���=��{���@�k%CCx����X��b��(������Y���.�����9JF����4���;W7�g�D��l��������W�p������~�������i*_p%����~��|���>���t��-c`�c�����hcf����8��)������B��%��#1Y����Z��B8�Gbz��@%�����Q%	����������7l^�5�*s�!�0z����M�7���Ld���/�� �E9�Y�RI��������#_0�.o������������������V0!��m�h�6b`�hiq�r@�h�0�6���;3�5��]�T���5�Z//e?;�fh��X-�k
�+��;�J7[^^/�M��H����;�S��.8qt�ap���2�!����l�;w���n��-gL{;3,#��d��J�s�Hf�l,E���M����A���*:M���a��y\�c�����8i�m��t<�nt�Lt�k����,���3���?���3�3��b��4�Q��� �� ��TA9���!���%���"��Qx���$	)��"���H�cdO2�cO� e	�BA�$<a?� H�?��h�~`*b�d�;w9\.�+�������������~dlEr���5����ne_
�!�;Y6&i(������]*)��*�.%�G���-+|<��-��%�G�`<�����U��e�x�~$�B
t{cL�&}�n�}O
��������[�9QR.��np�]Pv�zU�o��=vAu���������HqU;�L�c�y�?��r/��q��J=cu&��G�����4����������!~���NN��Y�mR�1����	2nO�
Z*�[��u�ka�iV{���^����m�
6`:�l���kAB��5��+��p����) �+��
jvh3��_5���M]�I�u1d��H: �%Z�|?��`li*��w��!��NCY���~I�����w`{��,�+�e+�0��" ��B�����Q!��[�w�
K
��-6Zv�A�x|����x�o
/?��jys��H���r3��m���J���C{��O��
�D�������s5!Jh�/��bXZA��i�n�G���7�[dS��F��
D�f]�6�Ih��[[��2����;�����!�;������VKc�����6��%8nS�_~,��G!�h�����]�Yx��7�>�C�~��?������;��; Gu��[��{�--�w`o�:��{"������[�;;���a	Iw���vv]������W9#��w]����^���o�:�J���m���	�������5o�U����~�3��m�{����r��X��\����G,����,��D�:�H�'���9K�V�v�*�k����0}�dY����5���N�������^��%vD���-�>�vX�5��.����k�9��\����=���K�~^��w@���������W[�2���������}�?X����R��v�}�2��V���Ny~
S���+����MjR"
�p 0�E��"?�z�����T}�|��<	�Euf�e�L�T���4��Rm�0�g�G����g����!m��j_�!���U#�dq[����6��m��@"�a�����Fv���x����d'�_gcl���@����q;�S�4��8���+^\"����M���H�	�o�z2�G�����xq�;������	R@�=o�4�a�j��I��Q�d�m�+��I.��~�_X��`6<��A\��l�\#�;O�����A�����T�(������-��
x�h�
�(q�>�h�-+��8u��
8q�xd�Qv��c<P?(`��-w	��y���,��}��L�`��p���I����Yn�*c��9�Y6���x���1A2����v�Y�J����9��Ad�]����Y���kMb�G�Q�4�,�
��G���[,��sk�"=9�=JQ���D��	OD�T��v�N�������C|�C|�C|�C|�C|�[���}�}�9��i:��>��>{�����RIz�9K�w����0D_�~�hyCP�yB!�E
���bFrw���`Wl��sj�IF���Z_��t*}�� nO��
�BQ�����ZI�������2�6MR����>�)S�&��Q,c������G�	�q����-�t-��-2����zY��v���A�w`5�����_^��'J���C�������!�u���C�����w�r�r�r�"=�	l�ot1C1C1C��S���W��!f�!f�}�31C1C�-<��	��l;�����s���������~�i���,�N�P\���B���{��R���}
��O��~`P�8R�����hG�p<��C�k=9g3�-�b�� ��P�Yx����f~�(4�z�L:�j��z����b�����3��QG��~������nJ��Gn'�#��������#q$���������#I���V�+����������z�����~1$�%�<X���}�F�;$hC�z��������0j��;�OS�k�~�\�)�b�J/�X�W�.\����X}u��V_�\�pz01����>��?8J�rX���������U_9*@j��V}u�9���t�Y��-u�\�%H#����w��g
|�J^���i��|1V3�'���jF�;��������]^��!1LiUjnB�������@yXS�����%�������,��RS
�yGJS�p�X�i�6�"��;_������j�tysu�>vX�73����j�j����M��*8�P���Q�L�EI&6|��)Vk�B�s;��~7I�	��f-���q�B��i������9�C����bQ�>�`=�`�5�?�`=�`���\���[Qx���
i�1Xoec:�`�����xb�b��-�=��WX>�]$�tl����q�T����}v5�&���Sof�B�6�V��W�qj�J�:Vg�hy�>�x�j�����������y�>fX�f8�W������>f�6h�2��-��-Y�:�=�j:h�V=>���mH_�|I�v���F�+<s�SF{��p�k��f��f��>J��4< M�Y��{?$���i,��w��!��NC1���~I2�!����+��������=���*��r���]T��!=�Jn<��+�^�0;�J>�J>�j�aZ�x�=�J�����{�!V�!V�!V���W��!V�!V�}�3���U�dt�{u���
*����8{GpV����+2�2��>yCOv!3O�ZF��-7�WkVNu�X���D���6]��1����A��_���?��H�[�pM�q���
�N��`o��/��|�m��%j��N�8!�0�%
u�'��'�4j	�B��!��x�����s��g�`2!�m�9UzmP0����j��.+�Y"�m����O�O�b�Da�DY�On2Q�1Q&
������a*�;�x���1���t5"�Q'�jX���j�����!������^���"���XN.����=�
?��^G�1�;���n��o�\3���F�\����\k��n�"���Y8u<TU����-����_y=$Ko��AK���:0e�,	R�m4�`\����U+<����[^�}��{i�a��0ce��s��2��2�8j�����u��T�T��9��2��e���aN�yAs+�9�_�\�R765}1��Hv�TW{���g����|�W]-��s��r�T����?e�8CZI���.B����Au����{I�r�,���{�R[��e9�
�Y�exYY.+�;��X��o�o4�-�al�K-��\+)��w��.=3]�s$�z�h��zJ\9�)�U�Z��)��
�s��6����)��R��g���f���Y=F����3���L�_9����n��
�'��\]�w�r8#�9��M�2$~����e@�����O.#ij��V���;m�%�9�_\���N�����FI����+w0���%NK�T�-0�%��qZ�&qZF��e�w'��H
��X�oI��F��8T�mT��A���
V�QT�w@����F����������zU��h+�Y>�B��-�;}u��E�N�YWx�����u����]�+��a����2�d���O�,������-]Z9�:9�z����G~<��`�*���o)/������_�0�I�������f���Z]1���|(��� 0�gP��W/������d�Xy�!�U�K�{g�^�!f�U/�{H��>0M��	����y��Y�gv��qA^{K��3������s+��>JG	��\/P
��.����"o%�yGm5�~�Y����3�����=�!v$����7��~��86��31�T+�`�1S
����]���0[>
�����7F,��!��Lyz��@]��n� e��gB��������3����I�5�v�[�U�?�������(|��m��)����:G?��}P]6����/w�F��v_�8�v���d4��N����`��������`��`�'�v;n��	;�>6l���n��c�9ag������p�6&'�A��p�`o��6:a��=��v�<9a���=�s{3�������l{����	{R}�	����=��I�w�N��j�J`�8��I���$�>vg;�����|��2���{O�|�ej[�fi�u�#~������#�mO���v7��K
5v?�0��>{���X����������/����m��mS*o�0�.�n}���B����hiw����������})	���~�w����E���<�$YqlJ�����#�2�3�>�� �����i�K�FG����a�9�������n��E��9 !�j���B������Z�_���q���t<�r���_	�E/���
��49�h
�=��fh��K+Xw��Z�
���:�0?j�_�q��i���������	l����Q�
���c���qS�h}Z�d�x�`I���v��V4asV��	+L�1�������G,��x�j�T�66�z}�J����UPeP�O���%�V������_�Ny�/�_���aa��oa�Z�-���/�j�&���2g�(U2y �"���2�;8�XH��]�?��Dbh6�^biR"�z�*0������� �^+)��p_��(W��Cl@�+9��Q?�3�����;��1%\j'��dgW����������AW47Q����d��t���[%�`��#��uO�#���e*q��������;��$�F��y82�|
-������0�o%I��IZ!�M�h��,��b�-�Mh������t����[�Z$������?k{h��s�j���|(&9�b+��/fe�=���l����7��[l���XZG����Z��Tg����08u�1�9��J��=�j�&_���ox����%X_���j[��F>��}�����y����[j#��k����B�3w��
q�E����Jm�X�.�:���
8����M*:����|]G-�*:�kO��i��������s��\��x���Xa�f,
�7-6��J
�����zkDv/X����a��-��m�Xf���8�H
V�D�.��H�����}vu^�����!h���MgS�z�]����}��z�Ox,�w�o��T~��?���/�f����$����JqjW�B(J��W�R�D�G�����j��!m���V�6_���/���7d�/>�����3�8REc6�g�d��90��b����qu�L���]�sRv�I��ccnV�5\��Z=�8���8m����5�CX�m��*��������������U�W=v(b���x���&�v(����Q�
t�9{*�'��)���m��FN�CB���IM��c��Y���\�+w�`Gm�x�}F��H�t������������a�P��t�nt�G�~��U���I0�������j�T�W�[��p����g����-4
-y4���`0�����|2�I��a��p�o=�t:o��������t�����S�����f�6����A������G,S��#%���}���y�\@m#�'Ge#������g/�o��R������$�)3�8�X��XS��Ycp,9���m��6�AZ�R�bVa	c��E1G�g"i�Pt|L��i�{4������^|�g*������H���%r��]����\*�K�V�er����s7��,D0�ht����Jh�
%�"�HF�m�����d�Z�"L0��$Se:�3�gb�%\��t�6��6��&��J��%����l-qf�P4r������%#_TI����m� [l#[K�K�Ha���Z����E��?���O���5C��.�jK��T����J�L.�1xr:/A�jm&�]��2�~�4�^^fX���b���)Z��g�xhl��(6��#e�+��
�,Hl�ti��n�C�e��P���Xu�^_Z6��Uu�fk,[Bb�j*�#�2u�u�I7�LKM��V5��[�<��e��u��������m�����kx"��Q����1U�� q�o�X����(�Z5z������}�w���i�6w/^�!��T���:w��w�����������O�T�������"�����*h�Y���@���d-u��.1����D.J��4jZ��7X|:p<��$�o��I����<�r�T�^X&5z��d��W'6Y"�; ��`�H�Xf0�\c�ec�����h�����b@1���X4��Hj�El)�`5�������i�
*f5���Z8�V��J�Z�~e5-T��z�&%D�t�Z�.FYI7$il'i+�:��J�VH�b������H�I�
+��q�N�d�$���H�S���N��a���q2���2��n}���@���e����#?9r�'gB��9�.\�Q�M�%�=���E���f_�����Cs.�aH�
INA��]��c_B����oU�m'�S���$vU���iI�([�2*�Y�T�F3��m�kg�G}�A������A��U@�+���}'�p�����IR[
��4b�@���D�]��oC�����OT[�����6���/9��wQ������n�b�~]t��&�����E��P����X2O���.8��v��V8�����L�]clvuzKU��]��[�6e�2S�^-P��9��|��bI���$�5My��0�"�|�MG]<F<wG������-�������U�d���	�;�w]�������#�:�L�Y�!Q=�R�L�Mc������������#��-p�3f�>�.h��������67>�����6�v���pV�Rk�$!�0����G��d�l������pZh�TvQ~�<�M�-�W��T��;��Z8I�=�s�s��m���������I�C02<���+3������D�l������s�'�f�7�]��H������������������!��g~��H�����V�p��~�^��O�;�����N4�~E��3
�����JC��|�����ta�$u��,5�DmW�X,��idS���v�	;V5e�%s��SO�6Wd���O���,���a�����01��i��R��X�*;,����b6�:%�Aq9��+B�&���qM�$l�CUPQ�E�C��
l��!����q7
��`�uZ�/�u��}(Z	���]���L���%-JH�B��T��A�p�84�8tCl�t�-7�b�D����~?���4�-�Ar��� -�����Q����D5����w����}b���{?�������am�2Wq�6�����:f��@��rO�����t�����&_�� �Cg����tb~\�2����j���ds#��~��	�O�.�*r����K�3�
���7M��H�+hZ#���P��oQ�����X�b�;iA�z����M�n���^z�������rwK�to�������O��F�e��N4�~q�w�!w���3
����%���I)U�@xo�1��?��7��gG|�Tc��Oc����n��;n6}t�\��;�N4=��<�}`���zO9�����e��~q%w���l�����%���V�"g�h�V��d�/.�7o#�t�(�����HS������xpt�����_�o����g��4���5c���lr��*��	�
�ml���*.��f��_�=����nd8��3
��'|�Y�!Iq:�i?��}<��?��m�
8!#�����r��E��{z=�Env>����4/��Y>)������lz���<�������������F�������)<0��	#�]N0}#��x
����������:�������k��X�'��dLaOP�?H�G�o��M��G��?��y�#����dT���r"9��H�s`�$�<�<��
d*l>,��9��_SH=`?Ja����Gk���j/�����%��o��2Ye��$_0����g����������|�#������e�!��3��VMo6,�7�P���"�wB���b�V
Y&N%��)9>�F!+�4����g�&_,�VI��6�	!4����3���raI�A^2����5������(��9�W��^���$�k������a6���/&�t043G�`T�|�h��I��|�w�`mN����F0�&��l�. �@��r�����GR�o���6�g ��^q���<zvu�D4���-=�ZS��fqS[��TfTl�����g���Ma6J��D_���.���}=__�C����@Ab[k��Q���l0���~�^���= 3S�u����	�"�'?�>�w`���r��5����������x���O_��G�Z���/�\����f)6F�q7KPg4��� R1�W�fC���L��k���HV�+1j���#�.t�����I�O��M��#���J�p�����7�?������?����<�I�@�r��Y1a����{�5�E_�gt�SG����4E�BH@D������s��}MV��r�^}�a��cV�!m4����p����P�\HE0���������N��u�w�?��%~�]@���>('&�3�xy������p84Ecq�m��z�Nd(����$,�l� W�!q���+�#u��8@P��Uv�u~�"���5"�[S�h���A���m�l�@.-Iec�6�"�9�VU��^T-l�Y{5J���oi�e��1�x�n��6Z_���|��4�_}T��]9�������M��)a��0H�3����aZ�^:�PPSm�8��j����]�B�7�m�pT�Y�,����t��c4G� y��N�
%K����:�0~��;�����4�8/.�/�O�N^����j�^��a�B����p����Fzx�*����I#/%T#������[F���bQ�\����� �
��y$��k�WZN���Zb��!���3@�b �>Y��j/����r��dc��h�Q:��:~�,d��_��t	��a<0���0�����l�;��1=��4�k>'��t�N	�2�m�#c\�jS���n�,����:�@�2IR`��g�E/�L�L�H�	����z��L��T��XShT�B��BV�!�X9]RX�.�����7����r"�!Hu��e���S�:x����0���	�W��{U��u�a���,0��+��]�
��0PRt����rF������u�82d���:�Cv�l��4��:A����+��@�� ��������29=%k��5��U���GZ`�����(U�d�@�\�T�/����C�&|������5��rm���S�410M'��a���M�����M*��
P(����$�a�|�U�3���@�@^��*uN����S�1y;>IE��l0��V\�kxM��"ub8���`���������{d���+�$'�r�����e�!����&�F�\�	���(����z��OY���M���P�����p��u���:\��p��u���:\��p��u���:\��p��u���:\��p���%\����5%
#17Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Langote (#15)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Thu, Feb 8, 2018 at 10:41 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2018/02/08 11:55, Amit Langote wrote:

Hi Ashutosh.

On 2018/02/07 13:51, Ashutosh Bapat wrote:

Here's a new patchset with following changes

1. Rebased on the latest head taking care of partition bound
comparison function changes

I was about to make these changes myself while revising the fast pruning
patch. Instead, I decided to take a look at your patch and try to use it
in my tree.

I also noticed that a later patch adds partsupfunc to PartitionScheme,
which the pruning patch needs too. So, perhaps would be nice to take out
that portion of the patch. That is, the changes to PartitionScheme struct
definition and those to find_partition_scheme().

I am not sure whether a patch with just that change and without any
changes to use that member will be acceptable. So leaving this aside.

Regarding the latter, wouldn't be nice to have a comment before the code
that does the copying about why we don't compare the partsupfunc field to
decide if we have a match or not. I understand it's because the
partsupfunc array contains pointers, not OIDs. But maybe, that's too
obvious to warrant a comment.

It's because partsupfuncs should point to the information of the same
function when partopfamily matches and partopcintype matches. I would
have added an assertion for that with a comment, but with the pointer
that would be risky. Or we can just assert that the oids match.

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

#18Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#17)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On 2018/02/09 14:31, Ashutosh Bapat wrote:

I also noticed that a later patch adds partsupfunc to PartitionScheme,
which the pruning patch needs too. So, perhaps would be nice to take out
that portion of the patch. That is, the changes to PartitionScheme struct
definition and those to find_partition_scheme().

I am not sure whether a patch with just that change and without any
changes to use that member will be acceptable. So leaving this aside.

I asked, because with everything that I have now changed in the partition
pruning patch, one would need to pass these FmgrInfo pointers down to
partition bound searching functions from the optimizer. If the changes to
add partsupfunc to the optimizer were taken out from your main patch, the
pruning patch could just start using it. For now, I'm making those
changes part of the pruning patch.

Thanks,
Amit

#19Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Langote (#18)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Fri, Feb 9, 2018 at 11:26 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2018/02/09 14:31, Ashutosh Bapat wrote:

I also noticed that a later patch adds partsupfunc to PartitionScheme,
which the pruning patch needs too. So, perhaps would be nice to take out
that portion of the patch. That is, the changes to PartitionScheme struct
definition and those to find_partition_scheme().

I am not sure whether a patch with just that change and without any
changes to use that member will be acceptable. So leaving this aside.

I asked, because with everything that I have now changed in the partition
pruning patch, one would need to pass these FmgrInfo pointers down to
partition bound searching functions from the optimizer. If the changes to
add partsupfunc to the optimizer were taken out from your main patch, the
pruning patch could just start using it. For now, I'm making those
changes part of the pruning patch.

That's fine. Someone's patch will be committed first and the other
will just take out those changes. But I am open to separate those
changes into other patch if a committer feels so.

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

#20Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#16)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Hi Ashutosh.

On 2018/02/09 14:27, Ashutosh Bapat wrote:

Here's updated patch set with those comments added.

I looked at patches 0002 and 0003.

In 0002:

+ * In case of hash partition we store modulus and remainder in datums array

In case of hash partitioned table?

+ * which has the same data type irrespective of the number of partition keys
+ * and their data types. Hence we can compare the hash bound collection
without
+ * any partition key specific information.

"has the same data type" sounds like it means a Postgres data type,
whereas I think you mean that they are simple int32 values, so we don't
need any PartitionKey information to compare them.

In 0003:

A portion of code in both partition_range_bounds_merge(),
partition_list_bounds_merge(), and merge_null_partitions() has an extra
semi-colon at the end of a line starting with else if:

if (default_index < 0)
default_index = merged_index;
else if(default_index != merged_index);
{

which emits warnings like this:

partition.c: In function ‘partition_range_bounds_merge’:
partition.c:4192:11: warning: this ‘if’ clause does not guard...
[-Wmisleading-indentation]
else if(default_index != merged_index);

^~
partition.c: In function ‘partition_list_bounds_merge’:
partition.c:4261:11: warning: this ‘if’ clause does not guard...
[-Wmisleading-indentation]
else if(default_index != merged_index);
^~
Also, get this warning.

partition.c:3955:1: warning: ‘is_next_range_continuous’ defined but not
used [-Wunused-function]

I'm trying to understand the optimizer side of this patch. In your commit
message, I read:

This commit allows partition-wise join to be applied under
following conditions

1. the partition bounds of joining relations are such that rows from
given partition on one side join can join with rows from maximum one
partition on the other side i.e. bounds of a given partition on one
side match/overlap with those of maximum one partition on the other
side. If the mapping happens to be m:n where m > 1 or n > 1, we have
to gang multiple partition relations together into a single relation.
This means that we have to add simple relations during join
processing, something which is not supported right now. ALso, in such
a case, different pairs of joining relations can produce different
partition bounds for the same join relation, which again is not
supported right now.

So, is the currently allowed case (partition bounds on both sides match
exactly) a special case of this new feature which tries to match
partitions in a more generalized manner? I see that this patch removes
the partition_bound_equal(outer_rel->boundinfo, inner_rel->boundinfo)
check in build_joinrel_partition_info() in favor of reconciling any
differences in the representation of the partition bounds by calling
partition_bounds_merge() from try_partition_wise_join().

2. For every partition on outer side that can contribute to the result
of an OUTER side, there exists at least one (taken along with item 1,
it means exactly one) matching partition on the inner side. To
support partition-wise join when the inner matching partition doesn't
exist, we have to add a dummy base relation corresponding to the
non-existent inner partition. We don't have support add base relations
during join processing.

Sorry but I'm a bit confused by the last sentence; does it mean we're not
able to allow partition-wise join to happen in this case? But this is in
the list of the new cases that the patch makes partition-wise join to
happen for.

Looking at the code changes under src/backend/optimizer:

+    else
+    {
+        Assert(partition_bounds_equal(part_scheme->partnatts,
+                                      part_scheme->parttyplen,
+                                      part_scheme->parttypbyval,
+                                      join_boundinfo, joinrel->boundinfo));

IIUC, this else block would run when try_partition_wise_join() is called
again for the same pair of relations.

+        /*
+         * Every pair of joining relations should result in the same number
+         * of child-joins.
+         */

Sorry if I'm misreading this, but does it mean: a given pair of joining
relations should always result in the same number of (set of, too?)
child-joins?

In the new comment in build_joinrel_partition_info():

+     * Because of restrictions in partition_bounds_merge(), not every pair of
+     * joining relation

joining relations

I will try to hop into partition_bounds_merge() from now...

Thanks,
Amit

#21Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Langote (#20)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Thu, Feb 15, 2018 at 2:41 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

Hi Ashutosh.

On 2018/02/09 14:27, Ashutosh Bapat wrote:

In 0002:

+ * In case of hash partition we store modulus and remainder in datums array

In case of hash partitioned table?

+ * which has the same data type irrespective of the number of partition keys
+ * and their data types. Hence we can compare the hash bound collection
without
+ * any partition key specific information.

"has the same data type" sounds like it means a Postgres data type,
whereas I think you mean that they are simple int32 values, so we don't
need any PartitionKey information to compare them.

Modified this comment to read
+ * Hash partition bounds store modulus and remainder in datums array which are
+ * always integers irrespective of the number of partition keys and their data
+ * types. Hence we can compare the hash bound collection without any partition
+ * key specific information. Separating this logic in a function which does not
+ * require partition key specific information allows it be called from places
+ * where the partition key specific information is not completely available.

Let me know if that looks good.

In 0003:

A portion of code in both partition_range_bounds_merge(),
partition_list_bounds_merge(), and merge_null_partitions() has an extra
semi-colon at the end of a line starting with else if:

if (default_index < 0)
default_index = merged_index;
else if(default_index != merged_index);
{

which emits warnings like this:

partition.c: In function ‘partition_range_bounds_merge’:
partition.c:4192:11: warning: this ‘if’ clause does not guard...
[-Wmisleading-indentation]
else if(default_index != merged_index);

^~
partition.c: In function ‘partition_list_bounds_merge’:
partition.c:4261:11: warning: this ‘if’ clause does not guard...
[-Wmisleading-indentation]
else if(default_index != merged_index);

Thanks for catching those. They will cause bugs.

Also, get this warning.

partition.c:3955:1: warning: ‘is_next_range_continuous’ defined but not
used [-Wunused-function]

I had added that function earlier, but it's no more useful in the
patch. But I have kept it there in case we need it again. I will
remove it in the final patch.

I'm trying to understand the optimizer side of this patch. In your commit
message, I read:

This commit allows partition-wise join to be applied under
following conditions

1. the partition bounds of joining relations are such that rows from
given partition on one side join can join with rows from maximum one
partition on the other side i.e. bounds of a given partition on one
side match/overlap with those of maximum one partition on the other
side. If the mapping happens to be m:n where m > 1 or n > 1, we have
to gang multiple partition relations together into a single relation.
This means that we have to add simple relations during join
processing, something which is not supported right now. ALso, in such
a case, different pairs of joining relations can produce different
partition bounds for the same join relation, which again is not
supported right now.

So, is the currently allowed case (partition bounds on both sides match
exactly) a special case of this new feature which tries to match
partitions in a more generalized manner?

Right.

2. For every partition on outer side that can contribute to the result
of an OUTER side, there exists at least one (taken along with item 1,
it means exactly one) matching partition on the inner side. To
support partition-wise join when the inner matching partition doesn't
exist, we have to add a dummy base relation corresponding to the
non-existent inner partition. We don't have support add base relations
during join processing.

Sorry but I'm a bit confused by the last sentence; does it mean we're not
able to allow partition-wise join to happen in this case? But this is in
the list of the new cases that the patch makes partition-wise join to
happen for.

Consider two cases for partitioned tables t1(p1, p3, p4), t2 (p1, p2,
p3). Assume that the bounds of any pk across t1 and t2 are same when k
matches. So, p2 is missing from t1 and p4 from t2. This case will not
be handled by current partition-wise join. With these set of patches,
we will be able to handle some joins between t1 and t2. The matching
algorithm will output pairs as (p1, p1), (-, p2), (p3, p3), (p4, -).
In an inner join between t1 and t2, (-, p2), (p4, -) would not
contribute any result, so we can eliminate those. Thus t1 INNER JOIN
t2 is just (p1, p1), (p3, p3). t1 LEFT JOIN t2 will not have any
result from (-, p2), so we can eliminate it. If there would not have
been p4, we will be able to compute t1 LEFT JOIN t2. Such cases are
not handled right now.

Fixed some grammar and typos in the paragraph.

Looking at the code changes under src/backend/optimizer:

+    else
+    {
+        Assert(partition_bounds_equal(part_scheme->partnatts,
+                                      part_scheme->parttyplen,
+                                      part_scheme->parttypbyval,
+                                      join_boundinfo, joinrel->boundinfo));

IIUC, this else block would run when try_partition_wise_join() is called
again for the same pair of relations.

Yes, it's all Asserts, so effectively, we run nothing in a binary
without asserts.

+        /*
+         * Every pair of joining relations should result in the same number
+         * of child-joins.
+         */

Sorry if I'm misreading this, but does it mean: a given pair of joining
relations should always result in the same number of (set of, too?)
child-joins?

The assert there just check for number so the comment doesn't mention
set. But you are correct, it's the same set. We check it when we
actually deal with them by checking their relids.

In the new comment in build_joinrel_partition_info():

+     * Because of restrictions in partition_bounds_merge(), not every pair of
+     * joining relation

joining relations

Fixed.

I will try to hop into partition_bounds_merge() from now...

Appreciate you taking time for review.

PFA updated version.

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

Attachments:

pg_adv_dp_join_patches_v5.tar.gzapplication/x-gzip; name=pg_adv_dp_join_patches_v5.tar.gzDownload
��h�Z��ks�8�.����_�/%U�9������W]����u�u����9q�A���Y�������@� )Q�BT�L7�Z��`���]������������s��N���om�1N����N�1����?����;��m������P�j/�u��W�[�~~�,���|u��4}��?�������U|��s�,��/��&Yo�����1./������j3�����Rr�x����A&��$�����{�I9������$8�74�������Z����������������l<���4�N"/�</8y|LN�����|F�)��#��x�H5��&����x�?�c�~J���d�IV�W�u�0}�|���O�&����a���x�k��p��������w�(}z����o�������Q��'�?�.NT,N�.N�'�'*'��������<o���3�e����������x����	Z$8�f�^����K���hS4�������3���<A��=����������x:Js�Gp�������lC���
<T��f6���	.�^�q�1�$�����dM^���>|��@����}�������'t{��r�~�������?��7���������������y�����������~�#�<�{6 w�X��N�_�nM����(D�$��W��N���*y�}_�;y��<���1�yx��VM��?zG;��[��\�����OP�v���?��'����Y���]'+Rz=x7<B�������!A�2����i�1���U��f�����	/����;���e��W����9�-�����%�Yx~B������/^��#<��Z��t<��;oD���������nq��S-��������S�&��I�|��K��x%�:>]��dAn0������z�zM~ �I�����;�����������y���S����'���3��?g�
�%	������t;��B?�6��;�p��{sI�ywI��\�.�?���i���Wv���B1Z�q���~9}Y��v��t�SjNfM��g�c���<���
3�_g�Y��{���W\��������c��>/��^!�a�~��,?�\Ln/�YF����M��=��0m[
�/��������_~C7���^���:�>����h��o��&��^`rs�3���*��1+��L��:�HU�oR)���U�&���i=?O�A�P^Y(�����_/oq��W�.nR{M}������Fh����h�)��������	��nH���,���0�%��1~��������������	�5m]�zy��nh�NT�M�&���WA��#�����lt}�����/[����k���AI-uk���AI=Fk���A_��b
f+b���kPh����k0����'��p�������/���Wh�����Ox`>	KO�����#��=��?M���}H���7�/���
�"�]�M�~"�S�t}���
Y�Y�8��Vq���zq�����=%/`�p;���M�������#�Hz���M>NL�����e/��a��"}��r��h�������BFR�����{,�a��_���+YM)��x�4}�������N���
����,��\����������l�?���|w-������7
�7
��4 o���A��6�4��44���i����i�vo:�Z~]�?�J�cDU��w�\��7'��K\����8��8�e.���K��2�D~I��iI���$�qzd%�K�0�Y��dy�G1*s�y��AX"�����_�����C��1Y%X�c�d��qG�n����%�	�8��b>������Om�N��Q�:^���e�Tu�`�7�Kn��U�����nfO���L�1�-+6��|O���#�IM���E
w����G7~������nL����p���D��uc������t<&zl�Bh<&hA1&� �`��*�M��~������*E�G��TS�
�{������e�<�o��&��
�#+�1��BcR(�G�82��]� O#z������[���(��kIo����@� /C_��i~/���qYOz+���2[&�eB���������(��y���h�����8������9�V��Xb��*t�����h���f�z#����� �r�,�����>=>cSA��V�A
��� ��n<�����h�V�����z�hO�1�"���I��I`;&Z��`L�m+c�c����4-B���Im�7��q�qW|?J{������>�������!�XQo@K��-IK��}4I�/�����|.��>����y9������;"dy�`~�2B�3���3�'�C�<?�zyIWg^J��_��0[�X(e\>����Lf7�j77��������!����f�����cj3�2�����r<>E�"�,��`@����N
������H�MjP���tm�k3Sxl�iL=fL��M1��tL=�1Ua�N���������J��bL}���c��v�������1
�1�bG���1
,�T�
;SC�^;c2c*���1
�cZ��
[v2�����v��q��(V�S����w���t���L��4�`���!:�"3�z]���k�H��GVW�<Noo��&FLwzI������I(�Sk��V.9�Z��|�����9g��/z���0����T��U����x�\~jMWmQ]��K#������*���u<�O��;��G����#� �{I���'Z<���6E�^Lg��#���������LD�K�<b%�[�r��gUC���������%)�������xvC�&����h���1�2'5�2�-s�n��f����;B�f��Yc'<��-*�����%[#6|����6E���O��������������8�?��#A���/�?w���V���5��}�����}���������<�e�j���
�6^��$������e�{�V��f����\A%H*�75��~2`2��0nm�F�Zg,7���������w�����9(z���Q�g��\?(,�V�� e����q���9�Yf�|��A2����r��r3y7*�z��t~&�9�a>�L��r�F.3���p�.��c7�������L+#�#�����X�l�3������c�]v�����JL2g���L�x��c�
y���!�a�-�����=v�����c��8��i��m���c��H9�������}�v���6W�
�-��O�,�C�K=�D74���Xp8^fc��l\�'����,6T��v���i�<aXcy[���2�p��/O����iyw:���I���zR��H��d��|0����mH��]oSzmG.��e�v�H�WhQ t�J���WhQnk$�
|�����J�XD)_�[�)�����\M�4Oq��y�����������m�>=y?�s���S��N�s��J���A<7����j+�����fr�3��'�K4�Av��y�7�����s��2%>�APo���O=��z�(�����
�*_�sk�q����@ak���������O�V��KaGK�@[�w���*�m�L�Y61rF�&����5
S��e�j$��7�P �y��sg�i�&�5���k��Z�����5o@qPP����x�M�����_��=����O���y
��7h<���h�&���K("e�N��W���E�A�4�?���4���7������jY�F���D%��Jjn�9<2�!U����!MG��H�]D�W����'�rJ��
W��;�P_�ZRu/���#fWe�/�M���"��D�����"�����
�y����?�]E����}E������y��[K��E�/,�u��Mm�6���l�iT������"]��e��DW�O]_5>	�MBZ��
�|+i�|��	<{kl	=uc�^aH��<{m={{I�3o�i�}�e�E^��W��$J�EC���+����v�T��)bQB�G����'����6�Q�M��h����jp~##��c���x�����,HB+Q:���	�Ei���WV)U`6��c&�J:]�����aYH��������8(��T��Z�P�^�^un���5l�}�4���Z���������k�j���x�P�5�a����7G��&>'��Ai�M�6&�$�n�k�Dnki��c���������������1�&A�k^�&<]KoX�M��ad�7HZO���&��U�=d|�&�Mh>�H}B�>����>������/��b�z,LYNx��� RY���He;���7�`e�^�����7�T
��e{��W���((7Y�������k�d,>X=Y`��
lU��qp����^��D*�U����	S`�2{
����A��;9�<�(�U�0�*��VU���`�*ms�.^��J�Y��� cu-c�E{��h�����������L�����"��4��i�]���
jz�~�
�;����{7Z�
��1�l7�s6�'�9��}�O��v���k�v���9��j%�u6���9e�}�O��v��k�v�}������d��hF�>��zwg�	����XCj���������YvB��{�k��Xc�A�p-����v��5��0l�o��3Z�����~�c��n�����d���N<��V����D+��s��`�`��4����T���aY�����5�vf�qp p�3{�7m��.��O���&�-����m&J���]mw�]�
�O{?\�L>�Xb������"qK��~��5Y z	}�O��&���������gr�p�s$G���q.9X��<��wY�t�3\������93\�����:#8�\��f�����y={�4;�F<|���q~�x
��^����a�
h���4�a4l��V�.��a��h���4��4\���?�N��=sy����?{P��E~�L�&0����S}zR=N�^��$y0�����I�`85{
��!0	N!.	Nk�U�i��pz�"Nk�[��pZ�������b��8NN�z�p�����W���S���;
">r�j���X�S��88Q,�f�qp�;XN�^�M��[F�`:e���NMf�-m:m&F����)E:l{)�T5��-� X������xU��:
BVM����;JA��d�a�_���
�+�U��;t�
lUf�qp"���^���)�U���Q���(�����N�0��'lU�%R��J���V`��6w���������+�
2V�d,nba[�����
�L5��K�S��k�D�*��88Y
LUf������i���M�zK��  �>�XuXF+Hi��t)��a�  �Q:l3�4J���c��d$GH	)��l��4 4��/M`8	����z0�����I�`85{�����pj� ��?0��?0��$�Z�����E,0���� n���&���5	��"q0�"0�r�����pz|�;&��~�2�-��l�X���r���3&g"�I�6u����K�POH�d�'���<W���2��P���v��Z�f�S��v������F/���y16q/Z)Z�]�*]l!����
��h�t]s�J�*u����i�T�uKR��i ��������k��s��$+���X+����*�Bk�Z�J���v���SWm���I?�J��F�)�]S��j�^C���~%Y����Z��
��8�V����4�j�h�u�vH^�4���yi�����w��5�&�W�h������Bkc@�K���v���FWm������J��Fnv����<�^Cm��~%Y����)��
�Y+�63���4�j�X�t�vH^����$Z��[�Tg^���l�^�&��`��"\����y��y��A����0��'0�w?`��~���0���GR?m�?`��O�0~����~����O��>�q^C�y����>vUv�>�]�OM���f0��'0�w�>`���}��f0���GR�>m��>`��O�0�}���^�}�RcV�^Y}Z2�,�jK���e���7�s������~Z���l?��$7��N'���X���`j�;��?'2.'e�������`j�;���\Tu:��A&��kP[���5����9�q9�lB&�&��P[����^���&�+��0�nk�z����Mv��@����	�A��@`�;P[���@`P�0X������~*����V�`'X}��S)V����V�������������_���/��l�P���@��O�L�t9=�_��s����x��mf���<�F�5zH����|D~��<��\�g�y��'1+��|��Z��|D�d�'����xs1��`F	
b4[lFx��?��K���WC�������G���r��\��b��A��e�����7�o��_/>��s�����z�n��`���3�jL��#U-�I-�xZO4N��x������^Z"#@���? ?������������;�nHk{J�*�$w�d5K��c\����C\���Bp[����o�U���+�v�2�
�/��i^|��H����/�I-�y��H��������[q"�b"�Y��OdT��@�HJ`�|�
��)���Ds�lTp����<M�~r1�m�.�]��`��C3c#����9������������N�i��n�M��?+����(�DP�{�����W�	k@��ia��i����P
S�y���p
!�#�u�y��f9O�q>Z�U��O��(0�P��7�����q��QR����na_t�k��������������=����`D�m�������@�`L�
+
�K(|���G������!����"���x��o��g���=/���|�7�Eo�Y)�)��B�=r�0�a���y���>�%h������E�=\Kz+�
}�y���M�{���z�[A����	�2!-�u�4_��'�������*�NJ���+V�eE��}E�&�&*!a�r���CX�NC���O�.+0��+�U,'���
<m�To`ioR���M�~���`������t�s����Q�#�+>E�����T��Oi��zZ2���h���xG�_�sQ��<��,i<��_������<x�>?���UE����!Yg�U!/�_�R�/�f�m(5b`���:^��av�,[�-[�m������fd��I>���0����E,�)H�`@���6�d���h��m��cl�%,���T�^������u����s�,N*� ��S�N������w�+(�\��e�����>�(e����u�<���5z\-_R�1[<��L�\?�?��8�$������mX���w�_��z���Z9b�b6�������n
b�)��,(�s�G��_$<�q�s�5���<a	Z1���F���T�Z���kc������$���O��W�h
�(H�Hj����������%)�������xvCR�z:$�peL�M���Z��-��T�7�U����t:����3U� ��,�w����}^�����{|~t�������������O�M����j��>�w�}�>��W��^��2A�LHs�|/�����evl�Y���a1\�Y>��(WP�
����=�*002`�Q�i.��2;�������|��/V�`u�dHi�����b�h������]���CB(��J5���[����v[TiK�O�������2"1;r��L��=0��H�DuJ���w^-Q(�\}�,���#w�-��j�CbCT[?;�?�������)Q����R��hS����p,����p\5V<�&�
��T�?=���x��hJ��/)6�P��)�3�;-�N�2Vuu}�.������'eU��&���`@r�n���P���&�lO��B���x������(M��4R����S��Q����i������y~�~\I�6�	�g����"��s�M}��W9^Nn/n&�<��|��1���^�b���M:=#�i�	*S���(�����N��h�fT?��{��V���h�i�?����*Yo�t�\~��l��R�
���vW"���sM�}��};��3�������5��F�U����Z
��y�(<Z�������]����GY������7���O5�?�(F�����
q���r����Z�a(��T�$�M3���A������0nT
b���������^���D�/3�T
"b>^I�8�������XR1�����c
(�|�*��;���t�����r>X����W�1.�y��!�b����B]�x�"�-��K�������|�
���������B��'��<��C����}pr��,�zI�2��i������e������$���e����+����v�T����
��(��8\�}�[�������!��$jR�
%h� ��V���c�+%�����;���`�25����o=��Ylu�0�Vla���;;sX�RM{���;��F����mL�m�fg��W�i�5��h�
�sU�|������d�"�3yY1@�C+�	�q.��,��:X���w����,���f��he�������,�]����A+�]������U@U{�������t
tW��Z��J���,�]i���+F(�k�eR��C*(w	{�.a�2�C��-4�d�W[a]�i�=	�R�{���'��8o	�;���Ma������
����	Y�>��9������N�lc��N&�P<E�	��>�Q>�g�[�p%�Nv����d��C[�����
GJ�n����U
r�j�����`f����3�e������`�������!��%������B���{X���%�m?�bM�����~�u��&�?�e��Hz��4�xl��$�.�j�(��b�e��E�����^�I^����=H��C,9B�����DH�
h������5n�v��%?M����H}z���f�x<�A	����I�@	��������O����|K�H^$�H�F�;
@�L5h#�6��5�4 �2_ud���&����#��&��"�VM:l�#����3��^���:�|��+�c�]�����C�V��2{
U�J�
|�@wU�i�V�}�AwuXtW��f�������&��@u��4�+�2���:,d�+��x��j�l0��a���t�@�	@�a)�`�Q:lUl5J�_����@��@~���~tie$�dPF��[����4{
���F�H��edM��H\����=���u��ed������c"���/��"����5�,W	�<c� ��dma����B=!���PZOD�d�G�`u6�v6������v�����Tu8x�J�z������0}16q/Z�+�.HE���"���v65��t]s��t�:VM��dAUj]5S�G;T����^C-�Z���@s�b-W�!�Ck���d���T��@k���C���eP��K#sO�Cs�����5���W�h���*W���*ZBk�G�K���v��>JWm���I�J�^+:k���S�����N��"_M�E$��Au�y��y��A�@u�P��'P�wT'�:�	�N@urP���	��9������S�5$�~�P��uT�iP��v'5]�	(N@qb�@qb�=P���'�8���(N�R��I��&-)�LQu_�Mvs�MT�	hO\_FV�'�=i�w;��X��\:Eu�I��p�A��V�v�C�%2~Nd\���&��hR���Ii�{�I�K)4)�����h�����e���@���T�3:��	<P@{����Z{(6�z���F^@o�o��^���7i�7����t;���E:?T��?����.�]N�c�:����j3#ApQ�x���9���h<>>����s���*^<%�����l�>/���t�����BR��|��Z��|D�d���}�������o.&��p�A�f��/���}�W���j�~���~����6^�7���^���:�dSf��@+�p}��6�����N?�J����
���<����?R����B����QW���C�����z>^}����d�KKdm6B3��G�,�� ���>�L>������d���Mr�NV�d= ��G(:?����������\M.��b�W_�����l|���N����WR�~~}��Jj��__6�>�������[q"�b"�Y��0��8��b����a�]��s,��K9SD���\t�z5����@�*�L�X�X�d�M��Y����e��5"��`���BhKo�G�V��M
�pKo�o��i@�T��;�i��m������K�4p�?��!k!�kA/�r1?�'����0B_g�g������X�1Y%����]�3��gG��	��`�n�S���&Vo���Y+��m
�I�)c�
F�XA����h��C
��X�������nL<����;��|��cj���)?�WG�HV��*��G��PC\�7����{^�����or���9�RS*+4&��{�a�����
�4�W}LK��i��=����{���V��
�
�2�U�����/������?-�eBZ&d�i���Oz+���J�Gl�������)��W��Tr����BO*Y~���W�������#8�2�rW+�cE�
	���i�O���7EAc9W�Un`OmR��=�I�Z���~���E�}��%�Q.��;��(�����S�t|*{������bE�-HKF�dDJ<�������(�}���	���/�j�N�Y<d�����������I!�q�a�E��U��e�l�
�������T��%y�� ������������d��0#�U�������5�/�`�MA*��T�9$�6�FC�l3��c�/�o�����n��t�f�5����uv�5-�oz���S�8e�-O��)1c9���N�g���&�&sX+��>�v��X��u�<���5z\-_Rf4[<��,�\��?��=�$������h�"�kn��L
��]k��F��K�U������V�A6��tF3��B�qGv�&�����������o����q�O�D�V�+�V��#�/��U�;��h65��.����c��d��#�R}��k�C��!��Er|;{I����:i:$�������F�]Bt��%#��z���!�v�a} W����N�Q�s����dQ�ET�������G��Y~������c>?4)���q�i�i��}^�R���N������j~^?��gZ&��	i��oc������������1��3�']�
*�A�����
mF�;���e]V�bS���4�������a����I#}������X�����Ai��'U^%�
�q/�����Ie��"j�������nl2:?�8�,���ob5HN��������Y���R��hq��D��hP�R�*c��s#2��H��3"SnDD�LL�&��1��R�c�c[Kf�FWO��^��g�sw\�����p�����s����9(6�G:(��g�so_��)3S���)����B;c�)[�_K��c�R=��c�&=��+n����J5T�?=��'��k������G(��������w�C����E��Hj����hF6�&����`�k�nO�iC~��z�2I;���+�(A�#��B�0��r*;�}�~���JD)_�[�)����9TM�4Oq��y�����������m�4|~�{Qx��G���I��W�	"C�s����s/'�7�K�KM>����@O/|1O��&���4��)�Y��b�yL7�O�C�^3����=nC���K3���4���v����6	���l�]�r���pfj�����A 3#wve&b�<��Im>g��V�w��	��
|��Q���2�+Mt<�6AV�w�?�0Y��QP���g���?���s�����i���n�P��A��3���I�#�7����������F���DO{%U���gkk��Z�tTX_�X1_Ew\
�T�O��
�l�B+��Yk�[1��f|���o��TI3]�����W��S�3n�y��r�?�,�W�k���v��?o��W��b}aY_����g6��f{���HW_T���W���x��c��~9}1��UXO��&!O��U���xu	C^]^]�$Q�'����U��B�����y�qt�����}��akCl;��*��K��X����
uu���o�����}��$���������F�o'ZJ*���������A
�c���fw���x����b����,�R��V ma���f;{�Y�RM{���;��FQ����mL�m<cg=�W�i�5q��w��5^��jor��5��-^��=��<l=UB���T"��h���g4�.,������1����G)�[��o�sdm;�z'3�{��o�s|m;�z=�{��oo�Pv{8@���l>,�l�Y����k����k����k���8�wp�;��4��S�����[@����6w��t���e�
g��d��C��ph�@7������!>%MF��1�6��mE�����������M�+�w9s
�+1"�n�:��3c��nl�l���D�t��(��s�8��n';m�|'�]��O�m�}�O
!i�u���l���-M����)��}.DV��N���1����d���4���i>S�=�5vZ�1��$�8�v���A�56�j�w�S�Z+yEz-�/���F.S
^��/��88-0xY������nO���WN������6C�{�iQ�]e�]#�i���E���-�m?�.�,@�B������"�9��M�����Oy�>=�I*�<�B{�����=���Me���B�x��Xq^�x�$�g/����J��C,9JU��^��[���z�#X%yC[���+�e��h�YP]1�.�7����[��`#i���B�8��-^`�0{�����a� `�>R0q�6R0q�$@�`��
&@�`��E�{L��&0q4��xy�q��C�i����T����$/�q����I[`�0{�7-amP���WF��K9��}0�"�e�uM:l����t��@�5ig({?�j��d��/���Ag
�t���Y����5S����5M�c>�i���u}�S#�	��a!k�Yk�;t|
:kms��w��d�
dsn���Vw�5r�jPY����k����k�i0�]<`z���[r��P2������J�(�
B���VdC(��+��J� ����� �W~���B�8��-^`�0{�����a� `�S>�8�'L5	�5�8i��P7�8�o����>�8���VM���c����/��"����5�,W	�<c� ��dmS|�}���zB�'cM�����	i=Q;� ��B6�# ���1�D�F�hg�z���t�l^�M�K�V�p�R�8t����e�]�M
��9+t�:VM��
����?jf��vh�3�L��Z�b�J��	��~��
��%��}���4�j���t�vH\�TX��yidD�vhD6���Z7g�J�-�[�]�R-^h�U�#yi24��5Pv�����8)GU����U!��������k�����$+�y��	W�!�����>��&CS�^5�������]�Du��Icu��*����W�5&��`���"��@3�Yy��y��A�@3�Y���'��w4����,hfA3�Y���fVR4�m��`4�����b�W�Yn�8�!!�>x��^����N�^�������eA/zY�zY���^�����,�eA/zY��J
�^�����^v/Uj����Z�%���"�S��nN���9�����
�r����nG�YKR��9C�x���5)*Z�0�h����T��D����K�NP��E-W�m�og�Z["���N�`A]kR��\aP�����P����
���R�
���������v��Z������P��	��E*ZP�V
������-(gA9[)����^����l�0�e���^�e�R�j��j��n�A���\^���C����?�2�����<���
��63r���9�������h/�D�|���h�����l:OP��e�T�H��Y������3*�M�r�?z�������-4�q��^y��=�������2���x��l�lo&W���$u�I�������
������O�3��
���
��j<�j�:He�Z|�Z��D����W�.nn�����%2"4���#��.������>�L�������d���Mr�NV�d������C\���B	p[����o�U���+�vs2�
�/��zN|��H��+���j�I�TS�I=����.>L~��g�/f1����,�Vg1H����'4����[�����_\\e7#B����I�ek��J��4���������Q��
����F�=��tc`��8-u���P!m�`����*�S�9��)0Ql�	|��
[M�7lE<�-���G�j���i��i1�t�(S�y��A��MF�Fl|}^��c����d�,��:"F�`;���m/���I����!3�^N�:��p}4�7��h����B��(=�|�k�����qy���y4��Mn�4GV�cJe���Pv��!�p����A�F�*��i	Z1���G3zQv����{C_��A^��*}��^@�����V��'�e�LH��l�!�2�Io�]<}^���$����B�"YV�sXt��9�e���?���URk��Q����k�To`�iR����xg.���o���)��A���~����O��)}:>�=E�S�CZ������%#Z2��<�oz2��:���H�=��p�Bt/�x���|.b����>�8�������a���]���e��g+]��O
2�d�����(~�JQ�,�-��T{_`\>��D��f7��`��j����9U��2�@z�Q�*��@fR���b����X2VP�A��cs�QN��.�g�L�P�q�U��VkS1��T���Z�����~,�����{K����S�<-��$�e��
jB��8�A���
���e�c���}^�.���4�F���K�2f�'�J�qz{���1���%��g�(wUV��5M����W�t��JG����������z�R����������.H�\wJ�&�$K���K%Mc��o�0��+:n��Y���q�hb�|��5��~����+���H�r*jO��&f}�c�#b��_�%]}��E=�7��$rD+NU9v���h���1���0����g��|~$hg��%����z�����"��O��v��O�������W��LP-�\!�FU��)y������l+d�O�,�T��BP����yjN�fg
o�L��-��[�Z���T��V�����l�S�la*����t��\T��zrQ9^h�-B�l��ieDbv���r#"�z`��Z�)%~��0��]�?;u������vlQ1jLO��C����w�1=���?�XjD-�s�X����X���&��{�d�����8���U�o�%���%�>���<�fw����P�F����?>�Z����l�W�$��|xrH�
�T
����r@PzT4�����B�
��iD)_�[�)����lPM�4OS�!5�{~�~\I�6�	�g�J�8��oc�M�W�W��]Nn/n&�<y�|��1���^�b���M:=#�i�	*S�������P��h�fT��{��Vy�V=�j���b�U��$�r���.�sq�"��)8���J�&�-�q#��
�Rr���$���Jr(�R]��v�M�@L���������������Z������VVe�o�k�O@`o�+��/3o1������d��TW��b�����V�W����.��T�U����WU0�~����Q-W��b}e�Xu�$�a��v35I9~�:Z���V=-���.��b�.����������P�?^��x?��R	���x��c�I�~9}1s/�^}����I�N
v�>��mtZ5��,�|��,�`�dl	��'�`�pe���b{�
_��|#�F������'��^rN�q�BJQG$�K�5���4T��\{����Y�#�H�J�Ho�����Z0�h��w)�@�J��.�i���4�����fT��uQ��e�<��|+�
�� �TiY��y-�)q@��~�n���J�B4����K����i�Ghw�q��%9;9�98	q�qRv�xR�h'<�����2���;�'�O�;��Q���2���2����[d�TOM<l��2�@PA���J��]�I�}l��M�����x�y:�m��O��t��W=,\�m��l���yy��iw|����n;�	��=u��zu��s'=��w����g����Mx��)�g�y��#[�i�kp�
p�<h�kh�A����5��� ��Jl0���(�v���i����&��.t�[��]rL1�b�S^�O������c[^,	�(���LS��(�<���bG���^1��$�g�����z�KB����1h�����+���
����;��n��mG]mh;\��m���am��{�������o������v�%m���[p�uG�<�;j�;@������[f������d���T���+J�-7�%�vI%�������2�u	e�]��A�28t��AP�+�����Ay?��OZO>7�`P9�d��k_�d�o{-'o���7���d���y���}��V�&��#������?\�
mh;�h;@����>h;�)�
k;�'���m�}������v���������1OB����lof��
^��m�1ayL�n��@�POH�d�!���<!�'ja��m��)��Gd����5}�x��a�6���F��c��s?�����M��E�*�j�R��?mJ�s�|1�f��~P����:�A1��9�V%7|�tyZ�%�MK%��B���"u�j��T��
;Y��B�*�h�����MK���B��bu�j�����
;Y���UU�v�������^��MV�����"l@ �"�RAJU)��Y�RH��)��,M ���b)�T��3)����R�9IR���T�QAFU�@F�!��jYdT�>��
2*���8�~�����Q[�s�,���Q����v7_;����%���j"9�R�`,o�����)c��,b�M��B�e��D�}��D����[����[)��G�\����O�=�]y��O�c�7��%�y�W�XK'�U�i���MyL�`2�(&C�*z��a�<�����mn}�!�������{'^���XD����t;���E:?T��?����.��.7��s���h����K����k4[�U�xJ�����3z������=$���|S�B�G��8�+Y����?;�:['i�$4[��[���������8A��\.\�f��M��:�]+�/9�����br{��/����^���=�������2���x�x��L��z1��?H� ��|��V����mr���'�a~�x�^��5�Uy&U�i~���7��SZO��'0�'��9��	%��t�a���-.������-&D���QF3g#4C@~�I�������������nH�J�*�$w�d5K�0n�N����0�[i%)��\��3tu}��rq�����4D�M>^�f��0{�7�\M.�������o������e��wZM�l5I����$�>_��$��y=��q9�����/[N�z9��r��������S�^��D��g��a�=���B��'������19l1*E��sL���A����h����US=VXyg���6n��F�<p����z�����T�W 5�>$��=.$�d}Cy�]:�j���k��k1����'4�IV���C|�FE��q�����d�(&����'�8Z%��*Y�'u�����/);5���mq���@��l@����������GO���$����B��(
�=���{������e�<�o��&��
�#+�1��BcR(�G��0��]� O#z������[���(��kIo����@� /C_��i~/���qYOz+���2[&�eB���������(��y���h������;a���:��sB���e�9��V)�.�\c�D���,Y��>,���v�����[Qf+:kE5
i`����� ��%7�ShR��7C��8����bP�����xC����@�c�����_�):>�O����(}�xH+V�����dDKF��'��	)9��?=�����y�'X{������;�dy�`~�2B�3��.5�'WC�����������:J�v��3������Y���������,3�[����2�d&5����xQK�
"3H��*��QN�!W�V�q��������b%n��8�[m*F���u��L�!t������
�-e�S�F�����]k�+�iy$T�S�1g1<y�s�����%�^�k�\-_R�9[<�U2��m���?v������{�\M-�Ll�M_-�044Q#��,���P'��*���s�-�!5�R���"���l�-4Z(JU�
|�#SU��e�*�"`I�q�'�
�Wt����7��������H��X��$�vCsA�^W��d�F4�m��6-��
�����.���Uu�t���<K�M���Yz�W�1���a)���yE���v���s\������/�����b��y�b�9���x�*5��|�)y������g�O�,�T2�BS3���x>D�fg� ��Q:�����TG���:��t��-*����W��Sk����q*�]�+��J4M��(�%Z�
s�'g�|n���1���d�h���h����@E61�	S����p���$<��J3�`U�O����'Uv�������'�y��7�=c����g���<�CzK�D'���XpW>
c�1a,x#��.���bC�j����V�S�Ij����4G(��������w�C�%{q.������'��,�&�����s2V�R�7�2l���0����.����
�N>�4R����S��Q�-���i��vO�g)����}�I���v:>9�����R��?9=�� ����l��N�l���*��n��^��}9����\���	��4r@z��y�7���H�~�O�����b(`:��������qZ�Z�_��wW�hW���&y@���gtI��T�#hP�U�"�qR�%��j���9���Q�yup%���| ~%c46e�&�	M��g��&�����bkbK��H1-�f ��[b|��epU�7���' �T�+�^M�������������U5E��*��Rn��ZzuT����*���lU5]��x
!������J�V��%�W��5uI�qzH�M>o7�N���������`Mcb>^��x����&���L�/,�u��u��s�n�i���"]}QY_����e�
��f�N��N��Uw�-^��4z��r��b�q�P����,O���/�Z�������~�%��<^�U�\ A &�����#�����[b�AlP��X2F�r?�D�����O~K=������b!JB*6�n^�iHu5I�C�<�����Te�����LdV�O2aY)���c���P�@�up��Qd����7����q`������V�1W�h� ���A`�H�6!{�K�^��za�����.N1: <Sb%h��P
�@5����Ps�����t���4�g�Xu��6�<�-6���M�����3��QfTcZPM}T��,��-�*5}5f!	<>$�r���oS�*kb���& ������� ��a�BR#1�b`�0���K�w��p3n���;qCe���*����P�ww�T��qG�lG���4�v���F�L�Z�?���������d,�+�c@c�	��l��7���.�}L��1m/�C�v;r,Ml6=���&���|c�,���w
���������i���8I�"���Q�#�O �<��d�Ab�y������=�A��K�Xr0�,�=�O���z�$yC{h�E�#���F��,2���ky�1bt���h\��e|N���u�iA���	��K����\�-A�g�P���}���/����o��,����.u�@������x)	��@��>���x��M�m��-�)P�1hm�~&+���@��P
��f��4�-T�^}jvj�+�4�gL��O�g\�<nYhZX)���Z@5�j��5����m���p�P�)��������m8p���6��v���C�3��H[�`�\}1�7�|��f���O
l�c l���s%����9B>Y��u�iA���	��K���%K�-A�g�P�#�}���/����o��,���w�u�@����}���c����/��"����5�,W	�<cB ��dmH�
o�a!��<���D4OH��Z��l2:R�I�O����
��5��xn��D6�=����
l�:RH����*��JD�/���*D���
1��� RU���9c7a��Um�X6(�H1s��B���B���B,��B#dm�&dL��,�s],�M���0oW���oW���oW��R��B��@W����n��J*T�^N�����bW��Tj����J�]u�Rib��Ax�X����X���~Y�LV�������� +#����AV6-��6���2�����,/�r��	�r���wY����{+*�	g���hF`UI$e��ARI�-�2H�-T�r5��������RpAyo��43&���e������Q9� .���2�� .������,/�r��	�r����������{+.�t
q5��m�e�+����6����iA��B���AP�}7APn��}��RpAy���O7�����_./����z�����v�H�7�zH�����$d���������_�g�J�i����x�������uB#�-��y�^�������U�xJH����d�,�4�������4�qK�^���=�������2���x�/z��o&W���$u^Vf��@+�p}��6������?�J����
��u���~��0����x��������QFGf#4C@~�?��y�������r�nH���,�U�I���j����x<B�x���O9��|���������*�P�����Me�������t�����!��%��:��l�fVC������] �����i�=E�F�/�oN����'=�^k��#jC�#�_��?*�C#x����-0v�f�F�-�x�!&iu
��Q�?��L����+����=H��f!/�<y����&��F�������yr��a.�������#}4p�c_�Y}����-�;|���;^��>�h(Y\I��{WY]3�G����}C�=W�{������|A�/`�Y����g�|&�����|^��c�yY>���e�l�=r��a6
�
2#O�����.�Y�B��L���9,����
�:��;�b��,������H�A��|����0x"�+X���W�����O���i������������������Z?+�N9���:�����<�O0�x���P�v'�,����/#�=3]t&����w6@^�T��_��q(U$��`�o�0�A������@��&-�V� =���*��@fRC����U�r�
����lsHFm08��\�f�O1�u�EA �kk��vG>m)�����/����T��;�4SR��%���E>����^�C��[����%����S��k@�%��1%��%�����
G	I]��kM*�L6���+im�M�0���`u'����N��`9��:��>�w��������P$��PqR��?�|]��k!i7�o=����r�h�y��������o�O_o{e�1oV�t��nE-�Q&�����0���w7�ry�,o��
����zXv�r����v��������!]�n��K�Vu=��������{^rf�63��T��Z�����w<<�"c�3�N�[�ot�}�2N��D���mee�R�OE$���>��i��1��L��\�E%W��S��rJ?��"A�7�X��%k�Y
Uv c~�n!��rj�Yv���?K�;'�
��T?=���x��H�����uG(��������w�C]���E��Hj�����%�9m������d��JRS �x�X� v�jp�<�'�Ly�-��M��\B���N�� ���O����?G^p����<<;I���R�G��ip��0O�������6���*���G|��H_Nn/n&�<�|��1�eM/|1O��&���4���Y�S��S���p��kFZ���6���Z!P�p�@��d�I��r�]��
5�{�,��R��&C&���������$2��g5�&�n�3��/�8[�5�x���5�d��'T2e�!��7�'�o�)N7�c�`��Jx}	��e�'�E���-T�0�����@]�����k���|umS����cj���eJ^�#��8LJ������Y���1q�_�__�<z
��O�=yM7h�]Z�h�{��s��R��23����e(%C
��P5�r1���@�2������R������3+�0	6�4���nZ'J����k��l/�C��nwhj��v�>�E0�j���b�]��N�fe���@�y�[yW���vK���q��n��k�)"�z��S��Q8p�)�v��P�������}��B7��N8�aI�n��T	��.�5��Q��4�/�x�B����;<y�!:����}~NVOI�q�����xwL�rN\i���r����@������F���4��Tw9
��u6P��8=�yR��f�l��f��p�M
|��y[��+��������$�����V�A������k���g/Dq�k�	��A'��n�>��rYR���,qN=�n�M�3�}$�����1�%�z�\�.q�/�+�u��~�\�.q0���u����\�.q�0�����l�s��P�&b.�XSZ���&�-Y0��$L��{��8�9$���N�}I$��h
F����!�����=�Z:P������@=�z�Q8p�	
h@�@7����M�@����u���+�����zx9�:t��XM��l�7���Urp�d�����tT�	����^{��`�dA��p0uzc��f�t��g���j;t��A�T8�g��/_��=
^��m��'����~���	i��E��z"�'��D-pQ6�pT65���\�\V�Z�U�`DJ�u$���j�M�;�N��ya��W5//��6q���jS���,"$&��tU����E
����������`9���(��:�#��UYMF�p�Q)+i�2U�}�u��wsx[	��?��������������A� ���.���
�c'�A����1�j'���.(�Oe�$I��~�V����2~�G~����%�j2�U37�'d�0�c	 ��%D����T ���}Q����D����*:([:(�3zI�O�"�>1U|r�d���>��-���n�A���\^���z������]&��?��63��<��������7����/��/�<��{K?�/(]zA���=�_��<�������3�,��}���^q��xs���������fk�y�^���]�+���>^�Y~����^0s�[�h����
J����2���x�/L��n&W�����i����?�U��A�\�z��~k��?^�����!��F��h���3��T���G��_Y�oV_^MZg4����:�:�j�:��#��e�����.>L~����>^}����dW�=�h�l�������N<%�do��u��%��23B�����T'�O��a�A�u_a"Z�u\���j�\M.��|%c�����(���/_�~�������f�������>�������N�������[�~k?��Oi��k���l����@\�~��1k�-p
��0�����n?^��B�v��	wy����c��-N{8�!��o��0�N������u1����P=�	����^�9����������c���u�Az��T����_������%�����4����%�k?J^+��e����D]�T����H���<��T��dE�������H�H�H�9��z�2R*�[��
�.�h�r����0��a�B2R*u[��
�.��� ���Yg
`J��J���l���
���Q�^���F7�W��l���C����,��/��g��^	?�3����'���4����-_7Q����r��|h�<&�dq�#%2���85�y�(V7��2�y�*W�Vi����������a�(����A�0���jy���v��:���7������)��H�u���)��C*���:EV�k�
�����Boy����b���t�hr����z�V��v��(R\�*s��Crr������S|�4���[�����/�%���<���/F3��N+�H�Y��-���s�-\�W6�\��f�AF��� �
zZ���=����R��3|������(([e�uO��1�>������G�����Z�4k��Q�Zy�{d����9��L���"K����:�n�N���oE��Ap��UJQKD����j�!2~#U'���j��^�I
���V��M�~�o�7�u�0g�4����S�Y���q�
;At7�BLu�����8�"@*!c�����8� ���
!b��������3)R)�qL
�~c�T���1�����D1�A>����-��V��v�4��4d�4��4tS���1�
<b�'QC}o�@�r�/��"����G����G����F����gH�����u�� Y_P�E���m/���:#Y�'Y�Z�(�g�;�g\d�_�sQ[�y�'�����%^�[���,3����_F�{�".�r�����O�
���/�q��dJa�}���
�M$�n����6fwF��%�R^LG%"��d��B2U{1��s�4�����s5*H�`@|��������`p
+�Y��h]-��|y�����������T���c�)���v@��bL�J������/������+��W���}�������6���@��S�W�i�SK���1��s�>�!7��0�!7�:T��P1��v
ZWc6p��)H������"��?8g���c��f�)$�r�s �,���^���bG�=��/)4�-��*���v�e�?��@zI����e3T��.��W����_��d6}�Y��p���I��E�
U�M�W�z����������+�n�o�j[��h�v�K��a�����]��5�q�t����}����UpT����r��/O�[��9���S1���],�%�ykd~�T�&��s��]��VD����x_�F7[.Y�S�L�;q1"�
�������%):���I��I�s����n����y��q>����=w$=@#�C��u]%�������k+��-�b��$��A�]�����~T�������L��K�zp�?�p�&�IH���Z�'���O��t#>�
���U�y�e�����:%�[�"��Tlg��l��q����'�q~�F��	O8+OQvPc ����}��v�����������U;+]��^e������R���`?�RF�,�C
�?����2A�*'�+g�+;G7S��S��/��pv�0������fu6�
��`mah�]�:��X
<��0���Z�
m��Y[Zc�-���x������e�Z�����z%:\�nV�s��R%:RU����1�:��5���D����J��7o����c�e1�c@s���K����1���cHs�)�[8���1���cHs�1��y}�c�b������y}�c�b�����c����6�����c4Yl���:��OO��)�$5�7T�y�������-���GR8yO1�:�����?�7������_r`l�W����_W����|�����$���z7s��j�l[jZS�Yb�A��-�"c4�xZ��Y��&#��a���]e�5�j2&
��|L��RY��&c�>���]��5�i2&9bcEp������	w����bv��W����&}\�g�>.��I������
j���5�#L+?�V^
��y�*��s{��7����0�OG�$^o�H�te���������C�h�	��V��bJ�H/�8��O���R�8*�(=�`C��'����o(-N$H�c��G���!Z�g���=�,�j�;S���!�\^L>��N��U���y�l��i�<t�A��P�F9��r/����&>�<v��h[�����z�����L��|�.����{�`�]�{�2B\�������.��f:V����@�i�;M*M(��>]Nx�:R�	�i~3����rA'�40��~�&�tW�.'�u���������>Z�n�t��N�L�Dt��Rk���P�I�`+P9�d�?e��1gJ���e�\A�N�E���v�����5���PFU�%,<Q�	�����ue;��)�Gt.��(k���(<��a���O��=i���%eBe�� ?i�F��O������"e;��o��z����������<�2�����������v��K��+v������<��w�������*<�� t����1s���
��!`�r8�����,�Y9�� Sn��?��i�Y�*���c�e;����>�-�N�,"
���J����e`O}h��1�����j����r�6��-(�
�Y�0��5�v�j��"�0V�,��JT�\av��S���Jr�
T�h�����:�*F���a����R�����@D5�l���B��l���'6�)�~:�r�s���h����6>;�M��'KU��hg`k#n|���9�
���v�����F���*6ms��nU�F�}�����	Xl����6���aTe��o�!���#����8�=�#��]��?�r�%H>�#*��NN�����������N�L��h����D����8�)�Tm�A�qJ�S�@�q�&�d���T���)���&4�S`��	��>Cy�H�� ����"��$K��:�� =����2��@x[�S�Z@��3�q�$��S N)�S��Oq�K�
��aJS�'g���<�uU�6<�+K�;�������������H�L����.wf��sv�l�������"��+vE
�����:}�Q���5���'!p]��hr"	�L���AhM��I�\��GN��������������km��� ��x�I6-��x���>.�d����;���X�-���=X�~�\����O�ko�B��$:���5Y�g�r���9+�k�\�,� _�Ak�5��I�+�n�*��{w��p����OC��%�������������>/�0w�y��[�������HP���3�,����CY_9��^lQ5I{m}�Y\�u���Z��l�`k�'��$����U���*K]�Z�Q��z���m,L��*:��`�G��Zg����l��Z6� ^�kYk_B�B uCn�H�g��g��	��Y. >����#;8p>�L����r I�-�	���	�fJ�--hI�n�������Z��na��dN�(�G��%�=�CEj��.��N��3HY�4���&$���u������9��*�C���tx�C����J����:���3{i�w��~<m�'���y�P�d�~}��|���	9�U7�	!��sB���!�!tb���A�iu��uB���u�!vbN��x�����n'r'�:�������u���~�� ��2���4zi?�~�?���.�G�����@,�#����\E'�#�{	���~���,����G~.-������#��`j�G~���G~���G��[{���$��6	��TziG���@���.�H����v	z�Q���L�=pO��	!	Op��Slq*O N$�@�R$�d	�)����=�F��[��`Skq
lQ <����Q�	�'E�I�@x2Ho@x�]� =�Sz�m�
�'�E�8�&����Aq
�)EqJ�@�2Ho@����� M��4�-�^����������N ;%��@vR$��d	d'��W�S� *�N ;�.L��*:A|U��P�H_U�@|�%��
r �W�����$��
�yO���N���_�.'�W�����!���XliuV����F/M�G`������G������[[�?��
~���`C���~v/�~!�:��@�@|�%�#9�����E�#?�t�~{�9���@����@��/~���c����/��"����5�,W	�<c� ��d�Q��|���w"L�+�������0I[!-���0o+��������D���!��e��lul�Ji�J��<ED:?�H���������5��D����ADn~fF�F�*�&
\/�o�Q�����L*�*`L*���&h�����|bk,��_��x��eW��G��Bom+[it���fQ��
�uM}Gh�}�H���<j-���&�o�n���e����
�*�j����VU����u��Yo���v#����N�����J}^-�2t�nd��U��
p���R�W@K&	]��*tw��L�����eA��F]���'�*�����rZ([	o!?&�K�����`�Z7����
t��fw��W���]��+���)�O�������>��O�T�SP�������>�+��K�i}A}
�SP�������������jO�%������k��m�i�@y
�S}fP����� (O�����<�)(OAy
�SP������<�����A��tou�[T��-\1��N;���5"��jB6I��L����M��j
���8�QYA���
���j@�j�]���V�=*�QA�
zT�������G}��(����U���Z�bh�]��m���QeT�A5+�����)�NeAw�Fe�;�)�NAw
�S����t��;��^�N��
t�����ts������"����������v�H���IP���*F���fF��B_�g��h�F�c�:�0�h���������l:Op�d!���,_�����-�*���o�?z�������,4��l��%�����!�ers���#�����\��b����u��,p�!����
������O�4~�x�^��?���
����f��fREZ��G�e}�Y}y5i��x<���:?^}��������f$r6J�C@~�h-O�"Y���n��f�z@�u���|��Q:��'��=��\��_�*���o3]�Q,_>]~�����%��d�|�tI�SM���@Z�O&�^��s�;�qT��@���*��U����
A�����1�������Rq�����bh9i�H��R�:x+Ga4X6�Km���@����Q�w�7w��X�Q��@���(8i,u	��` ��H��������%�� 0�g	i ��X�2����e�',8I�d�Y����&CP�$��������<9��0�zLV��>�#���
���F������M{��`��d�&Yc[�l��-@4�`Kp�I�
��uZ�1j�yw��7-cmxw��@O���n4^��c	'�%�d�7@�������U��5Pl���%�o���[c��Gryy.��>'E<��A��������5y�$F3���
�[8�WV�\����>_ ��[����2s�>���@��|��M�B��5^�:W��S�����`A�,��Q��GcV�&C<�)_?';����5���Yac��h.�dfF|3�G+q����0�q��;s�%����+T�Q������x���"�]y
�4o��xG�_�sQ��<��,�<��_����h�<x>?����@��^�i��QZn����*�}���70'���������)��Hl�:���������OA�|�Y:6�T����'c:��)��gA1���Ol��d6��hXi�)S���( es-�������e�\�K�%o�U���7D#��OY�[p���F��F�`��?�a�cV�*ej��HJ�=��/)�-�
�f��L��?�|?sJ��?#?�t��p�f�dOL)��_b6�j���������^�~p)ToQ��d��,|f�s	���������v��	1��bl9��X��{�B��N=K�����1f���%����<6�#BU�-'����$��|���AL����v���?��u��py��������t�L#�< -(u.8J�<��[���3������f_	,����or���b�D��x����$9��$'�B�MYNi������H�F2N�o�r�b�/�@���evl�������3��
��!fu��,�hv������5
9�"2�&��Xc_�oi"65snO�n?H��95E���SS�
��Y�;�F����cWVn�Nm��5w4`.�'�
�|�yv\5?��$�D��hn��/Sy�:<������G){��bx���D(2���^-����r"���vA�z�Py��[��n��{���=g�<C���j���^gT~�d�?T#��bC�=���OO��)�$5Z���KJ��#y����N��dK���^]���|$����)�����7�0�G�����5�o����V�y[�������b��w*P�}����7'�p�j�����Y�U�<N��)�������R}��1g� �01X�sSgA�(j������%���������E�/�I��d�J��a�|����]l���������q;Zm�����&�@�r��7��\.?K��m�6����}�(���h�����������"T^N����4�P�M��(��f@�
c9�U4S�}��I1i�8�QK�����7�#���;%�	�H����_�'��Lt��`��G��4�T,"�|����#�����b7��l(S�$/��?��D�Cu��������V�g�S�����_f�Au�Q��?��<��TL<�����b��������'��o�X���1	*q�����m�($��[��q��q�G]��q�
Q-��fJ��X\�X��X����
V`��9���X �I���g�O��N\�}�[�c��WL)�����`eq����V��}�h�+�I��Q���
r��"��F�
Z�1�`�Afg�����6��%�oq��1����0���������8��0��Q|%�Q���{��$;z��w�{k��J-�ru�b���^+�{��?�|v�2�NAW��0�F�@n�� @]�71�dG�0v�0�(01@�B��Y�\�����9�����l����k ���1t�	c��F���&A�M�y]���Mb\����Z�zl'������5o�y����v�nq`�IPw�gKH����$��:����x�'vG��.N&=e��8���	P�;��-�r�l�I�J��}�:�`����������?k��p=[AQg��A�R��:����]���pmR���
�nh��v��=��v�����;u�D��/��{��;8���Sm����3"y�/�~� B�rfP��Ek���4uue ����d��_mH��,���I�������kZt�P4u�td����xW�7�
�����*_�$Z�rM�.��&6���
c��%=�?��(�� |�:�)���� ��g�E���)|
�}���u� ��g�E��`(|
�((�5����hP�1A��o�{�^�rO�����9#��.��N}�(���=}��(���(�*�O���;��r�{mp^<���Dm�^��{���^M����i�z��@��t�!�@�6b�z
C��1v��a�a�w�/��.�m������=\����	�������a�@Y�b�� ��� �����^�j�� P_�l��5`�{�9l�?F�b�"'���\��`s����l�Jl�rX:�����6W�c���"l����*��|�@��d���
P��z
�=�{�ro���{��9�f�r�{��c{
�=�A��=�t�������r��*����D�/_��E��-k4xY��y�a��c��]��v���k������Pf���a�P�VDKe��Fy[���Q�rA�l�"u%)��*�HPQFm�(�SX5H%�SQF�*�:�������VZ;V��P[Cg}�v���K�5�",���	�.�����(�:������-n�����\y���
���O�����t���4^�Ry3��7�.)���B���nK�w��x.����$��iWK\&��rq�%�tZ���Y�-)
�]m�F�W��wR5�IT>��-(�������N�in�Gb_��n��A���H u����H��]���C uK��������}(R��8��i�Hb!�Zec�@��dn��
�g��
d��H@�f�� st	dn����������&������|�
! r7�
��G��T����� x�������@��[�%�
�
��Q�@����1�����E'���p{1���6V��o�u��
"����]���; rK�����@��O r[tD�vD�7-4�A�����_���/��x�P���������D:�/���1~�o��x���(q(^<���9���h<>>����s���*^<%����w4[����z6�'��s�����,_������-�*������c���y�������7nh��b3�+'��'?C�������G�B�r��\��b����u�W���W����mr���'��~�x�^��?���
����f��fREZ��G�e}�Y}y5i�x^F�e��Y�y5i�x�G'���W�.nn1�����������!Z�S�HV�&�['�Y����#��q>J/���`��'W������H�/��,��(�
�/_~�%�����f�%�����>�%��������_/o�9����@������U��']���:bxj��T\5�������\�v����%x;�Zd��,��*{@���Q��j��qR���(�5k�7k  
��"���
Qt����*0�x���b�q�L��y�����#�u�y��j9O�q>��U��O�h(yyC�[���b#17��iQ���|��mu�����D����Hu���k4Z�
������I��h���5��Wn��0�����c���Ik�e�s�%?Ik�7@S�WN�@����*b��`�����������\^�����I�+Br�(��/�%�mvM�DD�������������U2��)����$9�
����\�Oq��/��!_-����GyM�%��y?���w��U������M�5Xv�^q�a���|��81���N�^
�fI��	[�����Y2��5k�����-�R4�,�u���W���H�q9��H���Oew�����@�7��<�������(^~���	1��/�j�N�Y<�������J�SJ/��w�R^�T������&���:3!/'�0�AJ�{����u��'gD�'�S�*[�@V��+1���b��xT�|�=�9VAL�����!����4Vh���g"_H�b�����k}*r�����%����v�k-��oz�����8��S���<���yj�]�����<q�s��P����^�Ci�^����%e���Sa�n��)�c��Gd�S���:g32�R�X�Z*�@Q��Q�4�:d]G'��Vk��3f4`��|����K��x���&�/f�8��K0�}�%�.p� ���Wxb(#���r�/�DI����-CI��7�t�y�ZN
��Kb����y�)G��|���{��_��F$1]/����KR4���I��I����Z��������Ey@.�P�\p��y�=��/[g.
9�#��+���Aru��|d
"+g"g7=YNO��r�qLe7e9�u�ZG�~"��8m�<�,g(�9�BDJ^f�F����Z@:���0JQbVG����fWI�Z�\�P��Zk�W�b�� 1h\ll4(������c������� �1(*w8H}l�����p5�98����r�����b�������
�Tz��7��\.?k�-�*�l�:�1
�����FN�`2r?�8�,���ob9�N6���������v���MH�!�$��D�S8q�3Q�?<�(�H��ZO�,�Vi"]���RA����y�C���Jo��a��d��`m�Uz�a�SdX%�@���d���Y%�V�0P��k�*�W�%�O���
}���r�d��r#���OO��	C����%�����<�fw��]�	V�0��o��?>�z����Bc��c����f��2�G��9�������h�J[�����^o2Z[����mC�r|��-|_ �=�Y����'�p�j���b� Z5�q��y�����������m�c�H�E!6��n��R}9����\r�M>��x�EW���'�z�I�)G��y��"RC�a��kF��^���hU�VS�e�0e%��d&����irv�j*qt��jQ�c�5��� �@�+W�j���SG���}�9a��|�(|V�2r �����M���q����3�;���������� �������E~����;�NF�3�g�Z�9*����$/D�f?��Qa2�XN��2+j��*�������O&���P]}�c���@\��� ������_f�Eu�Q��?������u�Q�o��+��	�����%�"Z_TW_D�E��U��-^�����_�__�v�Z���-%p�0p���;v��Z'��8�`�N^��%��EpP�u�B�g9;��9-q��:����t���N�������4�^��C���6x����>n�q�x7}�\�����e��F���y���w��&����5��j�����S�R�LVQ��Yw6�B�����I�^���b
���3�)�x������Y 0��������A�X@����]�.�����>��(P���g(�)�,�, ��6�� @����t�<4���oh�����Uik�(�M}��������xw�����O$������]~����Y8?���;�V��q=�E
O�����Y�I.���$���>E�i0�Q�F��TG���;Ou��Q�S��r�l��S|N${;��>��i0�'��Ocv�O�C�����=P�R8�z�}��f�]�At*��J�n5fPw�}Z�6\���m�L�|�2��i���^�C��"m/�'$D�j(�9V��
)��P���v���lP�@��U-������u���L6�|U3F|��$���P���y93^o��5��sI��:�7���
�B|���6Z�v�K���k^��Y�rM�.8�����,|ZP>M���1k��t�^���H���kZtA�4u�td�����t�7TD.�{���������4�*n��5-��4�GhzP�^��6I����4��$�
��Nd�E�j��
�3�
=��@|
�}����A��P+xIB��5Hb�T2�E7it�IX6����L=0��Az`���m�xH`7��z�4�X�@�@z`E�@z�J=0��@���O�=0[�����l��.uE0(�A�`P�\�C����`_!j @��W���N�8��n�@�z�7�@�Z�
 @�A�����x���^���xT�	�b��@�c�v� �X`��c����=�~����4��1����H��cK���������S�?Vd��c{����1�����=p_���w� �X`=0��A����V-���&	���B��n�0��}wA=0���l�����c2���/��"����5�,W	�<��8@�1�����;su�&i+��2��me�7T��R!��m����e��^,�N0����H�fGmj�#mvd������QW�lc�@�7mv���/���E�P^_�E(��BA��k���e�r"l\	���Bp��-�����aw�f�������-��P-�@oe^��F��JcI��Jc	�Z��R�n0e%-i�doY�����iu�t���Nr�AGa�����>w"!:���-�D����"�  ����J�J@@���C� ���Z��#�[�k��Y��CqA<�-
�!��y �x�a]���v��7��A:�����}�A:l)�~T{EF[��2"�� #���T2"�� #�%�AF�[b�P>���s����x����2"XA:��C�A:l^%H�l% �t�'�����oP:����t;���E6�?T������?�2����!y�_��9^mf$���9�������h/�D}���h�����l:OP��d�T�H��Y�q{������|������xs1����
b��f�WM�sO~���������xu��v3���� M�?H� �,p�!����
������O�S��U#���^��x4V����H�H�"5��U��f����uF���O��x������~V6�S�Q���#DkyJ�*�$w�d5K���Qx~>������'4�{r5�����E\}�f���X6�|�|�i���$�1"�MT��Q�(i�����J��������[qv}��=��n��������-�����/.���!��B���g�:f�EN������o�E;V��#�OlF�)�k{5S�R�U���z�Xj�S��KI�RJQe��V=����3|��7k�����3
����'A�
�'�zr�'Gur('�o<h�4l�<n��u��E-_�����C��1Y%�������7t#��n
��I[%0����9I
@�����~������r����z�k(a-��!zkLn��[�����k����"$-��[��^��f��I
��g43�,�����xe��5y�ah�@���[�*��2s�>���@��|����B��5Y<]_�d�$�- T�N�`A��L�����9��jn�47S*��1�i�@��������Y��f
L����w�t�����x�(�@����w����.����H�F��'cI�������{!G�_�s����	����x�v'��,n������[KtQ��0������/U)���}���}��'��B�
~�� ���n�<n���������)Hk�S} +B����x��J3�U>����PR������*��N���/��E�Q���'R��r}*�Z��eC]_�U��U�2�%3�}+fzpt1��������86{��7B��?��9����?�
���u�<���5z\-_R�5[<�U2�g_�J��?�h!�,O����H�u�F�������W�������*�����]*������~�]*�Zi�F1�jp���&q��.��R
.,��B�x�%=�y�"c%��r�<�D�=������O{�T<LQ-r�������OD�A?�I<�I7�K2�N9x��\'$?H�b���y�MO������W�V�MYNi������H�F2R=�.��uV�@������z���*��`*V�R�������t��UR�c?N���Gj}���1��>�&eA=8t�T����IN�����	�$�V����@� �R Z	D��W�gr�3Q9��x�2OnP-�G�����|�V~NVOI�4�{U,�M9U,�����
�����QI��Em������m
�7���JC���!�T?%>g�.��gj-��WK�055-6���
��Hb�9Yl(������i�<���F��~}I��p��/O����iy�����!��}�����|x=��(�wL'��P��c���A��8�='W�����G�n����	7\��7���kP|5�Se*un������R�X��S�/f�i�������sK������fr��o4���Q��_�b���M�S"3���6�^�b������e�~���*W��(^
���d�I��r�]��
iqY%.��� ��J>�Vi�.N����Czhh�:������x�l�7�l�e�TLi��J&��5��	-�gNA����^���������?q��O�3��d42ScW����R�]QpK�����3�IK�wE
.����+�qI>^p(�,2������C�����������4��n�2%<����T�����RU_��K�����_����?��+��/^����@�_�__�����F���f07Kks�U[s��P����l����e�����5#�?V� ��Xv��,��8�����p�Bb��:�����U��=�����91z�~3���7t!���_ti��F��z��W7�f���gk7
[�y�X���������e}��N�������X��QL�r������En�p��[��f5p�j�N��(�������vI�������!�0���W�_�����vJ��������m�M�
�+r�� ���m��2���{l��"v�I��*m
��������MwS�j�g��������WZ��eS���t�����)�a�.���.�z��H����������(M����Nz���N�������\�2�M_��C�]qPE��@�j����e���H0���o�G��d�7�i���@�jY���r�C�����M����&R�nJ>C���?�j�m<�3�r}�VS���>����q�m'�3�"�R����J��������@%*�';�'�wup*���v*P	�J�>��%����@%�xQ�,n�0�:�	@9,J������(���%hj��&m��2����Y�i��
�+���Vw�V��}x|��Z�A~���(������Q>���r����:�����
�+���Uq���`�����o��
.���
�l}��K��w;,�d�K���X�A%*P	>�>������=T��@}��K��w�6���J���w����e:[���r�F���*A�g���<&Y�p��1K�VHKe( ���0@�h+��B�+��
i-Q+��M�g�����S�1��b#�E��bC<ZC�B;�Fh���Rl@�-�@��6�S	���^�Tm��E(Qj�I%x#��Ap��3�#Y��bSzC]�,��Z�I�h[�����J�^RW���W7������*��B����v�QKa�����|g�!<%�N�'���#��t��Yv�n@����n@�aH7-�
:�fo���#��hl�@��FB�mj��l��y�d��m��l���)�6�c�q[>�#)�6���,��n;��Q�r+�9����6�l��%7@�NhxR��,�k������m+Q����(d}�j��v@��.�;�e��\ag�~7��ZT6K(��GR�q��#6vm@��,�
�6�u^���m~����N�ry����{~z���O�L�h��<���j3����/	z�?��l�V��)A_�g���q6�'h�/��<���MY
-Q�<��d�V�K����l����l�K\l�^��sRoB>�mbr�Hp��e�7���w��P�|�������7�h�.oFx��?��g�~���~���W3^�7���^��t���B�����?\���M.��D?������Q���^��x4V����L�H��������7�������W�(�����:���|��W�J��������[\������[L�pe�����F����m�)Y$�x�����,YH��:9?�|� ��S�������E������U���&�5~+�a�=���L�&����	W_��Yt�Q,"_�������/:i���l������������|��u��EU�����$�����/_v�n��m,�@���u%�9��<�e��aUd�?�r�=�H��F�TF<)��
������:E���	����r��VV�^����M�"���@�i�������4���5���oI:�P@���H$
�x!�)���\z��r9��=)���y'l�24O7h�����������yr���U�����}RG���������a��1t*P{��@���S����MY�H\9Ma�^�(zO����7Do���1{�#��<wM��"W���Pv_�K����<Ie���L*�*,n�<^Y%sM�f�[ ��[�
����N���_ �C��LsP�Q^�wI<�@&���2�����N���'l��w���UJ�K�������S���b��9_d4��C���WrK�lId�If�|��4�L�M
��00�7k@e�k@?
8KA]�}��������w����.����H�F��'y��p���u>%����>�����%^���� ���������[5t��]C��r����W��lq�)n`�,��3����?�n�R�jv�i]x���h����T��R?����L5x��z2�Z>�r�yn[��������nHfc08�����k���QY%Go�>�q�O�[\�S�u}��.!E�h�K �%���gV����C�#�)�S
��a�,�o�d��N�s��P��9�^�����Z���{�xB�dN����-F,��i�?�jG�o�����eN�5�L���E���������G-��6��M�jw��d�6�:J+����������CM�����m�������������Q�R\,��A�����x�(c���r�i�D���[k��Jr�-2E�L.	e�c�ys~�>3h���l��o��o��o�������H�.R~�E��C����,�'�Y9���d7e9�u�ZG�~"��`���,g(�9�BN��������b�WU�tS�n��n���t�����L����=������Y��`49��tX�Y�����J6&v���<��n#�����Ne,w7���K����j�
vH0���:�H'��������tDU=����Yp�l1`=r��C&�T���`9��[�~�����]0�/����x]3�k�u�`���6�e��g&�
�g��&~zZ%O�&�1��__R:���S�gvwZ�%;�e\������GRo>����kla�`H�|�xD��g2���J	N��	�;!�:�=�NB��/7Zx���._���q��*>�������YJ}?x�����?;�����G������l�_�*��n�Rx9����\rLM>��x�����'�z��>)'
��yw�"hE�b��kF��^���h����ML$���d�I��r�]�-\j���W
��@�|4��B�L�,��kc/iX�:���E.,"�L�����Q3q��bd�I2�&�HO��g L����l�j{����A
Pd�aEt�FzG���"?i�d�x'cq���bx��F��-N���?�����0�U,vb}���b���������'bi}��>�1T�?^.��}��8���/����������=*�����>jh��e�8q��3�!�$_D�����h�������3�!�i>Z��������|���_�__��z���������U��wd��a��
�+��J��X8�X��_0Or�Q�,�`��X� ��v���\�c�_����O~Kw.8�Vh�I�P�H����j�Qh���:G�vS����x)�
h�C�V@#��zvm�rh{Y����F�Sf2����]�M���l[��a�H��5l����
i���{��HdmY��O+����Iu�p� �P 8m:(F@QmQE��(�Z������������8go��������.8d��.�P���P?��x����C�#P��/�����6�KP�%Ld��,�`�;��v6�uU�:b��gk�n�� �8�w	�t�=F���>�8S�8�baSe��C��7f���ES�����\a�ww�
���SWX�����:�����ic�f��_�g�����g����*QL�9Z��s��>Q�`M2��P������Aqb�p;���qFP��
�]c��M���0��5�Yx��,���5���������/��i�I�>C����j�E��3�"4I\���>�b��q����3��DI��I��J�:�)	��� ���g�A��8l
\�`Y��h��V
t��a���
���������]/q�����f�[�Q
<�mx��������[�>k�xQ��	��4�=P�v���E��]mn���5�nu��s�����j=E�m�aE�
���W0Q�s�_�#8p�R�P
�mx�p��� z�`�#P�@��Pd��.AQ��������^�J���TD��
��
u�0�!���AC�����z������e����B��h��m�;�G}�'���C��a�w:}����=��t���E�]m�!t�����{��_���e��#���.xd���Q���@��=���j������/_��E��-k4xY��y�$ @�1��.�u;����A�VHKe�0��� a�h+��B�+��
i-Q+�MG�g���M!��~2��Q��I�4i
v��`xd�#;u��4�W�q�j����B82)��L�*�&���U-d��f�M����x��W����YD!v�#D�Z`z�8Z t�hZ�m���g�#+`�z����������w�m+4�m�4P�Vi� ��R�8�Uf�:��,����
U���m�i���]G-����Bdw�T�wt���Xy���P�F.�������$FT�A�7��>�� ��T�>�� ������}�A����(�H�]K�Ng�9-/I`�]+��� �Kja�}�����_�@�a��2�t���;
�~+����@�? �����������
Q1��?leS-H�����_[$~��A����*A�W�$�v;
�����'������������V(��������>��y!���*Yd}��A�WW�>�� ��+��iad}Q�����t;���E6�?T������?�2���l�/��<���
��I���M����5���6�h�����i���j3#W��l���	�b�A����l:O�����br{������/������/������r�k�fr���A<���AxUY��C^�����������9���Uz=����k48>�+���u�t�a���-.������-��pU���f�F������S�HV�&�['�Y�G��y�!�F?�4;��p�����o��,�j}���b�K���e��G��d�~��#��e������w�D��00���������~�P�C���������Nk���jE�d,�������VL�*
*U>Z����P�#Z60�C�����3F?-�<�?��s��=����p�h�<n��u�}�#�H0�����'�8f7��*Y�'u.�������(����:��/����n��h"��$K��
)rI����Joa\<J�an�\!�+ �.W@r\.����\>��s�<���ry$�������(��/�%}������F�F��*iR�A)L����|��1�qs;��a����i]�I���4��#:\�9� JR��=� �U���z���y�?�#�S���x|��E�y�'�<��_������<����_F�{n2���:�C������S)���}�����`4��M�%�������f���^N�
�B�������Y:��T����'#���)��'���<Dx����nHfc08����RU���/%������%Y���'V��hC)OJ�F�s��Sh^�$�]F�W�����R���j������Pe6 ����1e*�r�������22"��h�~*�*�Z6�F4�Z����<;�VW6����R�U58��-*�|S������'O�e��@N�K��9_s�j������^(e�V!U��|�y����4 \��Y�$X��U}��K�<C��FN,�o��E�����%�
�\���s�b.O��	����[�'@z���������ty�Wt�S�;J
b&1]�|��Y�+?G�j��#O(���LyW��m%M�����GE�����k���Q��"���W�a:�B��(����[��F�?g����s�zJ**��|��X���W;,���"������������Z��}��Y��>g�pyQ�T�'\�g�i'�����Y�d�����p���U�o�%���%]����<�fw��]�9)#iW����I��x
2��|���O���7����~�������S�Vn'J���./��e(�j�f 8��?�SW]?<}>���,����yxvB�k3NAx5�������RJi�rr{q3����|���(s����'�z����������p�.���h�f���=nG+�5R�6Q�HW�z�<����3�$nZjr�^���%��=��Z%H���$��Y����T��+�@gK_2T&��.�����c�I��0�)H�o�[���_�J���)QR�R��){�/����?�\��P_UqZ"Y.ZW���*�8]�,����EUM��I����i��R�^J���V�E���j�:���y�����3?���]3Y���vi��mg��/���)��J.���_0L���ygh8C�j��\��
�������RW'���������Q�6���aA��0��&}����v��d�bD�y3���9�84�q��VB��Co��mX&���}e�e����j��!��q��H���q���H�uA ��J�;$;�J�%���MR
x��U7�H���*��K�M"
xHu]R�dg���
7���������5tr^/���M�)Y6vr��D?9r��8�+G�M������]�����.����^���|s�G��c]+WA�$���)-o���`�u`\M6e7�����_��s��D;�g�g�����D��!Q��^�l�)wu��?q�����K�R����.��Ky�R��T5���Y��$�K�K	�� p��dSvxW���
o��	���h�>$
�S����:0q���K�M�
`^��8�9�x����6�T�kR�'��mJ
|��@�5���"�?�s�T����O�^H�U7�T�����Rs/�e�&��R�m�
���#8J���rb�D��!Q p�A{����@�.M6e7�w������v�� p�	�6�C�@ p��	��T����t��7��b�/�U�6�x����a?�n��&%m��T�����'���"Z*���������c�dwTI]U��-��H��6����G]�M��a��f'���������A�e�
���
����j���,��1&�����
������
�4�Z���nYN@��������jeV�T-�`d�JcVU�������������
��vd����M�Dy0`"�De&���
��&�+��+�x 0��(q{A+%^��K� @"�D���Hl��~����B������G��������
R[���h]�" E@���)6�f���[�"�G�[�'���6�;��D�����0"`D�J#v��2
�#�ts������"���{~z���O�L���9^mfd?N����B��z���Cy��M��y����*^3��;�W^���}����v}���t�yF���=$���|�t��?K�yop�W\�%��?'�������~�%~��<A����st��,?�\Ln/��$�b4[lFx�?��K���WC�������G�^������vp?�AR��g��{�������mr�����\�����������'�NQ�oV[H�������@R�O&�^���>]���:o�����Fh����S��-��x�����w~����Rj��,�U�I���j������!.N�cZ]]��=��1���jr��'�II��������w�_6/��L���������KTb53��j�2�L���wC�
M��n��J����F��(�~G�F�����[�K����Y�������/�5E�f��t�|�����5?�5��f\U���JaEaQ	�(	��0�Ui�hM�;���
���{<�4::A>�K����b���I���/N�o~)J����1�EA����D���$5����/�^�����
�W��_��a��
i�iy�n��X������
����!�`	��&;�qU��?�E��3!�R*<%�$qA������<).�/���9�)/���3zA����D���S^�'g>� m��IpzB/���?	��^�z���tL/H����s6��'���h<#�L��2�'��d@���!��K2��}���x=�����E1�=#��c`�$��	r�z,��	����H8;.k�Ol��,��E,_7�00�Z����yr���U�����}R�BFDG�3�����7��u�1������M�L���!������*c;�����ce��t��5�41���y����d�l��n�,���������-4\y��e�>����w�����0�!)�\���!�:��K��+zy>$�����������<�|�^��S�2}z���S���$O1���~���L���e�>e.�����}Z^
|m��a�#���O���S���(������eebcC�K
���-�Q1��l�L�,f;NUV�Y�2f�����h�t��N�Tk��z�������c��c���r�����,R���N�@s�_���8�Cc;��ed����_n���j�l��5������|2��wm����j���i?T��������C�n�\?��G�j��<�O���|�K�Z���O?�~���+ ��A���m���_F��rY�u�����X�,�Q�EC��
l����s�O��D�����!����,��7�-���~cJ����"e��.f�������4��4�9d23����R8TSL�8�Y������_�����E�:����{��@�f���u�� Y6���+��N[*Z;���:E�d��%-���)�O�o8E��u���h��(�LQ�6E�d���SNQ�:E��_M���D�90��a&-1)C3����_~�H��_
Bk����N�]'��~S��i�M�q�|)`���d������f�Y���Kn�}�t��B�6�	Mi��	�o����m�����U��$�r����0l)���%\k�`���Xl�����f'�:X��^P*h������`|s{�B\��J"��/<
��P�1P���O��O���N������
��"�]"��MD�^Q�}"������Q���n�?"�qi����K�
`�Z�:!~����������ab�����q���������n�R�n�R�n�R�v��Q*�!���M7j�\�z?�'�zsGbo(�u/'�7��B���H'���Y�r1M��'��C�S�F�%��P9$��<�6����"��=
qs?����p���qOX���	[O6�dall���$�%1%�����*y�����_W���e�<���b��v���W��5�F�[�s���4�}�e�f�����,�w(��2���w����8�>)y���6�S� [���f��5a	��:��e@8
<�z~�l������a����(pj����j�����F����
3oj?%Y�O�n����g��]�3��F�R��y�1?�fXz�e:��#���7OJO�L���1?��Y�=Fuf���SU�-^����I�_�__�=��s�H�{�?��q!�[5b�(�������[��y�]�C(V�V�.[A���>����o�/�-���%Jg��S��O��;OV�H�e7������0����<b����y�M�6�O3q�������:���`C��)
:�?�~��O���:BU!���\�6����-�����Hc��+ ��r����\��?YC���O�\A��@�w�TKm�%]e��l��	;�G���� �zg�ms�����
q���qY�v�A&���x�J[K:��6]ZO3����/�v;��8Vtk�s1�t?��bh$�5Y�|1�-/������s������:����Mz[~���$q��&�m}f�����Gc>��H�����$������a����z�=��T���sU�y����?I%K��������U3���i��{O�-����SU\v�K�Y�4|�����������e������x[*���K+%wO���c���L��� ��I�*$��2-�<E���U!�=�i*���+�O�t��
��q���RH= �(���.��S���z:��RH=�7%�l�0�ys�bi�@�����=�z�\������{L]~����1G���x��d�\�=4�.�������8Rct���Y��*�%���%�f��~e�LR]�LG��Q�_��5s�M�1�~���WN+�.#D�V�B�ABPN���1�M��]v�'��u�G�����aK�������[~���/Yor�		Mm��V���z����u�[�����cK��XXe�j�
���@�CXl��)��R���D�`~�w%���Ja�������12*��"kP��4*�����%�t[s.ix��'����L�@1k.Y����z*�z��y|YV�c�K�����'>#~�8�g�J�$�;���Y9A��'g�y�S�21�SFR��>#��y_�S��d�}�NA�3�_�w@:���=3��5*�a�{x�7��x �}�>,��gB��,��n*�53 �9�vMk������3�3m�@<�I!�e'?��g>�g���xv���Y�g�hVd��ee6�HVf��cE6�(Vf��ae6�Vd��_e6��Uf��]e6��Ud��[e6��Uf��YE6���g��x����@;t-_��@��i����-w�"�8��u�G���0_�!y"��c�����#nk���1_g�`�J����(���[��������f�:Y�7?_o~���K�J1�S��!����v1���)���0U�a����.�[r��3��.�������c�J0�����^�K���Y����Y��Y�ot"j�<�7���Jn��bV�5��47���}�X�Qk������Yr�+�e'��
nr��ub�.''���d�6}NVf����l�������
l59yM����9YiM����99Y�&'#���d$5}NVP�������H�d�b�Q�b�E�'&L9�����P[3�Ce���W��%�4�Q0
G���]�kA������Ie��t1�2�\O��G���I�/��
K�#��)�$>:��MV���/%���6�����}�C������e��O�f�!.�
��xj�5������KE�����r����W��d�����}f9���\s@��r6���X�.���^�-�U��p�9�.�@��-�b��s�,����k�wY�&�],gsM���s��������@4`�X,���,�����e���f�"�l'f�N����e;�g��T��fx+�y]��q�|3u�j�-�V3n>O�l���8=#��� ���/b��1l)�.y^�-�7W�Z�X�_j�3���/��r,�6��h��������U����n��g��m(/X�d��F!�����d�o�U��u��ZL��MZO�����O\(�8�h�J�2����j�n2y�knd=yf���?y�|�����W����2j�T��w�y�����"O!��
���TNA�8x%��R�Q�NK�JEO�D�2=���`|����u�#���QT��BlG��2P���WH�����b$n�1�c�����xw'}G3��mh$	�N�
yw�}GC�������(���b)����X
w��n]�}�/���a��1�1^'kz\�b�(GN��y�����b�05X�N��5Y��R���i�����h�g�9Y'�i�6KD�@��T��cr�������,�����i�@}~�2r6����70�pgp�"$��\M��-=��?��9�H��Tz"��&�J(E����|dv���5���Z��~E�+dl�n0A5.�U=e��eT���NP���98���
(��0U��9>d,h�Q=2��<}������\����wr*3XJ2V]'�HvJ��2����Hj�d<��B%S�I]��2s�$c��A�1��D%���'=�g\�20��W3�����d\sh�YtL�d>��|j���&39��3<�G O�
LE��*n��<���cG�D��@��H�	��@��4���-''dy�J���e2D�$!.�T�,O��*���AE�|�GuR
��)�.�V�����m�/�4����T��N
*�e����^M�Q5K���\G��6i%��z�UQ�VE'mUt�VEgmUt���X.�o<��4����-M�C����j�!�!8�$8Z!4J}��"����{9�N�!��O�T�������.#'���Ea��5��0�"��� ;�d%����~�a4�a�FE�&�a4����[�=�=�=!��m��M�U�-�����{G�$[%�d�"�d��d�d[�d�%�{�QG���G�M�G���G���G�-�G���G�m�#�TX%�W+�|���MB��g��qjP@e�
D����zDP�s������-���5�����&����k!��l��Pk��S�
�=I'��C������!�.8�������8oQ��T�y��p_�t����B�?�|-��F(,��+�W��@X�����Hq���b���~ji?k�jl����Na����)k^v�e�
d��]$>��R�x�	kLO����H�j����'�oJ8�w��3
��������]mH9��(�,��dW��]��_��e6��o�j�@�v�M�-~�
~�|�9���7�f����������4 ���4�"�V�������T�FZ��J,��J4���J<��JD���JL��JT���J\,���I���x�V�c�J��.?�����~�*{$K������fry������������'�{�E&$��'�zC�AJ��d��}���"��=�H�
��YcTh$O����%���2��"n��(���'��//&�n�-:L`%��w>��qAUUN#-����F��n��L���	4Yy��������^y�qO�Z��c��y�y��K��-���Om��j+N��=m�a�
N�i<��:S�4�+����Oe+n��;��a�
�i<���Z�4�*�7���~%~�7���'H�~��*H$��?��Q`���H�A�s$�j��	��?GBP����A�s$���	1!�?GB���V�G���RB5��8����P
o%"�jx+a*T�[�]����(�e��^((�+&�7Q��?��1��~\i����d�&�of��+��q��e*�t�0AC�?�������>��mV�
���c�����t���%U�JO��>����d����A4Nuh�V;�Z�Qv���0`���F�s?I�Q�NDl[�Hu������<�
~vZ/��3b���D�^����[�t�k��qe=8�������Hm��#�
��D&Z�?���o�zW����^Qi����Sn�c����������F���s�r��E�\f-*s
<���Qse�^����N�`G(�$�c��;q�X;���v����\�.=�����
m-�`ek���yp�l�
���.����U,%�V|���|�_������]i���5��[}Z[m�,1u��$n�����}�R@k���z����/n�Y�>������z3���I-M�����;����K��/Y�*�oT9�R��;��&' \��=��&'�\��;�R��=��&�b����;�R��=�R��=�R��;��&'c ������9��.�9#�z��]��wS�Qv�
���T��8eB9�*E�1�T���mZi&>:m�c���1��.�U���{l��O���c�[�f�2v9���,���A.�j0NuOE���v9�8V���r��Yv��~�U����T�S��
aN�]�M�jJO;����xc9�F�v�!�����=ulVN�w��Y�����6.++�������fcf���O63�����</����
g����~l�@<$M5�H/����
�������]�|V��U5�X/�8���r�����
�������*�o�-0��e�����e;5�vf��\���WP]��������V3n�o�.[���-��	�����Dhy���l�p"4�
'B���p"4���D������~�����lo�n9rs�X$+/��l�^����~'9�n����
U����T���Z�����V������P�����l�E_��i�8*����wD��.�������*�q�1�k��>�5��w@?'�'�����'�5�e���t�+������sD���:���W%G��7�T����a����w��g-�%^��y2w����~�v�t� r'J����� r����5@�'�Q���A�`��`����G��`=(�]V�_]���dE���(�����(.�fED�hnA��(6X�#"�#���<��"*d@a�"2���
���+o������A�?��m�g~y�4�h'��t���7�����Y�U��U�=�5�%48p�,�Y��,k�8����qVg����#q�����0`�,�Y�,k�0����aVW��`P��kY�%�$�b .@\��q��f�?��=K?T�����x��&o��eZ7�-�[5����}��>/?��B�v��@�,Y�d��%d�?��
?|�{u+"�,�����������LG���t�>^]]�����w�Ni�#��'9�3�u,-�[i�������i�(/��
��7��a8�`��@�]�fd�
�t��m�1jLU��I���l�y3���V�i��TL��Lu�V�C6V�o�s6�d�{���c\h�g�7i>��vt�����UE�p��M��c��a�GJW
"����6ZI!
)���V�Bc��.�U��$�������1���5;@f���P��n��nEF����Uhv��;
��4�F�nJ��.4�����jW��b�
4>�HL����5�"

t��1

����whpj���kB������}1�4jZ�����l���~�m��Z�X���RG�2�i���V���e���KM�2��-���;�	�[�/5z4�-@��_j�h�Z&�oy���������}^�S%�����NWv��4��U
����dX�����.�������������-%��5��zA�@��fw���s��iO��F�@��>�?�����-k����I�������)��P;X��:���r9X��s�;S��x����%�?U�����>guF�������6���=F��u�d��V��4����3��,�����&�����[�a��&������fz���=��MU�ewZ����9�n�����w4�����>QQ'�(Mo���
��3%m��l�V�J�PKZv�-J�����VKzS�zGIhRi�%u����6�����oJ���z�ze�����+1w��jL��1u]5�2�	6nA��v�A�)�
�I�@��M}"�[�`�e�IP^
%@yi�-P^jS���V���{�k�h-���RXwm/ln���eO���z�I�����	j4�1�����Ve�����v�M��DKS� #�_�	�/�������-��j��vU����]�N����:��>�Rp��&p�D�CN��c�EN���]e����W��29m����O�<1A�:�}�i�ts������"���;w�p���������D:@W�s�%A���b�9^mf�4��yv��fk��<���M����;y�y�^�����"���,�����!A�G�J��i;����xs1��`�
b4[lFx��?��K���WC�������G������������ID�.�^��~��A�\�z�	}�wdA~7"?>�	�O���R����VsBN��YZ���O7�8��5-�Q�����z�t`yG^v@�=o����3.>�nH��S�HV����1)O���EcL]f}�����m����|��I�����o��[}=�����a��/vi��a�TH*�������[q��-��0K�t&V�4��f/�V��|7���D���v�~.����Wl���
v����Z\I-O�/������	QTl�1%��	W����2��'�q\<Q��]+%�iF�v�f��9[O�:������YOm�+Lv�uN����#.�����*u3�6����c��v�2�U��V��Wh� ����N����`#�4���r/��r�ZxQK,j1E-��N
9c��'��C��[��t�����>P�}<<$��V�9�5���x=�����E1�=�o�c`G�$��	r�z,�D�������	�)�7B_g�g��y9O�q>�J�U��O����;������
�-��������p��3�,�i������r�q'��A���n����#V�	�����+O�����������I
��:�:$��K��<�W'�ayIb
K/�!y�\������0}�\�O�|zyJ����)����O�2�i���^F�S�����^�����@������O��G��rR��,|%��\B� �>CX���)����o���!����E�����.+�'�j�6�'�����Wb�t�2���Yc�C���yKd�j��Mv3�&���`c8���#	��@��k�U���)4�b[[�@�l�� �~���	�t��/�j�'z?}�t��
�Q��l�YjHB��������������1��x��������9t=���&����!���������h��ES���*D����^���4���2gd�L`F�����v2JMG��:�]4aj��+�f��oQ	����e���-i�z���J>���f\�.��NYh�[�xb������5O�&�2�\�y`$c=�aM�.2F�]������)���^'/3]'��~S��i����Z����s�"�X��f�Y�����B�q_�k:Z����������zj���3���k���_�aOjte���U�o���]-_?�1b_��j�C;��q�c��J#�%�z����1�\-�iE�Y��T����'����D�K����vL4��$��#���'��D��x�iw.��]`O#q��h����;3'd����&^����t|�^@C.�pY��@��{����q��!!�;t���fY?�������q13��|��"��e�Uv`��P�y
��}�!S��J~��I��F6��sS��B�+��G�$^o���\�+s9����\8�[�SZ������(��/��>	�������t�N���q�5%	o�C�^3����=�	��S5 �qC�,���*��D�'O2����R�:B������{��9��	"���.�f}|�T����]|��O�	�	����P����A��g�=���U���i<��A�4��y��/Z��0q�f����VL4�x���k8�Mn�)���w���7�� �3K>�Co��~����	0?��Yi�������<&�������z�%���1������i�l���X�/��9����7�+�0e�K<W���zN+����rc7�G���Er|;{I���1���*�/Cy�K��4����&��K���-��S�����_>p��'������x�f�c���w���#?�\�[�1!ptj���I
t�sn��������c�5$��������{�5X�/�
��4u����;�$s;6�h<��f�j���~)F�o=�:��t�|�8f<*F�����#��F.���8���Z��r�Io��2����j���gx���}4�������G'����^0S�������RT�����������F��O���x[�L�h�����$H+�{�m)��#OUq��3<,���i��oK������e������x[����������c�������WD\:�V���X����)*����
��q��h�4�J����4h�����
�Q���������"���L������L�P��"���XG��%������s}������pi(S7I�������C�N�$������V���>-=������� ��B�^S��t����M�{��\�b*���S���rZYw� �Nau) !��]aZ���������;n�i���<�v`�j�
|�I������?���?%�c��h�f��������4E�z�@)��<��%��r����z9
#k���]�b�e���8�g=g X�_���X��3�S�z�3P4���<kT@5�dY��� ��t\���r\�SF��>#��y���t�H�5�h���kx��<����F���d��fp ��3�����VH{l��W���`[��lK`���6�>[m����=�M���e6-�-�i�l�Md�lZ[f��"����������E6=`-�i�j�MT�lz��g�!@��E6�D��*M
p*M�E�l�7rm�YC�m!��B�e���b	s$�=c��:F���7���5f���������Z3���;(VCS?��9a��+�dA'�S��2����n)���K���^U[�sW���.-if��A�{���%��d����D�8�5�l���Hxot�}��{�
�0�)��@�<j�^mN��s�p�&'�v�99����b���������>'t�99�[����59�����\m��`\uH�4A��)B��q��S�����2"1��LhF��������FG�p�N�ps#��}�,���$�&��9pa���0��P4�.��
��sY	RW{�N�	<KL9�/:&p{��:�����O�QE���&��
4�l�����H��������6��Q����7f"��6�r���|����:51�k�&>g��@���#�1�t���l�Y��,[h�-2�vb���,��Y�s}6�_Au�j�����e����P��f��l5��g�m�\T�O������������i����%w�v\���J���Y��j%cP*���`d)
u#��
�8R��L�S�{��� ���c��T?��(;1i�mk��w]�~�IB�i3U������A��p�����"��$�D_\�Ca�V�ml���Px%F��	SD��i	l(`�\(����Ey!�
�D����_)4CQ�����nS���M�W�;����bD'�r���������ywa���b[*�C���S/e��]�wf���`�e���n���d�c�N�4:�b�(��'���J���r�F�K_�N��5Y��R��i�����h�g�9Y'�i�6KD�@��T��cR������<b]!L�019��	�VJY<@�3����\�1�?Kf�>|]�o�4#����(��f�m�nfh+�a���F���p�1�o�����4����&��2"���S�N�O����T�U�d�Z��H�.g�c��_Rk$�VK2�J�����=����S��Z��jbRf��?�48b�.e���f���o\sh�Y4
i2��d>��|f�9=/�0~�@u��8T� Oyz������� 1@b������9��a��"OR��M�X���P�������E�x�>	��AE<6��:��$����k���G<�l�j<�lP�KT�������`Yj��r�-+�m*����z�UQ�VE'mUt�VEgmUt�����]_�jx>:��K�j	�|��n/vKv���u[#k����wX����H)^=p�70��������[�,@L�
g9l<��]���hs����M��{BS��j��*���c�G�H�e~$�'?�y��|��d����"������H��}$��>�yb�|��d���wV�Me�p��������m��h�Me
��&�5�*����y�J�5�5���19lu{�����bH�%Zo' ����;��!�19�P��	a�js�*,o��[�yS�p_�~� ����������{<Rq�����)�W�_l�P��������f��-�lY|a�M�`�����E�,5����;�Or��)����J�^luw���W#��^nW��������3V_����� �\�T����V��6H{�Vy���[�#�Q:��?*�G�}��G�~��G��H���H��� H���0H���@���tcb��*yLV��>�����z3[�oP�[�T�\Nn/n&�G�����_��O����o~(/���<������u���g#�A��i|-a���By�^��-��G��},@P�8�F���'+./&�n�-:LX�?r]Z9� �o����n8�L�1���[sZ�����,*_�J�����U9h&~4)7x���4��Zke+N��
��0�O�qJ�4���7��:`NP�b���5�	������G��v$�O�4�=J��#a����H��$�9�8I��}O��#a3���H�!%�9�M�����J5n�
V�q���R�[e+�j�*��T�V�������*�B��VH��t�Hj���TBm�t��4��wdJ25��7��[����b��� ����F� �n��$�m%7�W(�V��*f�3{��[�I���='l8�J�azpew��!%�e�N�K��wK&5��.�~�~�\�oEk������u{�uS����d�g�3����*s�h�����?��5�a��EA�{.*�Z���[����l}����ZT�x>�:�IU�p��:�r"�:�Lp�PKkN2Jp��:���Cp�:$I&
\8uN��P��zN���S���!8uH��U���C�l��:g4���g4HkF�h��$��`4zpF��>�!����/��"�7^rs�X$+/����X�����	�H����<��Py4DS��q�|)V�3�$7��e�y�����������	�&&��wT�|G��t�,W���_T��g��,�=�����x����������/W������y�G
�$�`������4��Q /�pk�v���)��j�\��q�<S����T��%^��y2w1���E��#XE�V��4�a`�mcC�������no��|F��9
�8��(>�l�a��ay#��gg�
2r~�7O�����i��N��;��7��i���?�XU��U�=�5f$4���$7R
p#�&{��:#�����	}k���0#`F��H5�����!3���+/bD���#�
B��j��|(�/�5�e��{Jx ��=q�xJ�&�+WR%�J���d�d_�R�$p_�����G�f`E������P+�kr��h7�o_���Uw;4xY��1�\������G�Kf:A��.n�h����t�_����#H�BZ ���@�l!�BZ ��y��A4����F�6��m���{����[k�'#
i��-��Ys�����i�p<4�cZ���l�y`�FA|���e�Q4��`����Gh�=,Nw�S6� �M;[��"�lc�&��Hv�����t�"�1Q7��~~��;j�G��#���t�
:����?���}n�� d���?�����E�/�,��rC�����_�Y$����~)_dq��?�����E�/�,��rH�����_�&�����}^^����������d{�R}^18�94 :���D�5
I�^�Db�0H$����I$����z]SH$zi��DR'��C"�_l��$��`�
Q�{kA��ST�Ml�,�9)��&�\��� %o��n��+"mzF���+=�������
�V��5��i���W�����d��X�qe�me[6�����������X���c3�g�t��b��-R�6$�����D�jW��l���T��i�U'������v?H��"�v�Z�^�e�v��m�j�97���BX��v�()�*��IG-�E���nR���~s��[�Y`�2�������-Q'��m�����v��bo����`$���y@o�Lgw����� ��R��tq��E<�'w���fF6:�}����t��f���:��]II~*k���[�}���HHM%��%!#���P��n�DX��:�5���x�iC�8���7���Fj��$(���0S�����Cr�����{�����l����P�hS�bok�$�oIb�z%��O7�����_.�����;1�/�Is�i��t�����K����u�A������E�>_/_�y�*F��z�����d�h-_G�5��\�gxI����B��BZ��|D�d�F�x����'������Oi���
z\��*^<%h�>_,7y<?�\Ln/�I@�������{����s��_&7�o?����?���?H� D��{�������mr���������'������'�NQ�oV[H����������SZ��9Ok�x����g����2�<���B0QZ����k���>���������E��7��:Y���`<"�
qqJ�h1tu}�vd<���������O�wS}+�����I�]'��M��6�I�e����.>L~�����������X�=�/;�O�|����q���&���K���F�������2q-�<��|J}�)��S�����N)�Q��f�rx���RaR\i��g�tq:f��8b�`3��Z%X(���n����g�����C=����Vyg��vVc�y����V����8
��T��:V���C`�$=��ire������)+�j%X����Uy5���%g��U�U-���T�d�G�2('x����h��vh���T9���x���������D�M
8D_��?^O./>�xQ`�����q	�gF�����p���bN������[�����ee��:�<����yr���U�����}R�BFD��3��L���p�6���{k�M
����L�M+{��	��a�u�\\W1"�xi��
FL���<fVlJ2f��J7f
��x�J��K��	�<M��K��Vi�;�
���xD���f.����^����%y���<���e����^z�iyI�b�E/��)s�>=���)}Z^����K?}�\����2J�2��m����>-/�V�to.��M����S���(������ee�n4����0�������+����l����<+V���;Z��n���v
F�Fb�����be�������1��`�1�Y���m���(� �G�4�XU�#��4�-#�l�����U-��x��V�} ���O�����3Wm��3����|pUrtHz���+�G��HP
~�����rN�����zf���O����
��~���n������\C~]��}!7%%������&�//]O$7���'��oH��Fz�W����o��M�!	��MxB
��u1��������
3�C&3c��u���b�����=�t
����5=/�-�$����zy s3���:�u�,D���||�-�hY[�"_2E���t�|�����7�"�u�:PQj��SH��~��t����(0���u����t�"�#�l?0��a&-1)C3����_~�H��_
Bk����N�eO�D��=����p&���7k4��xE�X:��h�t���A�����m9g�J��������r������y�d������jv���5������+ZI��7�W,�E�L~N���vN�
�]m��'�z��'Hv�@'�qR�����C��.��&�^����>�zFD�����HD�����4^l���dc)`�:�:!~����������ab����8#������������p��jp��jGp���������R�
�D�xn���P��Z���<����x>W��^Nn/n&��*�����z��b:������F�O�)m#��GiX�4by(m�g��B�������{�~�{�zWM�z��%cc��E%I-�)Q�O��V��m�]��Z�~.��1���O������yi����nC}�L��]|��F�e"G�'%o����sjt��Yu�����u����	�"<�z~�l������a����(pj�����2�I���7�'�f��~J������-��V�*g^����^+�Nc~����(�td�G�Co���l�.�7c~����{���r?7���H[���$�_�__�=��s�H�{�?��q!�[5b�(�������[��y�]���t��+9d"/��U1�I��s�H�og/I�%���y����=/"���&��K��&��R_������\��/O����#������n����������nS1��Mjw�GI�����ymez���K� ��T���������.�p#A��S(��W��{��`�
���$�&#���
Np	����c�E"���.�:��z?����������_��VR8l_���!�NV����7����PQe��BE��&3d��G����	���S[�:��f�W; S9z���L�"���Y�~�[��4����C-���b���5��+�?�C���|�32��?�5X���`2�*��*��'������d2Tl��L�&�I�Fg�h8&���,<	�d�s��g��y���4�.&�E'��r_�dh+Xj*dD
��f2��E��V�h�)b4��f2�F��V0�(�M
���f2�I�������JB�(�<���Dy�lU�C����g)8T�*��!�������3c�Hp���	<�����X��\O����t��x�������\��z�O�����F�8pq���o�8�>Hb�������S�Q��,qcz[�!�#���h�,�K���?v7��yT1*;���w�.~(�����kd�t�bf������K��s���n��w���K]���X��r��y�����Py�[5��R��u�P�o��ean�d���r�n�]�l�]*�,��e�������js�$�������
���yf������p�q�&u��Oh����\��zsfV�����#P���Xj�\��_���_��8S�O2z�&������D6��H<A����H��l"�e4��1���F�N�F�7:�s��
y[g0���v�Dh��e$�Y&�4��/��St"��o���b
F�����b
F���f����L��
�q$q�c�Vp
�����y[�)G�7�
�`|*YKO��k�N2~!��=c��c���#���A�g�/�+{V����Y�n��m��y�wg�"���6��g���g,��;{y�,�q�={V��3�g!k����e,�%`��X���	�\6L���`X>�%�rY/��v	�\�+�g�X.�%�r��,���`d���15�3���+������[^K~��w5��TV]k�)�vH@��I�|Z��e=jh��`���k���3��C}l�����zk����n�r��S���[��[Tk�uR��|��w����<�������������?J
n<�V�t�s��<���������o�F6����;���Nrn!tcVd7V2�V�����Q�Z���}YG9��1+2M+9,�e����(�E-��������A�^�87W����R�yl[���@JL[>���@J,[.����C�[����B*�Z>����C��Z.���@J�Z����C��Z.�N�4��E��Q�bK�G��"�y%����x�rK�xH��2Q[UR�$G����d0���5�����ch���G���RD�+)����oM<Zd�
q��5�g��X).3^R�8I�u��J���!M�[Y�V:�9-u��,���Y~�bh �����Q����u�40��67Vu�s�,�H_��-�w��#i9G�r.�]e9����X�U�����:�y�,����G9���P��rI�y�/����U����l��]����\g���am���������_����
���*��6��
l�6v���M��fn`�n`g�`�����
�W��E`����"��q�`��P�)r��c���8�:�R�H���c���3��b#�f-�;�"!af��M��*�))�iGF�^p72II{���K�D��j1����g#��0���f,p�em3Y��wK!���q�is�`������+�
�z���o�9����L���;)=yn�Y���	<yM�;��I�'���l]'N�X��d^Hgy2^'�l���N�L���`GF� y)����C�f�N��Bhp&�?�ZH��� ��R)�n�A����D�������-[a�%�,��K��=�X�{���D�����S3��MH$!Tv�Aj(��u�BM���w��*;� -^�v��BM�#[\�������������8MR.u�^�@��o��$M�U l�>\��!�|	6Ds$}I2���&����m�&��u�]�� Nq���|r0}����*�"
��i�@��~T��A��w��Qb(f@����n�!�}#���ERp"@�Ih"e�+}L
(#}@-&��-�Q��h7} z;
�y]!���T��"O��)��_v1A:���^�C����A�=�C�����
�����wd�r<��+��\�Mg��%��ND(
����1P��g�U(�]�O���A�����`�W� �C��`��afe���fu��K�6r��e�ge�O��<�c=��E�\�%��*���Sq���s5�
xn������<7�7n���
��^�r���!�������6�����>�Q� �?�&E\
��D�K��o��H�mjtM�}j��F5
Ry���U��Vy�:�H)�����dAQ��r9��Z4n��ISM�*h�TA�Mtf+��/�
��+M�%#UK<{7������{�
p��1��nS�'�'��Z����!$H����j�52=�����$Rr� /�E���F�!�8��8���[F��p���2����a��n�8�>|�z+Q�h���]��,uZ��_J�{���d2��A��=����4� �E@zo	���L{��a2'�A��=�D�� ����{��9�jb����a�[�5N�q����
�&+���5�"(���a��Kg��|%���-�\��*��k��h	�eC�O�FW��$U�0������P���"I��#���+5"s�����nx���wT���t���z�m�>���k#]�P���[��~e�%]oFY���g���tr�~Z��)����u�����OqX����Z${�����oI}`r)#2��|���>5O�v�����t��(���`������*���q��.�J���qQ�#������G����s�:��\h���4���)�?�
�5�6�������jw�����t��C��?����R�������65R����Vb�J
[�a+=l%����$���V��J[�b+]����e��J�6�M�IV�$�\�^���j�
4I!W~s~�������g���Z�(�|@�s������2��-E�p2r��" ��?�~�W�4e�Q�I�TS�����q�o���J������W�.�r��`�G��|JI�V8���_}
�������&WS�C}Ji������*R���pO��0�a��i����K�X=Z�����&��Z*Mnn��
����;�L������i�q�i���4�9�0�wjO����>9�*���G�������q[�����vB^��������'cF2��O�p����
�O�p���O�O�p�����O�pa^�{�mx5�����L��Ws>a^�#�mx57���|WX��s-2����g
�&y����:����^�qA?����0Q��67��Z�2%\��!C-��xO�2R\I���4h�=i	+�c	���U��QuJ���
�l=���M������MX���B_^�Ge����r
������U�:��UV�Rd�\�rZ'�
�J1br
���D��r�wA�X��qUzp��j��jS�M��<���E�J�����O��������������a\'�3���\Hu(��A5F����Cq�)���Pn5Z��a�NpT.�9�H���G�i�#Dn�I|�^��
��C�7��];�{��E�w0�{����ZJH���qmww��)�T�	
u�O�r�R�iD����~#��O�����*�CT������p�e>��7@Mp�����d�Q����8V�L!�~)����A�0D%n�[owj�c��\u��U.��k���/e�*��@}��D�,����C�q/ %����J��|H9�e�����TB^�C�/�!�����J��H���R�#����]�BN*F���w+����4`��_3��&�����q�����T����c+]��a�C=Lz��6�im�d��'����0�
�U��2y9G�r���|��������R���;�lw978V�.(��<��3�-�������.����i�nN�]�u��?��z"�]��w���0���\ ����aS'�*��ii������	�*3��U�
j0J�n2hdF6JFp
����v}*�m(Y���#3H�m
���T@gFX5�(L�iE��j��)P�z��P��z�Q��@���2�l���Fn`c7����
l�v�v��+��`x��-v��q��LX��e��n`��)rh~|D���t�G���}Dh�G�|Dh��v"B��>_�]-V�[����j�l�xu,�1������B���2�o��H?���f���V-��U�RWt���h���f���	6���J/�d�������\��W_�����s�����P�~����O��#u[�����d�?/c4Ro��f
��)G�!��.�JhU*��� D;@��Lq��6��������x/����w���Wv���'�DYa�u�o��9�\�|
P��T���uP��E�Zpp�Q��'O��z��9VY��"���""�"2�@�jeE�����"<��A�SQ��"�"�b�������&U(�cB�I4��X$`�+�"�P���"��h4	���N#�0#$�T$��h$&��D��d�KPY�=�32����Y���t���<�U�tOgy:�������N�GHh�`U���Yzm���d�'�<������{2��Y���Lf�u�?*K��5,�$�^��).Oqy��S\�����}�k�G�c���q9����[�����k�����VA������������GCh��h��-�Y���D�'�<���,Od��'��s�?���)b��[��|hrFd�0hUW��������o_�7��3�-U���#5CD�������1�0��<��Z��f��a�30�	z7
i�0<6ge]�JV�nhX'�����T�TY�W��U:U\L�����9��e�7�qj>��\���p���v���0����
�)F��r��I�������k_����
,�4����G�iNEL�3����+��Bjb�*\����X�:�7�+f>��}�q_�g���^��?L��h��0 j���Ym��H��n�d���H��e���8���BoB}XY��7����|:��c���q`��F;��#�+�Xo�0,�e�7�?+����KBT����R��kZq5mH���\�wn�.�m�Q���6K8�XU�M+��T������t!��^f%G�"]H��WX���H=��Vr ,����z����t!���e�?E�*
������.Z]�Z�4/��/n+K��Q2������/�
����-���v������b>/����b�&��c1_u���{�����y	_����5��K��+��h�p�^���${�Bmaa�����{A���m�oM�W�����/x���
�!��Qua_eqF��m9��>�J#ES����i\2)���|�>U�U��
T������>u��*T�����>OB)�	iaU�YOnXCr ���hS4�!����S4�!a'}:�Ck�?��%,ZI J�'�Z����2&�%��!V9LZSJ*��&mDb*�����>���5$���&�$[���PP|:�/�|RB�����W���y���-1������R/��k�cZ�.��;c>]}G/�4rx	�K��3�����)kMz����/]�����O���N��^����/�|RRKc�5���u�U/;"����9�aP�:�X�jt��QIG�E��������$�XGK�.��^�R~��ep0��a�eC(ToU��s�v�zsF�5����*�J����0��A���1�B�:M��0s�v�zsF�5���:���^��e��"���������o��"s��H�L���L!P������
�����>�l4�m�)	��]\/nn�M���E�
>���$
>�.����b�D'����:���[QD��	���qh=R������E��hD�4X�Q�u�.��	B�Nlc��JP��5�%e|^�v��<�I��w�_�_��� ���b��=D~�����m�9
~>���5��h��y���??z	��q��Z����~���W,�~�������[�����b��bF@1������7(���^��@�]����]�E����Y_����0��	�7|~Dq��d�l�q��c"I��A0"��`��f��� M�
NqC�������N���������DD�D���h1��b�����>��X��\@I��Xn7��>:���??�Wd�TO�=���.�`�$�NAE��
����B����������4E�4Y�*�E�4Ey��`�����i4+u���&�����ej���R@>�h����
V����ewz�>O�S�]�����KC�+�RbGG�����4�ZA�
LIc>7��E�Z�Z�r����d;�/:{c�i������"�
C\�x��;"������Ko��v������e���>e��G�j@��
��5��;���w������=�O;K�i$$bRj��I�du4���L��~�2�h@�����er���MBd���
0����_��pe����;{�����_��D�C����Cy1��?��)I�^�G������(������5�_�+��}���")���K���lJ�#�U���G#�:!_�W����)/����`1�!�Y�(����"uA���0�@Y�Xx���a�W�K���dn�V/��/�����n���0�~Zb6��A.���6{mv;�6p�x|��u�d;,�������b���(_���04�����/�y���k��)���H���.^�E���J�A������ �J��#u�����[@�%^]�3TU|��C��0<���=g	e�5�&G�)oxD�6��4$��j���$���������	d��d�7�$��rdeH�BV�U�O��������B��"Dnv�������48��@5U�:.��M��r&����6,��������<�Zs�C
#��q>=2���6�T�N2�IFO�F���9A��+h>M�y�q�,��V�WL��f��������&	���6
�,K�� �������b�7�L��bTU�"��Rs�a��nx�|��#e�)B���c�"su��E���������c������Y?����;���0�z���k����(�Pb�;��cE�W��
��`d�A�R
��6�k�*���0���=L5�)��1e�1%p�U��a=LV��;���)���MQ�X~�Tf���1���l�x��Q���o=}YTd��WUf���*PtET����fr�Oc�zi$d�7s�G�]��m��z�����\M#�|���SD��@;�L��]k��
v�!n4"U�T�t�����\�8,�8�^���Ma�������o2�!5��@2S� y�0�Z�2�������A�����GP�#Xy$_���GGA�J7oiz�Z"�O�Wk�,$}�o6����b=�������A�����E_N����p7g�_����v��;�:�;�p�����x�rH�UT����aY��"�/�������C��n�X��2n9j������S��������w��[.�~D�����@`�OO�Sn��}��[Hj���XO�
�[��@��x&4A�,�isH?�v��a�$�������XE
����0�^M�grm�M<��bf��,a7��b�T����6�EA��ND�j0�����k�t	u�	B��Pf�x�f�
���dh�v��*hZ�-���9����u5�FH�	�G�[���!�9�	/�"��f�S���]���.V�����������G�����+��?G'1�C��LV��
��%�d0@2'�Kv����?����������_�h���M�Wl-;q]9H���v^���Ndz�������
��}�SC��/T�Kx\���#��'606�LQ������6����p~������TV>)���f�a�H�!�`1
���p���(����l�F���d�>X[<:����Y8�G��i4�|T4�Df2��E�SK����t"�������L��h|f+Xi���Yt�� 'M"[��S�����Ih&��"�!X�HEc$��f4	-GS�!>g#(y��x(,k)x%�|o����B��G==��QO�zz����=��������G��������y���:�:����[�	��;E��,��kq�8�%�����B]i�;�Z�dm�b���Y�
�>*{�C2�a��MB>���5��mk��J�1��z��@�����Q�<mim����f���F]	�+w��.����K�9�����;�G��-�4�JU�
�`!54bNq��x� �l�1���bd���H�����	j$�M!�C��� /BN�� �T{a���� '��Y������k���A��LF1g@N#���}�j��#�������b�G�vh�G�S1�#>G���f����L��u�q$��c�Nh���X��x"�y�l&F~�k�F~|*Y7M�5�CN2R��I��h�T���[<I�<II������')=I	g�$e��?n��u��i�2�^�r��*����r�H�GB
�<�1��%T�(��H�*�\Py����#T��A���*�4Pyda�Kr����;�L��zPy�:�������O�4de:����!xGu#&��,w�O[�%�x�o>�����]Rm��%t���j��n���NB6������UU������=�{<�;M�{�{pA8�*h���U��"��"�}�>�E�y���{�*E69�m��G6�����B�2��6�e���6	m���6�
�������3�)��ydf�Ld��3N"0��d�2N!.��B�=�N&,s��2N"*��d�2N&(s�r2N"&��dR2N&$s�vBF��?�}l	�`z9�"t!���B=G�%cL$�t���*�����x0L�I�k	�w/��IVF�E���B�\I�]��	=��bo��H�����h`Y#�,IUp�.k�A�IUx��)n����0vJi�a�@+
T���%��V��0�h��+�X�}O+��P���5��'iM'�������Gg��|n�s[�Tp*��,�.%��e#Tiu.'�����	M^��Oh�<�B�X�q��:���d�y��&��c/4�u������Mn��gh��<VC�\�q��:���d�y<�&��a;����x�
,r���M��fn`�n`gN`�0,R�qX��F�<�����
���o�b�0sx������Cr�8v
����2U���N0�.�Gf�M�a�svZEB�NH���Uw �2��+��+�)�l�#�c4�b��jP�*%����\x�R*��<
9�O��cR�iB�$��{n~)cQ4q�q����G^��������oB���-.��C�WB���f!d�M���itm�haz��$2Z����m��*��<�0��d;q��*�%�W:�pK��
N':�4eh�o!0�� x)�a��� _)1�j���X����A����)�v0	4�Z����e��&���3�xh�!�sI��~)��82�t��[�Tb�����N^N+:������IP�"%J(�����!�����[Y$�-d�g�h�����b��>����*%h�����8MR4`�^��$�o��$M�U,VA�p���d�%�:>}I2���&����m�&��u�]�� Nq�|�\��D����G���q����|�iP�6
����VD����k���GO%u���P�����1
����4��\�){��vcOo�����o��6���
�*h�����]����
��J�VM�cJ������&��V�a�t]�#���
8�����3��	�z]3�c�j���C �C�D�u(��4�t�:�*���Q`��5xZxV���Y	`�g5<s����% ,rd��,UWd �r��D|�>����=}X\����V(G���~��Q�E�%�]�8��L��~&&�A�F�d���>�:�%�rR����n��Tj�FA*�\�k
5]� ���3k
5^� ��f��l� ;=�TAQSM�*h�TA��
:m����

����S��������r�{SE��c���vm�4j�_������U���k8���n�ge|��T�l�e��%�{\�]���h$N~��1�)vZe�8
���>���'��'C��b`����P��.�z����V�8P���LY,���8:�.�z������8$���z����v��
��D4��df�0�z���6	�O&�P��#�L:�eU�l��$�vF���L��U~JIcJ�������'���'�>���#F9�����|�����#Yw��\���S���H>��u�%��DN�����N���	�<��gp�)�|&������v�r^��b���M������S��x��z��Sr��D��'��u$Qnr������IT�I�3��h��r��H�Y����~x��~G�N�������G�����%�������+����+����+���,���,,���L,���l,����l`7�7�j�:4E^
L���U�����fV��J�Y�3+�f%��D��L�jRM3�_����`��$�d5O�	��E�]���@���7�������x��Y l���������zY&q��8�`��U.<��^f,���vZ���2���/iz�j�����W�Lk�+F\���o^����It�r
����xl�SG>�U�W�6I��u�f���$7O�U�j�H�F]G
�?������:�u�)�S>VC�"w�V7��9S.g�.&Z�Q-���v�4ho�%2��-�G���U�������gD~H;!�8���2J�P<�~2�:��2J;3�z�5��Q��P�j�viV|�O�0�z����3�m�Ji����|�x�6��E�mX53C��R>!�`�K3O�
�f�hV���2�������D�36��I1�K0
���}��)���9�&�2���	Rs*o��
�s���x����P�
p@�AW�����O
�#w����
��4Ts�����gg�@q�������H��?^-�����
w���n��t��Ac�N��N�\N�N+���xzVJ��gtc>��c�{*�CTa�p�p�?	${So%^h+WI���#t�%�qi(j�Q�r=P�Q-(���P�m���<�[�V�>�}@P%�}@#����V��G<�s�GT�|@�G�Q��&T��Q�7�G���Kt�D�Kt}@�G�Q}@l���j`>
���S���:����@��������:nm��u�����)_��`���Z��-V7����*���:Xb���Z'�jX��4Y&�-i��l�w�z�e�B��Z���Q�;���1J��6�Z�X�����	�$��k���e���7������x�ZMc��F�m��5������
:���L�:��,AG���dU+A�yLTD�@}i���6�!���o��2Y��Y�m_N~G#���o���)����D���Hk"v+��������T�q��S�,5���Iz6	�(��H ����H��w�%�sl4�
���D>�f�!&��
Q����eT�O,����X������b��9�����;;�����E`��X�;�������W��0�����
��e6��]��������>���>�k��r�R�?����O�,�?�jt���\���9��#��3����+\A����-�?�jt�������9������5qL��~D��V�0PK�|�~���{���<\_��Z��B��5w&f@5�i��0�F�&4��f��c����N���7��5F);��M0z-�]��V�g��;��
�����S��z;&������(��������V,y�k�O��L��9[��|���*�:�bGMt��=}�t�,g/�m��qjf�H���J��f��#��jkm'���s�����e�M�@/����U3�,�y�>��Z���*.�
7W�G��-�7V1�W�J��G���+�%��j�������e���om�vZ[8ip$[c$�����W���_�s�!{U�!w�z�a��,uGU���]��U(~��w�����O'���<u^U���)���o�7�7��S��b	�?E����HB�O�.�",���S�I	$��Bz��?E�����@�O��JY�*<��ZZ���W/;�]P%(~��.��r�<AB��� �
G
���H��*��E�8 w2H~v�������s�)���Y<��`�CYAT��.C����Q3�,"6��
�U�7�t�x\�7.�R�����.o�F�Wm�Nnr�
�+�������mX����s)/����2���U9K�v$(p�<�����O�� �9��<_��-�	���En����v$
�maAqe����.ehP���pa�u%
G��]��Ki�����������Q^qT���������B��T�����p�������u+E�����T���n9)�����.���T����b����&�!UYAz#���b���-)Ol��:�����@��(�7y%��)�D����k�;!%D���xC
���!�7��":��&_��m�����j{�k��4�i5�a��l��nD����6"[rct�*�K��V�3���<	Mc�5�����YM�����1�������Q2]UJ\B�S�J���*WU�tT���>��U�5rS�&eh�����xg��:�_��^u�W��7�uMqZ7�������d��2�T���T�Ej�9%e�����js�O���\bK7����spq���Wd�^)����)����}>�h�.���M�����%�C��j���N��k{l�%�TH������^�	�f�Zo��u�.��I�Y��EJ��^��$�d�X����:�x%�X��QE�Z��g|�7��xs�|�����k�'����E~�(�a��Z����~���WH4��x�<���������-fl+f����ys�2�~����T��;����� X�3����x��=��~�;�������M�M.�d�H��p�cW��A�l��w����t0�����=������=�?44�:�Sp#h2�bf��S[1�dD���Z���t0�&#RRj:h�x$^�	�SP=��J�������;e�3���v
A��2kx�"x�"x|v8U��
v8UQ�TE;�*��b�������TN7
2	�h4�i�A�Y����*N� Sa�J/�����\��u@�c��J�X�8 G����<7VA��/����"��-3�����Px
�]Px���i��H��M�f4��Aed� ���2I�I�r���@��N�����;���w�����=�B;K�a����2�ok�(C+d��hBS�r���"�:��#����A�]/�c�xi�	��"�4����A=:����
'S�@%�G_�/���X��!f:pn���!�����@�i����^���S�1x�yh��� DCR�!)Az%_gS
AHG�[�O��Rrs6�4�l@�0�?��t<S��Q�����G3��@�4�����1�j�-��e<c��_�jr�iD��4��t������3�p>��}�4�j+��)���~���6����a!��������?��]��O�bG�2�����/
^�~��/��v��M��L�����[��X&l����	�o?
����9R�����%p���TU'�����$���{��Y	���p9�O)Cd�i�&���Y-���dX�#��<�e�qR��>�9��!Y�aSy!lF��f��4�)�*]v�.�x=P�H�STS��2KA��6�"����e��'_�r8a�55��G�t��6���y�Y�����ir���r���=mIC��4�i��"��A�e	3r�gR4�t>�C��U�&l�b�\���6��G'-�ri>a�bP���Vd��u
=���7��xk����f�pOgL�4�>I�����3����i��38�7z���F�c=��|�Z	�E�=�k�EMc��y,y,�L�\X�����6z�XQv�j�e�5�wK�����o�x�<)�3R�Z8�rLm.>)����(��7���I&��� 3@aRG&yb�������O�#�
	�$Q�������@%E�I9R��p���z� ����F`7���]�K���_����KW=�LR�u��2���e�\��p��_�z�&��Q��$3������D/#jvI���hd��M&1����#���0�<:
�T��J��Y'H���g
��1�"*���aB�A�����E_N���m��I�M��7��=��" ��'�OMo!Ejz)rr��-��7Ei?�o"�����������;{��
#MEcT����#�:l�%U}
����\=�#�^$aP�#��)���)�f�M���B�s�*
��B�0��z��P�`"n5/~t5���rcC��z�pW����Iw~\%�3���F~���Y�,�_�� �	4�Z���Mr�((�D���L
��j*�P�����=�������Ed��������
l�a�
�����hI�"k�Q\{��hd��pi�u{���M<����g��X��"�������#U�z{�9������O��h@�������I��{G$E����k�%R:	>H���	����m�M��n�������:�"`t��ZR��������>����9�������X)����Nd�������
��9�SC��^T�Kx\���#��V606�<T�����������0��i��h�T�)��f�a�H�!�`1
���p��O.��!89X������d2T�������,
����4�b�+�
O"3d��Ix2���3#��d�a�&��`�!��d��+��y\��������<��i%O+yZ��J��V�D+mU{M�#1[�����!�
gLW��sj
�kb����LL/1��e|$V}�������vwz�49�5e����rTV�N2�a��6�4�-��h?�(��G���S�v]6H5���.�������r�,;��r%Mq���\�%�d6\{�8�Z��hT��c�����w�������a�1O�i@�������+&��n0��@��#A4��4	A��P�
#���9�
}����h(9$�x�F�J1�W&������D�#^&4J��T���4J���k����8
�(�y��(�����D����L�������TR��4���Y���7+�?���'��'��k.WR-���'��,�Tk���M����;M���R��A��h*�<Py�Y�K�	�<�,��%�T)&����*�Py����#�8T�.��c
&i���<U]Y�gW���}Z�2�Q�����m�P��9��-��C�L5�R�I��.y%����:��Poj�g�C_��7��}Tq��z
l{�D����=�������0����N��l��Ll��tEz7�s��r��6�t����S�q<@�a�l,Ee���e�kmD���K�m�.�7F�5'�<�s!v'�I8gN>���7;�B���I�[�L���)d[�L���)$[.�D�����Z�B���I�Z�L�����L���?�}bq�mZ��?�v��E
�m�����Nm�L�V���do�8��`2�V�����^�
��������)������?=M�bo��H����82Y#�,IUp�)k�A9IUP��)n���G���~iiWYW�
T��j ��VM0�h��+UV��D+��P���5�z"iM'�������Gg��hn�s[��P�*Q�,c-%d�%�U)k���Y��uM��G�k��<�]���Q���5�p�d�y��&l�#�5ik��[��xM��G�k�R>P?sW�be1El�6s;u;s��`��D��
��+V0 ��y��Q
����r�8v
`����N�!�0/yh��9��{�a�HHFf��E�~����q�^6�����|1=\��|���\A�J.����!OC�l�����W����5�rv�����)�������pX��������|W�9�ufF�����U�X��u�W�l	U�JT)I�%2����\�n�$��R����Sgqf������3�B�U�\����x(~_���HL�6tJ��SNZ3���DSR�z���s~[����2a�
����x W�I�#�IJ����+�-<��~���b�J��*H����l�6/}I2���&��
�m�&��u�]�� Nq�|�\�� ���D�_�d:6F����������gOB�f�>���r�T��l������9)�������U�t��C5��z�����a�+���80�
�E��G�$V\�AG�����.�pM�?��X����"3,�jw V��vaf�h[�6��y���p���h�z���T�0�<�
i�5g�i�Y��2�g%�	'��(EY��3����%��*2���\����\���4����+{\4��C�
5�^��d���!J��^���TS&������0(�H��PMcIy
p�K:��R�5
R)��f��T
��)j���:������4nSM�*h�TA�Mt�PA�u���pX�T�/�F��^��2��������
F�-2vi�a�u�W�^�x���Y�����&�x��)g���x�I�qc�3k��N�<�4��^�&�z���v;�-@gV���L���n3���p��\a��=��Epda��`wn=�
Ep8�\K�'=�]Dm�C}Eh��M,	�10�����MB���p��P�G��`������16z��B�)%(y�6^�C<w����I���X�+�`<�{
�a2,)&p�N����p�n�.�rrf�'O��)�<��O��30��4p���?+������8�5h��U��)�z�<�����p�Y�bi>�6�V�n��(��D��v�8W,����e�E������c1[�X����Jl�y��N��������#T�(�nl��s�{MndA~dA�d	A�d
A�dA�d�����`�b�j,X���
�c%s������;Vr�J�XH�t�=/�Mr�l��<�'���v��oM�X^��_�z������g�P7&f�/#�_F�e�����5���S:��Hv�����S�}�
 MYe�<�_����/�)�����,ES�x_��@�o^����It���8�2X8u��_}pEo�t�\o����Mrc�_XU��}a�nD�u�@�E��z� �h���:. �)��L�C�u�����\@������Kq;J���yN���#S����*�t�w�V�3"?���A
����?Ej(�f?}����?�p_2PI�~�
�����?�OF�f~c��4�!�P�7���Y��������r�Q��l@��@���JR$a]�~�H������M���Y1�K��C�^U�AU���3Q��67�U�{Y#t�7l����|���-�o�+Z
������p��`&�+�$u��sg��}PJ�TV2�O�D�!4��j��&,�U����u3�/s��� ZgZ��dZ!�p����V���-�n9������A���jK�� {S�/�u�v!<��=�V�����e��@�F�
���m|`�����6V��lS�v�����F���m,p>��;�/���6��l���U�E^��E>��l����6�g�D���6O ������x�����C������{�nm���������1Z
�����*�b�U��X��M���%�OY?�i	p\��BM��2�o)oI�J�@�l�w���2I!��E�����Q@Ia<�|�`-Z����2�/0�Jf`��F���P��N;�
�)��/]�h���R*
Rd*��K�s���m����RoL��b���]W�,.uM���/�f�V��	�K���jy�c�0��S��v����mOW���=c��P$	Ct�%|=���@��"�3�lT���G��<r���K�f���_��9��y��qsn��=n���j�h���g���$�G�Md�h�
��?��=�n��v=R�G�Ig�H�#����`g�di=��=f���c��l3W����0;@���9�����C�6"���UM��&
��o��zoX�B� ��*��!|r�Z0�aL3�h�1�0��0��4��g�L�g��mF�N���O�9��(�)"y��
n'��8�B�'Cw\���l>&�������vM
��W�c��I)���) �O6�W�<T����.����O�.�^<�������y
���.���[�}v�������n��C��p�e�/�b��W�@@6�_Q��.wA;'%�Y��N-lF���������j����T[����Sp_ ���]=�W(~��w�F6����l���T�*U���/�?

<ST&���r�5���.�_�Q��3���O�.�d��[�)�U�y�������\Yy��bL��;W�1��L�1�N��^������T��/K��g�<���.��i��.�Hg��;d���cy��>;^D�t���5��1�����3��s0p�uc7>�jkw���9+���J��Q�C$�2�V��s�BU�su��)�Ab��7|=�/{�9�E��Y$R��r^��8cu�L5�}��Y��S}w�G�����ZX~����^t�s�M����|QhU����e��&�����Q�����y��lJ
��~U��~>��������6�c�M��w4>���B��.�����%h�k�q]v�����������H1"�	%���6�����$���k�Su��SI��^sJ
�r���g��y}����^���{�i|�7�{�2���)�����&*��@e�/"��YX�	����j�;�+,�q��@����(�j�}'��2�5��Q�9������d>�SoF�V�i�i5�����+��Wx
�xO��?X�L�#�����6�����o�*�����@�����]���;����=����W�=^����J5�	�������x���spq��8���r�i��B�������\'7��r|^lo��x�,�6�/�q�F_6���H
n�O	�# 4_�%4���au�wqV�}��.pnR^�v�y�&�	��:MW�$�Yo�x��q�������:�x%
o����j;@+���Q;6��xs�|�����k�X��{���E~�(�G��Z����~���WH���8"�����p�-eEne�iY�3}n)m�����ys�2�~����T��;����� X�3����x4��=��~���Gg����
"k/SD�&i8&C��l�l��w���A8����&��L-�R����7��!�Q��D�'=�&=�6�4�`Y.�A��'=���CU72���R�N��P���J�������p��������D;@��������47K��"�ETuQD�������E�-�h����<��&)�F)�7�F�.Yd�;d�y�4��<)�������.)�d�sYF�[Bfe�ksK���VR�BV�T�u��(�p���%�<���x���`��
A�A������])_�!�,���v/Bf�s�^��=����S��������,
2N/�K4��\�x�_N#����$������e�������/3�r�����S�/���&U���4_��$�'}*�=&�nW	���
�����7�>|�*���x�1(��/�N)=�����Q
��^�*k�K���R� � ��Gv<�����1�����er���Mr�l��<):>XD���"p�"iB����x�����%r\Y �����|��f���t��x�Vu��?�X�>����0�D���V�RG0fe��1��H�1f�����k�
Nv���������g������)gc���#�"�������gG���J������~��+:����|�^�������W����C��F���J:����|�^i����~��a&"��gv�{cs��<3�s�Q���h��Wig�����yY�R��
f)b����~~���/i�.������Wa���#V���3}u�:��1��1���rUbj��8�_;Gq^'�,2k��$��q\eA���V�R��t T.�W�'E~;76P�4�4�<mI������=��*��O�����NE9|�<&�������<���yD���e<On�KT�S�I����_�x��![}Au?���O��+�R���"CG4�,�^�(��*��/]b���q�w�18K �7��"7�)�S����4�O���e���C����vq��cO&;G��.����GA�Y}6�*.���;s��-�$��iZ1{�t�X��1J��	(\����.)lmA���E�3^�E�E��(r����� ����W��0EyQ^�N���Q�)9N�������H�1)41H�Gv�12J"�$��9��ao��H*��o��Bk���"��Ci�"$-W�i�(��o�t�}�%�`����E|m2Z�-[�*��w9������.?���n�8PyH�����7��xk����f�pO[,X_\[dKl-���7��Y�
��_'7q%��
7�B�x��],��V����E�s�f��=�G5���i���i0�_i��	�V������W.����������0����Q��:���w�YW#*;^m���W������J�x���U*M�6�����HF<�~?:�L;N�g�����~�^�l's�J�o��������z����Y>��<23��}"�=��Df-�e����~yd�
30�/]
02Q3�4��$N�����E��������o2�2u���d��c�r5`n����zZ�_FW����
���?��G����W��(HSI MOPKd�j��?����l��V�+�����oqs0�o^����It����s���"o��������h����z����s���`���L'���S��-�j#k�"'g�������Z���u�����V0(��(r�4Y{~K�2�!����G���>���2E?�^�~��z�Jw���3�h�1�`'����P�cb3��'����P�c"8�L'���3���{��t�t����rc�|�����X	T ����d�g�(�)+��eY��t�*�����C��$�2]��6rb�s�Q��6�k���X�K
�MV�����r����!5�/@��C��2� E��P��\�o�s����[tF���]���.V������K���G�������?G'1����P����
��%[�+���mKv���u�~|�����1`�C��P�-���"��K����ey��j�X��R�c�:�y/�^
��D*0�u��$���<*���H�uG��k`lr������56:�L�ca.N�:aB��U��{���8��1�,F�w0N�L��W�����
��h|z2�6��N�g��$
g��p4��h��f2�E�3[�<�xvM	�����L9�h�
�OQ���l|�� wM"[�$��6)xr�� ���x<RgC(����&�3e�$!N3�m���&HB�*�<��h�<t�FP��x��)x<��	]O�zB��������'t���}b��Vu��D+c�EF)}����SR�uL�����Hj�2���MhL���<t$PV�����������!7@����d��QFAW��W2���N+>���J>����+e2�Fc:+���w�Z3K]�s"�OtY;����e��=����+7�Us�UZY�e`&�X] 0<�]aN�z�k�2d��RYg�N��5g��V�sa�i}�E|�E�z�y����Xv��j�p�q�&u���e�������zs&-?����K��},���"��*
�5�z4��U�-�7��<	�L$@��?�&�0V���H�A���M�����Hr���q��M!o��;�X
^i��E���E���M#������`�n�X�����
��MO���x����frx���+`�Hb��|0��GG:�m3�vI���Wj��T�L�%U�d�+��+�P5��L�Q����:LV�8wve�j���������J1w���T�Y���yV�>V���wg�"�J�6��R���*`��n|Tm&*�2P��y V�I�X����0	+�$@��RR�I VI�X����	+S$@�Qbg�8H!5@H
���mN�>5�!���+��nsH���Y�]Q����H;�*
���4CA�����e��Q&�c��5VR���3�B^��Zs�W[m6����zk�I���r��SR��[��[Tk�9�+��7���Mi��W��M�*��U��'R��H��m�PZ9 ��t7L�5�b��^R������\V��C�Y���dg4v}�4�1�|���Bu��
�{�"��%����s��������Bu��
�{�"��%�8����s��������Bu��:(�+f�U.������R�\���\�0\���!d6�!qX ��\�2_e��X*�s��2�C�l�pX���@H|!�T �N��������.���3�w�?�f��E�$��G�%�}.�t���*)g���
F��`2�V������_1�d��G�S�{%��P�	�G�L�!�l �
����J1h��������S��/5���������!�i�{\Z�6�����HwBQ��s���uW��s�1A������#e9G�r��X���I�9��sA��J�):���8$n{978Vu��HY�#}9�r�sI�S��<���H_�A�+9��rv���n9����n���JD�z)�.��\�[M���������s����a��{L4�.���gj�yw�:h���j7��
�v���]���9W�h�u�j�5@���:h����s=��0iLA��
l�6q������N�����$,RWPX��j{��`x�=SV0��`x#����I0��w`����9�����8�q��ut}�F���xIg y,�F��Z�0v�EB\��X���UpTRH��@��ndb����e���y%)8�IudNJ��A\���=����H���-I-�y1k�/�-xm~�������i�%F�������u&���������������#]��1y����Cv�t�UVLf�t�����"F�6(0��Q(�j���b�(�� �)'��c�=�a��`,����&�D�=��&�F�+wm���jSl�cf������$���(��D���>NtJx9�:�����h�19e��t��Y�)��}O}�H��S�I��i��$��:B���9k���'�j�`���wq��46�j��8���7I�.��4@;=}���C��l�zI��d��M��
���$M���`�pA��"�9B%��^�p�=��"��7� �����4�|�H��p/�x�Wlw�U����!y4#�P�@)R�@�I�+eD)}L5)#}@U'��-@J�+k7�!z�
+y�"�+�T�#O��N�Udv1A���
"^�C���p.O�PE�����G���X�"�G��t�)=�n:�.5u�	x��C��+M*�����8�.4u(]G����:����`(�L��2r�;C�V��3�����,cr!K�-��%�-{d �,U�Dd��rX��2����x������'�=��I����O��U�r���4�?B{.�4���0������~�RTf�j)*�P�����#���Z���Dv&(��Y��$��A?@����D��2n��I#�L)e�H)���rf-Ea����Ka `)5���5��Ux@�r�&��q�C���qZ�fM�����������=H%�:�j���3-������Pr� ��EZ��@�!��B�8��k[F�_l�>�2�����a����8�nY�ZQ�h���l�2��bX=�C��������S�����=��h�-����k8�Bp�|����=��gp���z�'����{�&��6�u��4��e�n\
zm��C/~2��b��
�j{:t�9��Qr����"���D�B�\J��L�0�9��l���AK�^
xS,A��u?�*R�81�_?Nz(�� �}�M��ic m�M���v
���i�{f��E@�H�iS m��igz�>N�Se�^8�����k�t�Q�taO{�M��L�dG{��.�/��K��{�.��_V����6�]�99�S\Ev87�3�IYI���(*v�Vx W�H|�4`n�_���u�����
��2�#��[�1k�������v?�j�N��P>�3
vew�d
v@�(�e;�p����P�
x@	(�e<�����bP�
z8m(_*�TU��`(6��-��P[T)]��������=Q�~�F�T:�`tN���F�T��`t�S�#�0j#/�Mr�l��<�����v��o�rN�{��_�z������g���#����9��/#��L�tKq�$�\x${x������9��.MYe���_����/:.���y�,�[ ����>v���������D�)�^�9�jR�G���N��WA�!�l|�&�������O)U+�#����[����G���
:��c�s�o�\j����k�4���r*L�C���S�,��O�W���������j�����4;�K�=��BV��x0�z����G|�kV�������!��l���y>���������3�,����a�����g��?=���3\�?=�o���Eaf���o�6����j�	lC��+�
����6��c��j�,C���w�BFU��LJA�$G�X@P������'��9�&o2���0[3�D�br2d�\?.�	ZF�#�7 $���-�jx�7A��Z��P0XK��1R��@�M���!�	{1�Z��	Ku�W�����cW=�N��[9��y��
7����*T
��k�V�1�����0S)�H�uY9V�[����B�B�����JN�h6�TQm�i�'�:LFf�`,%��@�7������1����K�Vl��Ci(��T���TcTJ��<���k,�V��F����B��
�[�yt��<B�����G�E��pg\9Rw����tg�Y��#t7�������ZWMw���O���ur�k�r�R�iDQ�~,�����X�o�\'5���/{[mWX���zt�W���NI�m����c5�4Q����J�
CT������p�f>vn�Uq���^�F��������/d��GD�����!%P�Br
C�a-�;kB	�C��-�����Y��a9�!��3�@HS�N
��c��w-�P65(�Z�i��c��.�9���+���@��x!�� �L����U���[�zt�
7Hmx[����,n�;�_a����zWQ�r�����F8-.g�c����������rnp��]��y�/g(�V�YS��Y�oio�y��}a�����U���Dt%"66������7i��C����d+/lbv�m�dP��6��TP#�l'���L�i�������M��Q�����*�����
*��2L���7��d0�57M�����6�
�n�����:�m���4�S@e>�4��Ae�>P?�+LS,r��M���n`37�S7�309	��TV0�`�;X��j{��`x3�F7���5��a0��@�h��qQ^�.�:�p�������`�>�/�������������~f��u	�K��|}w�X�[l���U�	��u������r�`H��&�d���"F!"����.[A�LR�@�XQv6j�yGP�j
>�&��k��"�XTC�zs��_}��|��2k� C��1��V?%��������_�M����� ,f��5��������t���
��#�4<����+c �j�%=D�I���M�\&�*�1
�;�`((��)�����1kj�9+K9�
s�~p��=�������c������������*�g�DgY8E9ZOOE�h4	� �N#�0#��T$�h$&���t��O�SM`�A�����a��G�?��V����k
s�v�5vv�����>��'�?�*w���`m!�C=�$��aN`w=;����Y�0�����F��0�?�|$GZs�����*�Q��2�ey�QV��O�(ko�&��b�d~����Y�������F�����<�����b��[7�h.Fd�0hU�)U��L���o_�7\���T�����!�rp�E`T��f�c�ad�aB3�i�	�0��z���� ����2di���Wl�I������{4
y�����E�����bG���
��>Y�Wn��?�Pz�x�����\qW$?5��(�)3ZzJ������$0��[+����^q���I#��dd�b��x�Q0��g�hh*�`{dx*�Bn�Uh\�)���w'���2�j���AZe�UA(�����W�m��<�*����7��5�a��X��m��5����5�X�!&���kT�N�e��K��X�����f�����c���o��Z[U(����6�����0��5l��G�x��/����������z����?�t����F����8��O�.�9,���S�	��(��B��"8�?E������O�.�C,z��S����(��B���b�?E�*Y���H�mi�j��w(/�O��������h�2iRY��e�<����E.�W��AS�nqU�}/-�W�S#\�fzS*K����|��dM�]^�&?�K���5"}���#�7��TN/�A	��T#�+H��T�-���K��ld�(��,K�.�i/V�Z�M�-7�.���R[�H�NH�K
���������ot����@w����%F��%��_��?7�du�WY���� ~��P�U��M�-B�O'�_Ii�������C�WR��:�sZZ�q��
V�4p��@�l%����b����q��H[X�F���j���IJ���J������j��?o��P4���<^f(P����m�[s*b�d�U1j=�b
I������O�����!�hVC�P��CH�PQ��Q���0kH\J�]��F�U0i}�deLZK@YC�r(���U4�)L��<U4�!�*}:�Ik�Y��%LZI�J�]a�2�Y���O[�E�^���E+���E�Tbm�1w'�|����D,<
���!-�l��V���B���!�hV#�X���%��_��z�Y��*�Mm\PK�&����5�����D��������9����Y^f��t������)kqzq����K]�����O���N����m@N�tdN^N�dTA�5���V�h�UA�Q���Bp�������9�Jo�S ��/����M�����kL�Tc/}�D3V��z�O/��'�5,�,�����P�Ua�2�]���i������K���}�����6+�l
�F::mT��k�t�9
���Y�t	�z�O/��B�(~��������o^�i}i�\&FZd����o�����M�������-^��E�
>�����@�WI�I�$fLpJ(� 1�x��5��E�������}��.p	������"Mhd��A��tq�D���b�t�+�o�4�i���Z����~�����+i.�~,V��'�g�Z�����������/^�-�v���.����@����~�����}���o~y�!����x�?����������1-kB��qi��~x���\����\�E��^�X_����{��?����QT�1Y%��/S����?�!B<�zh������� F���tp���0����������������.�?�4�Q�LZ�1',�D��P���~8����9���f�o��F�;�JRj�����������������;�Sq/'�n��s��(��w4��f�49�����,G�,G�l-
�����vT�}��lU@;�����4�+:��u0����d���R�?���\��_�V�:�_�f���������M�)"7���:��8�7�<N+��_�U�H����emy��l`.�ge�T.Of��\���)\�������DII#P7��A�O�����2I�I���G�	 )bt``�����y���W�Xx��k������"�3Sk��cj"w}@%�o������&�$�yR�X�y�x����
:�Gv�VQ��a��L}Y�0V`�����S��(c�#�"���c����^�W�5���|�^�W�9�kD�J��!���N�W���&"�x}:��0'E�F�f$���#�HG ��N-"P�:���ix�6�B�YC�2�S�{)��]/^�:��*�yX.�D��e��t8*y��>N:�����d��]?��muyJh�xlg��11YR���N��W��a�~84����/�y���k��)�������.^�E�E��C�APyp�i|%V���Td���FT&�U�o���OU1�KC��0<���=g	��������:K���$��Y-���dhXM-P6Ctx�QR��p�(�<+N��Y�V�9MX�3����L��"Dnv���b
P�g?�p��6�:�L����4�
[e�-��-/O����p�hhb�+���3I:����j�S����"qR�����
�N��E��&�d���S������c��m�I���E�M�>�f0�������d���
>%��U�&N�"�\��c�o���*����^���b�x�����M�1��S����������KS��	��f~5�[�H���������*��o��k���^?VBZ����i������VTi��k|���`X���SwD����D�h�x�m��z����vy
��s<x��:P���m��*�����_��V��-�����P#�j�"y���	�UdM�0�r�O*��m���^��\X.�LF.��L4.����e�M2���>S&����`����2^.m�#o�/^�?�I������������YE��5��_FW����J�x����#�^�0�<:
�T��I��Y�E���g���.&oR0nEh**��z����A�����E_N����b7��u�T�����g���NeG.'�^Eu1w��$�9Q�R�z����,�w�����fT���������~���01�w���G���A�P��t�\K���
��n��&N��6��f7�����`?�*��a����|�|�+V�o���!�f���a-e��,Q��/������$�&��"x�u��;�oi����$�[%���$�D7�2MT�P�r.�9���U5�-:p�5zpo����GfW�9�������n|4�b
H���$�v#���
Bpv��:��6�^r�H
*q	���_.����/?�c�*F�~y|�IY��RjY^���$�c�8Qpn1s�@��L&��bUM�P��,��!�%�*��%<.Rr��q�-��rGX��@DPg����Q�L�S%�E@KD��|`C�$��L��������d:����,
��*$��'������V0�;��FS��(�E'����$�,r����D��Ih&��j4�l�������Ih&�/�!X�H�
�d�h.�6\�������_<����l�~����_�~���E��Q���}�/������
���Q�>J�S	���5��xGb������<�*m/m\B��&&�������*�Q�+��H�,p�o���S�#`��v���l��|��/
X���P��P���C�O��
.����U�h8�Q.���e������Vq�/����1�B00�7Tv�+�Z_2��m�!��)��{i|��Wz�W@��X��H�^R��&�d�*���Id"��V@&#Ik������zI�*�d*�0��H�B���$*E�F�p��r{9�%�MOE/G�����H"���F/��D��yiF/���v�d8��r2�t���`�(v>�m�n����6U�rj�Xj�$e��~�GMI���*��(��2}�W~p�
k�)�Dt���Q.�cA�t�����Mqp;�!@��Fb�2����@jQ�JYd v�B�X)��NMp�R�����Kd��Kd���"�)wyF[�%�x�f>�������C�%9D�sx�u����=M�d�����)$�B�M���d��������QJ��RRI���N�SK�.��������b���H,�u���Lw��h�s��1��9a�A����!�����I�(��y��_%�P�B&L@�&�!dr��g���L��
�b��C!L@��\�����)�X{G��I1=���E
1m������m�L�V�$�e_���`<���d=�D=���������JN^�$���5!�h1�7D��W�H�c��h��*8!�5���*������m���f��Z]�lU�����-��*�+7qHYG�
T�ON8V��q��]�������n����$���Q��v���G�C`�U=�T����Xl��j)�?M	��#��@�$]:h��K��x��9ZF��k�1
,O@���	�P��L��i�93u����`���
l�6u�������9��I0X�.�"�����MX��j���`x9X��r������
L�;��|������Hq>�!�[;
�J�=#Y�H�y'�"���;"!P`��M����`y�ep�b�����j�Q�i|3��z}�C���4T�Agr��H���F0����8F�
Z��<CL����.y�����B���3����UBR&����������Z���I��f���$�X*�RNc�dJ �tP0	4rG�
�En�&���!�Hz�?�sI�A#�?%�Z����a�fp|�w�y�\���:���	����R��u���8/c.��h��"��.��@���+��6W?��'��v/�m��j"��E+��J�5�D�����O�E�wq�����j�^���7I�.��4@H<}���C��l����%�|���O4��I���v�2�8�E$st*����w�@�Qr�`z9D��+%7D�+�*���C������EV�X���UOe�$.�����b��e�	�;%O����'�8����P?�1*d}��[ `�g�xI^��H	�3H��CM@��5��:TvQ0��
Ib�Nb2}��r�i\%X!���F�c+�
�y��K��Z�R4	K@����Y������s�����=���O�xj������'�:����f����K��
�g2l����>}T��~�Z�J_U-E����b��Qi�g24���,%j��q#�L)e�H)�FJ9m��3k)
�IW\
Kq�Oam;]��Zn�F�N����7���**(nW��������v<#+������WPn��e��mW�6�2��I�������B�z�!!x��]lZR�v�Vw#�����q0��6�=��X;�g/=����6�C�`��L�k������51�2l(?M�MBi|mX���b5��N�m�!��(���S���<���k��_�����@�rl���B��hZM���1�6��@�H;���4�=���" m�M��)�6�N��3=
/����JY��9/zq1�qX��)LE�^����K<�p�=�I&Yb*����3��f����SyFv#����e��)!�O�v�����}Jf�����y��6�����\�f�|�h���j���l���n���p�m�J���p�>X����m�4>��*���Ur�`tN%\�F�T�`tN%f�F�8Y3R�
�X6^��&�$�y�O�\/��b5���������g�%���].�7�����to�A�.����HY�7��?\��$:L
�QK,�ZQ�WA���Mrcqu�8U����b������*�6T�Np��5�����XB"?V���y)n���lK w��uk�������!���A���OF�e��O�0?z�M��3���a��d$dx��K����K���K�>��K�h��K����K�r����J�f2��I�J0�B����h���1���9�&H0����E��gDd��
����x���"�
��AW�v�s�O��#���Y��Y�M~T^�U�i�T�8����p2$l�����MX�S58����q���U����WE?y�Q���"���v��?u;t���`�;��@�7U�B����������$�+
E���P�h�j�jA�5���dv~����j�B��>.����������>.���%C��>.�������|\��
� ��|\�n�s>.���>.�v/m��� �|\
�������M�*5����3zo�����&���I=_�]-V��6���j�l�xu,�Z���o�C���7M��|KYG�3��f������I
��~���6�wD%����m�U,���_`��L�zs��_}��Y�9�v�6p��V�oQ�����G������l���e��;av�Z�j��#C��4�v���iGx_�#~{6
?��x�L�0->,s�������t�Q���laP�v�U�b�������a��c/v��t��C@:�NOE�h4	cr�LE���F"��G��h��L0���}����
�;ZC_;n�����6&���	�LP�����3�-���#A�������?�f���.U��x�?��!���<��J8���V`8t�phyu�T���f�Z�?��5���Dp�r�'�~�V'���IA�nM�����c���U������o_�7���`�W�rD�.�!2��0�F4��gYk��c�a�3�y��6A�6������h�{0��R'��'-P����Q��Le����U%��2H�~UA^�T�#���^F)����,���PukbqE��EU��r�C���l��n�Z>�v����*�8�G�:h�?��~�t�����A���V����*�?E����We��H�9��,���^g���"]������S��t�^�u%.g�HsM�>�Nw�&��T�`�(�CU�9��4�u�}Jb�]vv\^PEN�\]})��B�F��2L��:S3��u�l����,��{�)?�y
p&��7��Rt�m��)4���?����ip��k����^J�Q���$��5��Su��SR�����p.�Krz���:������$����(���Ufx]��5D���4�����hw���W%���'CEb��8�DT�)��x����E�����M��KF����{���S��O��j�����2�0�e	�����n��4v��3��0O�b�|��^h�^E������X�p3��$�=vn�y������@�7@���<��c/=�����iRz\P��U���V9�\-�7��a�>t�>�L��.��l
�2e�������`�P����6���@�=�"�B�\�
DN�M0O�l���r���w?������^)����)����}>���`��`�I|^lo�������f��|=� ,��t��]��Cr���w�_�_��#�A+���v�)������6�?���x}���������U?>z	���~�-��w��?���,�q����RN�T�N�K����
,�QT�G�#����	.��Q��)~����
�\��u+E%W#G�y�B��:���$�`�s�b�O�4��&2:�t����u�]s%C���(�p�T�g�\~�^�9����Yz�s�i�;�2
�R�
�y���dC^VY��0�CcP�m,���4��^:��NS�^�G������@�)���S��C
��� �6����v��s��M�������r&C[9#�rpvR�������W�/�D�4����b,H�44��K<2}�����q:�1�>&�d�6�e��h����`rv6��`��>{��?��l��Ij�O�+�Q4	4	@9�$L&�$D�$�X���:��wdLE�F"q�4���,(8��	#@�}u$B�1�Uq5���D��<�.��V7�!�HC����3?�M���C�x��#�e��p�Y�6��Ah�JrZ�RS�TE��)�V.'��>��]�A#�^�as����\Qt������_m��:+�D�-�\k��8���#;��e�`��+�C������������'���<#�U���#�'"��'+���:j������<Ib_B�������'�c8�����]F�4�������)�U�r'���o����
�e,��*p���F2���uH�5y�/c���]^IG�2�
\$�Ev��7�p����.��
����Epu�uto�D���� �1TR� :QTV�0����Uq@B��U��[����>�Z�H�����Fj.[���
�,T�|��]Y�#��J��p�ej�B(�:(%�U���{�q����a��9s�v�R~}R���k��M���|d(��b�����~[_�_N�:R.�Hik�
�����y�_�G|Ao������v+i*�L��p�������
&���h��d�u�u��"*�.������y](�N%������|L�����Jp�z��G��x��|�|�[��9D�K�'a�;�����Y��/�	���ZJ� -$����P�D(/�@Zbd)q���G�6N�,�P^�3�Y#��O -dli�Y�HN�6� -q�f3�#�2d�/�X���o�,_�2�������E��:|�����^:Q���+�"������7Z�C�X}��"Ed7-E[@���s���%)�����\���brZ�����w$��$�;y�SLu�O��Ve��d�>�Q6��Q�Cju'��'W=��0�:u$W�o?I[��i�%�m�g�Q��W�-/]��Onc����[]�u6��F"+��=�J��x�mP������#g���.�{�D<A�H�����D��q����I��e�|g�+"F�QB>#8�������p�bG�V�8r����{�]���-`���n�����V��(�Q���ng����8���"����M��_��*3:2�����e)���b��u.����-����#iY*kY��-�
d{{����4��G��)��P��/���G�em���e=������v��tYW�4���K�_��k�� Y�W�����X��]���y�|\%�v��GN������<�H�6�����ux�����Zjm5�_R�`R�Kp����]������M�fT^����j��a_:h�m�G�Ml�^��JK�-T�b����}v��
������6�j������v�������@u6T��#���7<����*�b��r�e�7T��)3��D�j�_MI?�j���:%n���������8��7��v�6�vM�� ���^�I@�E���t�g��1�����X��M�.�� ��������4��M�]���g.�
Sl*��I�e2�
�q�<��l�w'����ut����\��4��la��gU��;��k�O��zs�F���Q��%���}�#<�aD�����H�a	c
1� ���+I�@�Lg�������K-��RsH�@R�a^I�"=k�hr�b������=�C�����1�&PN�5O���l���[�)�����]��K��:�u�z��,���Isj-g�R�)/��[��}1�n���}�F�d_����\�e]���pP ���T�\�>	2�YrE���cJe3j3��$�1����)3��������_$B�	IE2���h���_�4�8��/��f4�L�3�9�9�����ed�rY����f:��0��9m5
'�/���!��3�|�����X��\��x����OI?�E^Hi�����m��Z�[�������Rg8�R���e�*m�	�J��O9���$~R��8I)���$�P���B9��5%.O���;)����~��3Vu��]j���� ������������]������s|���;�&���-I����(�	��	�	���{�)��T�0q�Y�rg4C��;�O�����~8���E;�C��^<5����XG:r�"���C��������"�u$42y��q�c�R54������:�D`XH�1.G#���G�+�@~�4���� mq��w�� �� ���A���w�-��,tY�Wv�Q����,��F�+�W%����+�lpV"��>\B��ev5�����G3A�
������o�:R�7�}��%�$��/��p�4�_�~J�V�%|��������#T��/I'u�������K�I]���g#u��R��P���$����K�3R��/I'u����SR��/N?c���/I�A���.�K�������qz��������R�%�S���/N���I:�K���I]��=~���K�I]��8}J�R�%��.�_�>#u����SR��/I'u����3R��/I'u����M8	-����Cr���K���o�_����I�)�[���hHC�_�>&u���tR��/N����I:�K��OI]��8}Fo4�I:�K������I:�K���0���=�7��K���o�_�Np��/I?%+��t���q:�Q��$������	���%��.�_�Np��/N'8J������q��F�T�%��.�_�Np��/N'8J������{X����K�O����8��(�_�Np��/I����q:�Q��$������	����q��_�N�R���Gi��tR��/N'8J����K�I]��8��(�_�N�R��a\��/N'8J���O����8��(�_�~J�V���Gi��t���I:�K����K�I]��8��(�_�Np��/I'u���t���I:�K����K�gUh#/@�.@{r1�D~��x9o���e�J��,7����qL�K�6�s�q�p�*����qsG\Vj��*i���{_,-�������L��h��8��"������8���\��.�����Kz�����F��F�,vu���D��ACv�yd���AU�g(4��5��Q"��}�qn�r�����YU�Q,#3	�u22���2�Q��>b<4L${	�306#��r+	���Pb2�m%������q�_/P�-����9
+�dUt�����t�T��%���"��#��w��{����E������z���l���[:�fOJo��{��{m,�^.&K{||������1��.���%������x�V�����.���%U���x��,�EO����"p�����G��9�����{�|TaTQ��1:���`.����S|T�D�N��'��lR��1.>_��(���W��<TwS�X�r?V�U�u��g(�*����K�=C�[z�:������-��y����u��c��j7����GE�W�8������T������i8�vYN���!q�/����'�=1n'������Gm[�����������g�2j�H�K�2b�yr���	[$�s�t��9,�/G��H�R�e)2���H�5J���q�OZ��i�xS�T�6���
���x�:oW�����;T}L�`$�R(V���Q@,�����
EZ��B
�G�$�@!X�P��2
q&��S0�?b�	����M�g��L���(`8ai���[3�yFC��$�X=#�4���<c��1�K�������|��y&4�D.{J��R{H���gJ�L�<3�g&�=�p3�=$iF��L�C�Oua�N��y���?7��%D�/Uj����*	��ODF�-��D���1-fB���tR
(��/o��BNiYg��tr���!)&>7��A��g!������^W2)��T�"�F���o�����cc�q�`c�/��`l�;� ATVN��9Y��������i�@�g���VB���|s)@�[�qY��o)m��x�T��4�L��#h�F��U�N�l�����j
�X������pF
��������6�m-��v����Q����rKX[n���I�7�C������1+k�~m�����y�����D�pY0�UU��Z�3iG�"����n���?}$�,G�'�jaJj$��y.i��^�S��d�m���HP�sJ��	�NW*.�.V�A������gF��5�d�������#��|�[�l�h��>��q�|DKF��������+R��'��W�#�����'�:�zP���$q����M.��iM�%��+ds���3:\:�s��v����C'o������r[��Wf[D�����`�5���9X���8#����!�x,�X��`2�����������1�<��'���'����p�����Lfy�R�)Xpx2�R�������;��|fi1T�,���P}H��\��R�X�!����1��O"p�'�)8����������?lU�"Fh�'�p����l���(���C
�Y
����lr:�����)5�B�(������i���f}l1 ������Q8gO�q%���h�����m�"�UM�I�4�Is�����:Oy��-�
2C���Av�R�>�>]�S�Y\_5����xh��s�d�i��������5k��
W��r�T������}����D�e��t?
K�[���L��~-��*����!���R�!{���WL8�W�k%����<����W
���~����u���_)0�����!������3��k��{��W�+>�]`��W
�f�����h����OY�����.�_�+��.�_�k/WAX{%�c��k%�#��k������J�#��k������J�C��k%�C��k�������������*��R�!{���W|J�0�^���xF�0�����8b��j{%�S����J�Y�'9�+��.Pwt�W<f]�>����u�:�������g����E���G���������Y����.P���WJE�.��B�k/��0;�~IA������T��~�G�+��b�C?Z^	lD�]��|���m��u��Jai�G�3�W;�����^),m��u��J`'����+����OYgl���~�:c{%�3����������Xgl������u��J`�h��Xgl�����u���a)*'���R�S�v�>Z^	,9|�QD>�^	lDmu"f�c{��S�6e-�vD[?b���RX��������i���3��^�Y`�DiB����W;������^),m��u��J`g��3��+�=��?e���RX��S��+�=��?c���RX��3��+��4;���G�+�
����v��Ja��m�>Z^	,a_3C>�^)�)};e-�vD[?b����1m��u��Jai���3�W;������^),m��u��J`���S��+�����Xgl���~�:c{%������3�W
K[�:c{%�g��g�3�W����R�)}����WK)�#D���������WK)�#D���R#F�X_),m=#D���R#F�X_),m=#D���R#F�X_	,�<F���RX�zF�X_	,�<F���RX�zF�X_	,�<F���XJy�!b}�������bX�/v8f��������S���J`)�1f�����R�c��+����)�hy%���3B��Jai�!b}%���3B��J`m��+���g�����R�c��+���g�����R�c��+�����"�W
K[��+�����"�W
K[��+qj4�y{�1n�5����Yd��o=��L�}���W1��������pp6�]3.(1���9��hq`��v0h`��Y��-���cZ*wL��7�S\�-���9�ob���B\.i�?��c?�����`B3�i�	�0������4_�O��������������tV����r��/-l���^;0r�����9�Q��r=���jG���3�zM/����j�m�p;����~�8��W�X�8g|0i���j���j�L��&u�A\�����d��W�O)����f��7fX����S�c�3�u?@�~5K���[L=@�O��������X�xOYz����<�"*�Yz�8b7����xzb����h���
 S'KW�����77�����6��N7���I��z�|�$i�5�?��2Y^�������r�-V��oA|}6�oNN����i8�N���qM�S]�/^������UMf�i�������	u2^�c,w�������?��|}�����>�^�����������j�����������r�����������d|$f����7���b��gMX\'+��}	3G(f����Ul������-B����H.V{/ha��'�!�1�e]�����|B������d���8������a�A��_H��x���1���&
���
o7)���T�N��~���<��L:d����������u��]`�z������������O_��K��q�/��`���`tv6E�	�)�_p���sx��s�*�~Ni~�k��RZF	��1�/x&�P�P��I�a�����#5_&���HQ-Q}Vyo�w&�F�b���Q��(_�|��'�Yy����#*E��
 ����V�����.�BIa|J�����[i��.O~���}�~@9��-���	-��?��v�4]��f�	6#�&0�7mj"f��������/�[���U�=7���E\�T��D��:�qyC�v=j���}���Z��EWc�Uh�A#SD���*�rHo�g=���0V;���{����f��s���z�#^Q�7	�+���]����|by^��b2l"�����
$�`\j�4R�D���f�-�k��xZ}|�f2�o?
���~�&�?����|�g�M������w^e�g.f�o_��%������kd����5fq�i�7��Y
�YA�oL�w�X���D�$�����5������\E�5G*n4�y�Xs��H��Pv��d1H��Nx�+�J+@j%�]4T��lG�W��]�V�����+���X�WJ#��W�.�W������9�qhx*}�S��?Y��H�:��+������T���;	U��o��LD������W6��
�A���Q'����Y�G�2B/�$N���B������K�Kvi�K���� M%���'��\�����O�
�G�%�m9(�^�|�`�=���J�uc�
�Z�W�
P���Hw?�jx�N�nSM�����g7G��6�~x%���nSMl�bU�Jx�*�j�<j���Q��]����[g�.����������|h�}n���;�=�Z;la7��9�;m���w�(w�M����������=��]6���UZ�M���&�|p��tw�����Q�.�����3��a���\���wW���w�M��W��
���)=7��lmH9c������3����s}��4�e�����ZW�(�2�����i'�����an�]���2�����>��������TD
�����X6��]�o�I�aI�K�`��������:�&y\���Tu8Z�73z4����N7mu��h�����T\
M��-p�Xyo��X3�����8�GB���[�x��5�[�x���;:�����8�]o���fy��9���>�v������8��{��-p�����-p�z�������y�4�[�x�X�<��vR��-p�N�v������[�x�������Y���3,�QH�s&���t8dF:$:[3V8�q���zS���M[3���)�&=�����x�o��Mze��I�7��&=�����x�x��&����Mz�IO��;�/�Mz�<��V�&=�]o���fy���9��I�>�v���7��&=���{��Mz�\���Mz�z����7�����4���x����<��vR��Mz�IO�v��������x��������0�!�<��4@�(��	dC��z+�y�4X���U�\�u ��	�@�=��qf_�Ak�'X�`V\-px!��t�)���Mpf�	���/�����.����[�8��*P��a!�����;�xGo����7���;�x��fx�o���w��������7���;�k�7���;���wtp����w�=��x�C���;sty��}���-o���w��N�����a�!������;�\o��}��h�7���;0hyJ}��4��x��.���i'��7���;������1��Zc}T��
X�|�]�o���r�\W_S��M����lH�?�o��iBMxt���l���l�C�L~���}:�g�����T
��c���v�&l����d�����h�f*\��Y�t�P��pA�3V������/�}=�����q�>N�E�&��Q�"�[Dy�(o�-��ew�-��E������[DL��E������CG�m�E��w�7�"j���-�:�,ou0G����������Q�"�[Duox�E�!XB�E���]o����Q�77:�&z����������s}��4�[Dy��.���i'�����Q��������Z����i�b��n�
���kbEL��F*
g�X;QjU�m�K�6U����
���v������%R[�?��p��22��������M[V.!;b�e�xk����(��[�xko���Q�ew��Q�5��F���[�L��5��F���yGU�5��w���e����Q:�,o�r0G��F��������(��[�tox�5�!XzB�5���]o������(�7�8�&zko������NJ��5��F���>��vr_{ko���]}�==k����������vq/)�P������]����?��	*d�
"Zj�o�xK�k��x;�M�Z��`3����f�	��2��)��T�L����@�Th6�����.K��;0(T2.�X���F9Py��e`��B�BAe�b��A�) �J����)�&
,�/$���X�����"�V�tT�-��f�ty[=�Y�W�d,�WT�G��KvY�����\W�~����?�W����&V�^�`�eW�S�l�_���[�%�������[���$
#1��m!���e[�)K���T��[�����\��t���}��Q�d]4�0 ���*�` Wn�u��L����]{�i����C9�h�w��g�`g�p3=+9%��A+�r;@���k�@������Q�YY;�,G�%�;�8	��I�<��_������qB*Q�����#�����(gq����fT�Ho�n]����2i��D�,Ks��cHhf�>��.tDu��/�M�z���w����;G���M�bNx�H���|�!������.+Y�o�G:���(�{�����Z�vd�o@�����ZZ���a�'/&�����1')W���������f)�I��=�S��[_������iF&o���e�]>g�]�(���W�~3c�&����R�&O�w{s�^�:6u���c������Su�������`�n�c�xYZ]G�3�;mnk��"�?`�k=���u���X��5q��/���nM]����u����]��A��.ul��|�u�G��Nv��R�&���]�z�������?_��5u]>�:��nM\������[S����cj��1��`�3�2��{�t�2�h�-el��0������I����YT��xV:�
'��w>�qka�N�p[��
���6.�6�&-��E�����E�C)�I�.&|��fx� ���/
���x��?����&��9�.��$��������6Y�����$X�dFd��-�����+����nz)���z��������m����6����y�.s��������-v2��s+�������n��n�.�GR�����G�|,]�K�������~��hP�����|����}wf�{��Lp]�Qm����]��;������J{j��I�����.��s�������������p��n-���o��!��nLU������nLT����'����U��#M3_�0�x+��]3�~4��Du��X?:1M]����NLTi�����i������I7���GT�:�����!���tc�:xLu�#MOT�L�MG:1Q<�:��NLS���u����c��C�����u����Z�fY���a����	���1���{��~������m��'bmZ1��k>4�Xz2O�Cf4��W�����������a��`���-X���`����5��7��������[{s�}���[{sk�.{s��7��[w�����[{s����[d'��u�����'����v�#���.*v�'����v�#����[���3S~,���4u�
�ZG:1Q�:��n�~u���\O�1U<����nLT����S��c�k���a���u���C�c���4u���ZG:1Q<�:�on�����I�Z7����������M�����=a�a?f�_~O�t����6�&v���4@��ia�Zo��u��U"�J��`��F�`��ND�&������X?���nU�z���M�(�h�
C�q���3������<|���o�^�e�������doB]�roB]�RoB]�����P�t��P��i�����&�����Pw�=��� ;�M���hoB}�=��TuP��k��DuQ��s=��TuP��k�&�����L	�����c�G'���T@�:���� 
��~tC���T�z��������tc��xHu�'���S]�H',	�xLu�#���R�G'���GT�:�������~xju0&4Oz����5��,�'l��l���	��1���{���~��	�����j���b~�,���:���[�o�$F0��dC���;��HjF�]�����zKTme�kR`vM ;ar}8��G��`�#Q�Q���2kF��4m�q�Va������'X���MhDL3an	�,�;h@�^<�=����V�fN�������W�8��8a'�r�[��_��\��
o��-�]��-���4o���x�uo��-�;�o�~������k��\?��tc�:�i���tc���k���tc�:�m���x�uo�~P�Dt��������E*�k��Du��X?��F���s=��Tu���ZG�1Q]<�:��nLU���u��]<����NLT�������#�k��Du���X?��:�:��'=j�X����Z����c�����������=a+l?f��r}�Z
[�E�����*�Z?����:1m�&44�wf�.��e��.[��
��J��7ec��w�px0a��� d6�?��M�GI�����Q��Y�i,��z�W��zKU��z���[�z�^�.{���7�[�v����[�z����[�d'�Uo���z�'���j�u�#���.��u�'���j�u�#���[��u���},���4u�
�ZG:1Q�:��n�u���\O�1U<����nLT����S��c�k��q[���u���C�c���4u���ZG:1Q<�:�o������I�Z7���0P������-T����=aU?f�_~O�B���#��U�ti���"��J2����o��$M�[�J�~M�������~��i��7b�Ll�!���{��<�R��.��o��������9�}���r�����&��Z���S�����������i��0c�r%k[��������D�g��R�nak)mL������4=b0.�b��?I��������%�>~�A����P}��G�	/d��D=���s[����14^�g&�f&�:343`i.3A3�g&���D����I��Yn7��>:w��??�Wi�!TO�]���.�m'%�^B%�w����t��;��y��y���[���D�}�[�7o�~�-b�V���f8U����R`<��v�)7��U�"`D[�b�����
�Puh�|/�t��)�������Q}�.2g����&9uhf�<R^+E�B�<p�GeWQ��K�D\843HwW���:�0l;jQc�0�GG�����/�����o�Y�z������!
��eKEb������������a~$'D/g��L���"o���2�o��br���/���;#�1�i�gY��,VE�A�Cz�:�7xY!�_��eGQq�"���m�NA�S��6�����N��vR]��o�k���T�k�!l���b���j�
�0�����\94�2�Bi�et5���Db���G��H>�3���J���7�F�	(��~��������7�@�qNa&7��s=,�?���z�5�]�Mzy0�$�t��������G,BE�=�q��%'*��H8Q�7,/�5�D5����������g�f !BNQi.�
�8���w�����U�����t�OP�;Q����L�L�H��_��3\���/���TK����zW�\s*mw�'~��D�7�8�>�`�[�hW&Z��X������z��T[��UF��b��\RW��l;�����f�;��?������7���Qn�xpE�.�U%��,����d0�����������m?���z�
%�8��1�i����6{���mg���������Zf�+�9��Pv1�;n.Y��a}8*NX�WB�#�Id�������K�����>w��	�D��4	>���:�e`
CTD2�&���gi%�����������Q�N	A�f�����������#cD��U���^��1�w�����C
w��S�z���D�����,��:��&�V�
�h���Z�����}��L(�K�e9/zq�����o���p;W�*�z�n��6��<�������g���O���o��]�����{���G��nP��F"�n�����w�%K�+���^�B���w���6�mk������D���6dL����-#]7� P�����3S���O��^�����>/���t�����p"5������K���I�=Y��v�0��P�z��Fm4��J�*���JM/�#]+����o�:�z8������S�R����GZp��N�w;���-m��W9�)�����D`�[������:8�M�
��<�������h��Y��w��
\C76��M��7���"�[�0�F�2G=$��g�Vx�cOP��Pcn���������iR�C[W�:>�.+P���Q3�_�������*=l���}/��2<�.*X�����~)S=��Q=A.��:-�xoR�"3�@u��bD�,&�m7��c�v1��~
����E��O��2���:	�7�&Yv�{~��X=w��2�3�?���=8Y<;7�9��m1g������.�n�2�Kl��x��6�a�S��Le�*��P��������Z���+z���,z������wE�]�#�
;��]�{W������wE�]�{W�B3�w�'������(:�_l��b���D��_;��n��]�h�]�{W���7o4�oA���{W�_c����������7��z��+VZOm�w��Z��+��>�]��u�{W���w�2��t�����S���O��^�����>/����wE�]q���w�8�]�{W���wE�]���z\���=���.��Sq���1������S��>���������.?/����f�up/S�l��a�~�����<$��a��ls�d���t}��:.�6�/�4h�{cp���A������Wl>��z�7�M��W�`A�c�{s�9���\>�i1��4Ki�4�����9����C�Y�a���yO�My���"�m���-%�l��x)<_���L�mB�]�{��������%�wI`�wI�Fz������.����h��.��Kvs]y������.��Kv������q^x�}{���/:�]���v~�K���D����L��.��Kzdo��7��������K��1�~�{��h�q�[|Z���%+-��6��dk-��%[i��.��:��Kz����d��Q�0�S������'�U/�����
��Nz������.���y��;m��.��Kz������.y���=.�m]����g�����O����pUG�'�����z:N���.���
�j�]�p#�T��x��![�������6�Y,��5�xL��� �*8��:~����{M�gqw�����
�G�B~����kf��� w��� O�����-f\����uY��g��;w���1��w�G��Ays����`�X�7���\�;����f,�I����'����d�.�K���z��M���d��KF���l�w��]2vs]y���%�w��]2v���%����w�}{���7�����v~�K��D����L�w��]2z�]o��7��7;���]2��1����d�h�q:)|Z��.+-��6�%ck-�.[i�w���:�]2z���%c��Q���S������'�U/�����
��Nz���%�w����y��;m�w��]2z���%�w�x��=.7g]q9��g���������pUG|���.������<>����R�U\��a�C����auM�����M�$<�r�xX�|��^1N��7GTl���{P����`:���	���������'��{R���nzS,�M����e_y��������=*����{Tl�E��b�=*z��f#�GE�Q����{T��GE�Q������������DW��y��-w��]�f�p'���ve��GE�Q��z��������>�Qq��6���"G;���������Xi1<�q�[k����J��G�G�9�Q�{T�����m����N��>��z���vn��t�{T��G���{T�i��GE�Q�{T��G�t��qy)���@?>�\VO�e�����:�2������E�T���yT���Q)e��S�|HN��R1��?%���|i|��|�#f,��������1s�x�F\)�������;����|0�iY�N����q8����[�d]���0�����iw���2��s�h�v��F����$s���n8���%����\�M��.����J7�M'�vL����.2��L�"����m��w��d��������.2�Fx��Ef7��w��]dz��Ef��]d>����G���8��.2[�h�'����Nt���D{��E����V�y��3`?|�E�������L�v�����[�"��bxj��]d��B�"���y���s�E�w��]d�9��F?5��/�}B]��]/����y��w��]dz�;h�w����y��E�w��]dz�����r;��~|v����H?<�	Wu���"�+����St��^%v����E���[�&Q��j����Ll��y���`�
VIr�V�]S.��v�y-Jz�4��s0�����L����l�?��\]�w��ux�}]z_�����u�}]z_�����u�}]z_�����uY~My_��h��uy�}]z_��n��u�d\)v�E'�����l���������;�-w�+�}]z_����C�����y��y_��k��&��.9�y���Vo���J������u�Z���V��}]>��y_�����uY�����tvj�p�	u�w�p�s��������u�}]��u���N�}]z_�����u�}]�+G?@��\W|9�����z*���<&\�g�~x����,��2<����04����������f��|=���m�r���w���q�{�l��Z�?���~lz!�@n-q��O3z7r�G��;�G8���{G��A�I9�#��f������
j��\�O�G���������Hec�T��R��hD��Tou���U�1p�f����E�w��i�&��8�N�����rt�`9�y.gJ�Q����3�h2�����
��(Xp_|�o6G�������+����
��M�dM�����`Qj^L�L���1QY�.��ap�1,x�������
���qi��s�8h��x�d"���������e��q���J8��3���������_�G�`Cv�������p�n$��(\����S?��Y�#wy�f
�(��E2\d��p#
g����F�p
��X�����p���otG�n�m)��`<E[�*"U�;T�	�WQ�wq��k�"=W��k����d��<J7�K�H���c�Cj{����ud;�q�O�5O*���K*���D��b���M=���l���F�QA�[��`#Tp7���oAm[�B������n�{��d@����s���E���M��m�����77/���$�.��{�Q�����N�A��)_�__�*H~�nb�_�vM9~VB+Q=�E*"+Q��Y�$|�R��tq����\�?c�.�F���SN��@R���
SL���&��g7[7��dD���J�}�����g����K-�oOg�X��Bh�ur��<taPz���K����y���('���s���}a�p�N�d[�� �65xa4�,��o[6Fitj-g�R�)/'�pn��1o�����F�dg��x�������?�&�U7����T���RsQ0��!	k_�)Z���;|�7��0h����z/����:���XG�c]�i�.5!��VY���jB�Q[���\aA�!JVfD��r4�g���^+K����5�q���8T�te�7����������F��a����iI�����)lOW�I�T���=��	�F�JRV9��O��AEt�H�fggj���,A��� ���}u��W��h|I*��	|��OJ��ptW�f�t��w���_)�[�e�%���$��v���c���Ak�L��MZ��h�������[�x�q�����	jGm[���]j�y�.����,��p�XH�|!����)�qfI5�b���;��?57����y���?7����K��9����'�=���o.KicZ���Lm��[(��/o��BNiYg���mfi���li`��L�`�=�91�x+�[,�2���F���o�����cc�%�>Y�r�	f�a�*��$h��48Y]�j4m���,;N>�����8'��
KY�Y�]��`)�e�'�������
����B�k?�����Z�J�+Vk.W�5�r
�v��p�ZF
���KX:��C����nhA�&��DQZ�A��������	���[@Ji-���eC�	�p��`\�
@�Z�3i	G�2�V���5K��
`�{�� t�.���r�������-���N;5%2R����(����r6c�E��p;�p��I���7M��PU<��9��
y��U�Q��$7"5������J\���3�b��U���=f��pp6����J�;o�3��,��9p^
��1O{FxO�����q ����^x2;9;��z��?�����?]^�t�P�v~��_c�|���g��7��qH�7D��3�t|��.NH	jAN�c��&C���p6��p���tN��k8���W0��<��x�+No����W��@��~m|}�ga���(�I�����l6�A����,N�����,�i�
>$�X�2����I�=\���s6F��1Z�����\��MV�ds�Ak���d���K�{tT�)�[r=������0Tt�i4��h�NF�����$����������F_O��c����"�?a��?�� �����x����7�E�	>%�/X�@����~���F����&z�l?'	:�?��M�d���Mh�[<�.�i|�H��Z?������v�b�w�-�!�����$���Q���5��n�u	�_]S����X�*����p'qI\�����[�P~lp�X���q�	�F�A��P[V��>^�?H��h�[�=�a����i��a����DjY�5�0����h���=�q{�FC�rJp������
I�Cc����&h����i��U���a��
�1>'�S�Cpc���a�]�/���pn��GDJ�Q�RT�'=2�wI�a����q+�C�x�I^?lpc�H��7�9�/��8����|�@��J�v6����z�Mx����I��I��r~����~\/nH��-��bcYx�Q���Dd�ko<<�d��e����cj0�������E'������.Y�,2j�Uh�o7�+��0��	�����i����3�)�q�p$��%���M��^����j�M�����n����/G�2G"�R��hO�y��LnJD���u���o{��}��hY��}	���h4 4������p�RR�gZ]V�	���&T-�7���K��+1+M��N��e��%4b�"��f�t"gPz��ld����O��7i�}�"���>�K�MIg��}�E>�)���y)���K��[�����Mr���7�,����@���d1���1A����G���z�
5W:>����q����s��-j�p�YB�!��>`p���@v�����a�����@����.g+�Ai�H���ASq��+Fx����$��E�,���������41�A�
��s��!&X��<x��o�7�'����o���<;�#��u��{xm|w�xq,?[)�l�>C��D�M�B{���������@Z�|��\+z��|�����!B"���$�AG�I���!C'�����}H�"M�5��k��Hc���}��ah�������	{��A�_l�����|�@�>����&S�x�zW��5aWPI(�X���GG�i��r�H�
]��$������7������x��A����6$��c�4���k<�k���~��]@U�3�:��m%z��#�.��~$��{�,��o_���w2j����{P���h��!�]?,�)�m����.Z����[�Y���Nnbtt�w�H���������7��bZ��x�@P[Y5��3�������}8Y�M4^JF�l`������n0-�
��I-d�nb�x��-?G8]zG9xr,q�Bl4��4p �s�D�0#��R!����i	�
�?���9T)lN�Rn}��C=�8Aed%!" ���c��8����:��0b6��GF��%����ME"���;=����;��t=_t����k��9>>��yM�������rd� u6������&�t�j*X���(�#0�Z�/����`R�8!�6��l*|���bn�2��@��h����.<����S�f�c�'� C'/��TF��*��x��e�[�HP?��Rw�I�[t���W�A��b�B�c�	W�IW$�7�A�FG�����VZk?C* �� F;i�����@i�����5��6-���y�#���f�Lj��`(sG�e�� cY."���-K<
�	w�Q 8r�|Z��^��{����r�
%Ux-R��h� ~�:�G���E$T����D��T`*xC���Xb
uR@fE��dz����J��l/��)Hrn���'����S/xR�0!@���b��l�����f��u������8>��A+P������1����=��4�.����Nd���K���U���wI=�0���������R^�^n05O(��P�Hm��$�!��������	�.	���� �������&�����6^�?~-f~��� ��M�E�Z��~����N6�������MQ���2����������e�Bm�����\xT�j}�p��(��
9���Y�3�px\\�uv�q`a�):��u~�W:D���*M�I��_��x<Ak�����^������&���I�*�@��:�-����xvrr��O��� ��1���Zzh�j��
���h��
�L&JI��q\�m����$�1�.��������7��t��`�O��e����m�d�� Z���A���A�}�}���xx���z|�HW��T"I��[*U�L��/����^��?1`��E]!\�
������XQ�����t�{��	~���!��Bd���j�:�nq|5_/����V���(.�wN�`�kIC�L��)�2a����{�h���A��F_}��W^�G�WN#��T@�&gh��L��ii\�z�>.�g2>�K6F�b���j!���H"�0��H��\4�g�)�Be��'Z�&���5�K���9XvID�d��"�u���,�~H�U��jF���SL����kIM���U��=��E�����]c|���%t���L�
G|v�z�p#�"g�_�C?J}����]"��6_���<~�O)OLD g����,����P�F�p/�w���	�7M``1��q�d6��H�R�r��E��C�����NQ�c�$��h(L�~W`���L�|��p���-g&_��K���UH������A�HV�aR�_����~��W�fn>x��T�����������Si��p2��cL���V?���"l�YM&��	�;N�74�l�@R�	�r����A�uzXz,acx�^��W������9bx	��4eK��b>SU����W��?X��OTn|�u/����~y�V��|���w���S4���"I)�������h2��uM�� ����o���+8���#��g(����_�������7}���n���|����wy���?X���z/�������U�Scz|�@����d�qX�����o���F��|��
n�L�,%��K��$k,l�y�*a�,\�!�>	��rZ��e���������'T�h*�������y�N���Gn0�b�/�%+��I{M	+�,6Y��.���e�o����T�*DK)� �my}��6��q��l���|�{��L������:��g>?PUJY|,�/�m���p��0g�����i��#�J�p�����;�v���B��&:�=�T'1y�����b���E\��u� 	U2��O�D=C�}>8�����M�u�z�� �BZ9.�w��f�����������E?���5�����k[&l�AZ�V�������������d���T^���������g��o_�n=�'G�f��M`��� r]y���>#w�1�������~��Kl(�'�2��F�q��F�H���#{����C��;�\��#�>����2^�\m��W:�A"�E�Hm��ny����I�i_,#d�c�[>��U�����X����+jS�a���>�G���m������(��p.��K��S,c��#���A_EJ�`��b%�|��+���e����U
9���G��j`��UJ��BkgHj}�q$�\��(mfG�~����XtX,-������2u#_�cj��s����_22��`�����(6��++O�z���z8�#D2��7�`3"q���'��w��}���t��*2��\2��H�.����i��
� [p2�/?����>Wl����(��B�y��	j���}��i7�zD�F�U0�����_��_�I��M9i3H���=+��B
`0����p�K�p+�gd��Ud�dpy����*n�,��2��>��/H.!���M�{����S*��eiV�i!2�e���9�Vp^�Q}������e\�%�J{�4+	kXH�@�IW	�Lj���#��z�Q>](���Q�[T��5C��*���`�5��3�0u.OH"gE��D�xd�4i�h�+Y\�hL�J���r0*��%��t����7��-\��aC,������e)�������ev���An�oy��K��� d�G�@�a�xC��;���^�����=)!A�X�N�hm!����A����8{S�`����a_���~GMkdC��
5�U����S�V�`>t�}[	�`�����"�jD�!Rr��w6�X|�e�����ZK�{��*�c���`J�����|�Q0D����Dz<`I��D�4��l����j���A���Lup@u���[Q���N,�67����z��s�r�r��|#����	qrt`�#��{y5���!4'M4J�b+��������D�M[V�
W,Z�B��<I�$8_.�N��S!���hi��U�9;>��T����@Y�<YGx�e''�����gb"�R������4�v��(�Pn}X���@���#O+�S�jI���P��#�������}h�5�E34����B���8��g8W��<k~�<i�	7\+�2��sD�5h�n�v)(�M��	�p�o�~z��#.B?��,�"P� �T\��~R���
2���~"](�DU��&|`
����CU	*,����/����W�F��9����p�I�|�q9T�_l8�V-���l��^�J��T�����F\?h��F�Uh�����x�����Z�!F�D�RI�L�����-S�}��K�O�U�>#�3����M���$|[C��bFck���e�}Y^Y����H�_�z��K2�����_�G���e���0���������������0�����/|+���1;wPsc.�����"��T������:W�AYY.�������1.��bx�\�Z�����#�P��j��Vn��s�<"��S���9���3o9�;��&_P�S����ZCv0y��(U���4�x�:g���gx���do'��|���}�q/eP&�WI3U8@����kr<�������d
�v!�l��Et"$��`����"A�������5�~n�
��%���91��_�����r�e+�hcIF��g^�$�}-auh9:�ov�������m�Bv�"D���Ym�@�As"��V����j�%m��4��
�C�������a�� �|�}�������^7�Rt��l5#W�����7�v<]�e,�|��N%�R4�l�(�-V��`�Uv��W� X��3������A�.�z��	s����^���_Ah�0���v�-�LY���W)9��-g�%����*������,��K���:O���v��{!,W�n �9�k��������w����E���9��Q���*1�+�.�O�0�����B3���$�(�I�	���_�~�3#u��z��<��00'�2��=���hF-������o��8)X�)�9^,-g�!�CH���s�����:,PetGe�'�
PY
�Q'��g��h���=jW%�Z�J�/��;=Y���
����8�o�xmi��8
 ]��)���v6
��\��zmf���i��~&�CK:��L�S�./?�g���P]��UZ�wXd*�F:p���8�2i�M���-8E�:���mR5���o �b�=�����W��Q���rM.j����T�\����0�fzb;����z�A�"�)�����G������r���mY�UF'RV�Z��(3>��x���2���J�h���B&���_�.�`D��/�#�e5a�DnM�j������D��/�dTq���a�*�O��@�Z!�+����w�����,-z]�a��^�<�x
rJP�I3�B>(�y��RL�~����7)m����7h����\*m��5*}Wyy�~L���3������)^T\'���6
�O]��P�����A�G>�G���1 Y�6�r��(��-���m-�_Q���������r.R�z����������%�a�4}�K���|���me�E}���Z �s�B���?��4�x,<�0�r/�.�;d��/]7���3�b��q1O�����D�@
�ZZI��Ky�o�<[8��<[PDh�T,���K�+
B�$wj���g:e����*�����jO�h�7�R:����KV��dH�n]	��M~@��y�PT��{�BEiqv�A'H���Z�n2�=cG;������G}�)��3�M"��x�[K%��9�'#^����F��VPq�w�����U�>���*�f
e��#ZI���('��rB��a~Z�v�S|��"Sg��-�Fd���6V��'\��~��8�l(
��z�u��ri���t���b�"�D���~����7.���f�@f�o��:A���M��B�M����e��T�!�1X�#F+>{&����Q�}AZ����,\��Ao����-���[#�4���NK��"��/������]�Y��%�:>���#nS��j�'��L��R��JUb5�39��������?�����W��lc����������D#��?��+T��n$V.�qe\���
�e��Fk���CJ����Y���p�L�z�t�R��Q�lg�*;��WRA�	���7&3�/����w9��(2��0���wS�q�qs����d��2���d�u�Y|�:
:�����l:YX%���!� �3�X��*I�^q	D��Ee�U�*9ND��-�je^!g	[�k����A���&�51oS��#�'{�'�T��F�I= ��?*������M(���|t��I>��CRu[���K@��L���7}���o���T��"��PB�� 'hUy���)������)��Q�y��������V����������z�@���y�������g�,e��tYJ|�h���X�����p�3B����2j���r�c�S�|�JPC}��1N����$+����L�$4��1�����4���5gH��R�fy��O"m_�Q�*�[n�J� �,����#�,W`��{��M p��/����j� ���em�P����!hX���.hOE��
�.���y������:Z?�lg����@�����"{�V�"v��Y���r0�?�(��'z�I��b�CG���(���G��/rF��Lg_��c#��(�v*� 
���t�eMu+r��5�sq�w��Q�k�#��2����=�083N���"�i�G�-U��I�T�?h=���A6��Z+�����d�)���g&_�����$�?g����s ��3������L������*�b�������l���B�:[/6�XM16�.�Y�:F��U;"e�����i���r�r�Wu�!�?��4J���pP��R��_Av��Hl��r%&yXUa�h8��=���	h�e*�������_R����&
�b��U�(r]��^iU�@���*�
x$�e�_7�����&Bb��l�OTY�A���\����n��0(k]E0���>f��r����/+�fK�}>�?v�)��F��&I���]r�F�DD���<���J�l�����<*�!�d�xO>�}+%��i@�A�<�W�a�U���t7��\Z�})(T��5)N��v���\���YZ!�i������t�5��V�|)]��x�~\�Y���j����k+Y��L��-k��H�#@[�i#fVIWg�um��9��3rT����/��~��g�������&I����4X�����6-�I�(ai/���B�N�.��]ZA���v*��]�{c���{�)W,a����iqr�J%�WTJ,B���h#��{^S<_S�����*��L�{
)�����k�2�V&_d5- e��U�|aU&_����CYm�LC\����6	�'�e.������0�$�d�C?q�8�
	�P��%��r��O��5����$�!C��o�1��
�23��8S��9����HI�*��P3��J�^�rRH ���r�{	kYkM���4�\�������)�'Y/��pX@����?cs���!��2����T�������8^����/M�*wg7l��r7s%���Q�v7����b���c),06?%|����&�=�R
oJlS��L���X������Z����yC���~�+f�kM��O	b)���q9$���OeM��s�]2�T�)���p6��BW+U��������l_c��������;�"��(��5���d��G�5
��A�gXQh_��g��b�pW��T��x	�	u3@n���0����)�|��,��J��_lU�SScd%�M+���GN����H���vR���_:�%�{��U1%������V��E��*��/�=3�:�D�`C�!
��j�%������V�4.nr��I�L>eN/��������$��*k)J��o�r�E�'����ic������f�`}x��>��(�3U������ �I��$��A�:�z�Zwx�
�`a.cy�
'�����t>e� 	Q���uRX�R����	�[)���5[�����[��9D|��]�!'�d�jv�W��^Z�e� A�w\����C��<h��":']6��E���E7�-����z��g^=-_=�4����$}q\�^����v�3�#~;0������d�8e�q��o��la�8M9�hC����x�2)V�q�M� �3��2���JVV�v)��C^�+yDt��=�d������������aCK(����k��e�k�r�5�M�������i��4�T��D��
�hb�{������>��W!jE��oPrq'�!t�c�5u^�4������G����-�'�D�O��^�q�������P'�T���e��oyT4)M>���ry����d[���y>�U������p��^rG"[�a�0@r�I/<RZ���X��>l��+M�;.���7��/�C��G.���qy�t����OZW��}�I:4����1|�~�n��������b�������[e!�����k-7d���^[9-���"�����lc��v#%X�T��C��W������ ���9��Tg+'��dI)N�2����q�R�n��$��%RMt�l�?��H�����(����>~	�r�;A��@����\p�N%b*�t�����"��d����?$�m�����F���juR��'�C���FX;�l����G=�������/���z�3�q����3�-^6�BH����@���"b����{���8���������b����dv��|w�e�qe1��7�|p���I�R.�Uc�eN-��f	�77		�I5������_Lo��95��P��lTM�_���9K�F���."�L2�����d�(�\��9nAUE�r�����{#��6��r�����UJ�0�#j)�E���>^w�{W Qs_I���OO�#lC	�4E�',M8�$�K^��Ie��Ji���s1������CH��,��%��DZ����M��S��gXS[�x)C�G���B�-��HT�z��u��I����J�.�t��`\#�7��/��"+�*i�i�-�� c�-ro�I�����D,AAg��
����`�b�l���?�q��c�/�T�w��I.�d��7�~�k���1��^�Z��3���;��@�x�t���{���H�/i;1��W������;�|�&$�7��]���D��X�$����TZ.��z���	w�H�tPd��K���
��%Q�M���d{���AEs��B�9���4��v[���U�]�5wC�H���.�~�i��/@!�2_�������[�����>�)�����K�7��YRq���T
���)\�'BFpL�2�������[t�/*S#n-�%�j����Ef{����f
7�[�:n&!h��������(�@}>@�W����������*5���Tf�,��!$����o0V���6R��9���<��IbSE+�#�P�CHf����#�$?Y�W'Ol��������y&��'=U�_�0K��36�z�,��R�Iv����2x�FD��V�����*����l~�d����E�s�����y"Yx���L1�hN�P�c��rsMc����}d�3@��B������*��V����s���� �4���Z��hs|��:��CT]� L��� [ �s�o���G1{H����4:��+�g.%��4eH��~FH��3�X/6���	s%���J���<�p�5R��g��J��[�{�����:�]��L� [-l��U@�u���k��"#/�2L������li�([ �D4b	1btE�4b��8B���C"!Pr�����F�Tz�������gy��~j�N
�T"���V�DEjl����7�C�76�eXc#�63��3��~P8��y�lj��+�0����\?��}��P<	��
m�T��k�'"��J�ohI��{\R&cr��a���������5&@#��A������Qo����{AK~�.�������k}�7�����5	~^����
�����"�.7=� �!�0�Nx�������c&�O�p�5�3����M�w�U��3�]����7Jo��4��*f3 �GZt4�R_�B
S
IP����D�D/U���$���s��^��-������h�m�1������V(�MP\u����f�sz.I����h���nA�`3:�\��?g��,7 �E���XDo�
J|���g����rq�XQ-1l���k@�d��l�I�9zL�&F�����H�`(z�pY��V"�	�Zi��X�YY!�
ds_�`%P��{�(��+tK���D�u�#Y	���nM���
�tHw��Xv�������0�blHI9����o�J���e�k3m���}�h�H��'��{��%B(rdjD�V����LuHI5�wUR#���*��{����j�K��E�r�����h$s��V�	���[b���=PB�C5W�"����k�ijIl��]@\^�^5!���1�P���+�I�VB����h�������G��4��\k-��NV�
=��A�[!��L�>��7��o���8��r���RPY�%����E>�*�D��r��j�����"��9SrxkA|S@xT��,�QgrWp�%�	��h�|�d\��g��X9��`���k�b���B���Y"��h�B8)��N���~�4����YH�|Cw�c�AU���j9`�!\���%}6��UTB1QQ��LMO���J0�o�(�@��Plf�x�����l��*)��E�{G�K�u���"K)t
��-�A�e�3�3T ��^�'��d�n�&�Ra5���
 ��c���#�X�%���Y,��������)�g�Q��z�a ��me�g�����c�]�\���V�S�����D��v�������9�l�N8��
���&�\^����o/2;U3����QN�N�����
`����?`����A�
�{-|�S�8���#�-�^J�mf�'f���n�%;��f�����d-��,i��tq>�0H�e��V�v���R����t	C��,&��$����oIW5c������~�I"J��?p�2�"�z)�AIa��,��R� ������-��b�mf�?{�<!	+H�+{�@�~i��	�������bd&4�@d�"L��V�/f���A�$��;vQ���J�6\[���/���{|��k�]�)lR�"�R��-q|#�L(�g���������l��))x�Z���]�#s��Mq�1�T����q��Z�6��J�0�Y+`1k;Lj#�t3�*�_P���P~����E` )�j8@bYE :���L�7�(���BCNv���B��!����b���J��$rT|�p��f hs��.�L�k���H�:`�Q�W��s����aJ���8�I��(+5\sh.�x"S<����>�����p���<r"�L-`��C���<b�*L*����yo���������6��j�Vt����(��[-X���2�@GB�?T���1P�4�K`�C���b�@(+#�S�%�E�Wu��;��3r�A@�j�@���'b}k�,��/����h�X/p�<#x}%������V����Ap|�3�_����W���du���~����N6_������JAc�� ��
�G�t3�7�W''q|}u���p8�{����5�^�x�\���+����� �����%����
�6�%�������6�5��B]�$��H�������5b|�O��x����oC��^��X��,/UT`��$L)��;���w��C�J�P�����������)�����I����@\�3]M#)2�B3	A��h���o�1^�R�!M�my�\����u�C�&���q�@��f~�������/I	�T������Z��N6-�j"�&I*���%B����F�;�,����B���/��z�gSH,�9~D��������o�n�H�����$'k�(L��%���2e+�%_INv���g%dA���u�������~O��Hc�f�k�Q�]U]����h�LV��\D�n4GjJWDV�lZp6'�=:/�&�2,�+Y/�;���cf�d��,�i�2�5�hP(��8�9���!�/�jSM�����m����Q��W���Ad	d��y�k�Xm��������io
)A?�N�?��~~����Us��i����%'���.9��;�e�aT����M�����Uyt3G��d�dr!k]��H(��ET?Y��K�X�u�A��t�1�����PVf��H��2���g���F	�g�f�L��#
gT��_�$���x:��g����S���^�Ru�QQ#�����y�e�]A�w �2��pD���^Ac,�4�9���Y��+3�%�Q��C�+8�0i�|��D���Yl(�~�HJ@�J��~xQ\��~|�/���}�������=����y���~U��r�Gfv�2F�+k)������^L�����bKGmE���'3�\���Li%�B-�U(;���X���eI����S�r	�J�7Xx�|��N�"v7������F��l��ai9��s�S��N{��H�A��������
qZ���c��_[����H$��V}i��; ���OO%Cd�+N�zY2n��d�n<�V�p�O"KR�t`v,�|�����_~H����1-6�Bby�?3��|�t�(���?��A������
�#���A���������*�6/�y�b��,s�,Qk`��G����h� �RnD�:vZ"}"I�jn�CL��?�718 I������Q�]���O����c.Gj�#j�
��-"N��������`E����G��L�k��_;�a���F���/J�����������6�������9`��CK����Y�
�s8���\L�C"�d�a7��G1rB
��#�`Sc��%Ub��tO[��L���UU������I��A���W�?S���cfTS!�u�����0V�O$�/��*��.��,��|��,�.�usE�#��������eig{R{�K�D@����z@Woy�z
���m��w<`�E���e��<F?.?���U��7X����Ms�Z��rY����B�n�tU�
�Z*���$�`.\k)H�������J�-/����v9���h�n���[��.���+�*�Hip���|}�"�����KwG��8�\�T8�X&e�u���\�����7T����(}F�����
d�/�4"�4}��je�t���FlU���+����-���.rC��M:A��C�1���"]-�����NF��q1�
���=���F����d�,���g��j����������� @��/����G��.9a��+>�2�^qn�Z�\w@��6�C\#�!ZUe��F<��v�
���^YB�������"�D9�t�aJ7�
IC�7_Er�8����AG*O��]-��:<-�q<���8������x�O���T��UQ���d�<e��s2�I���(�b�`�7�i�f��y�b��W������q^0abU�%q].�dG���i�p�j�����:�K�y�����v���9������8�����/�5	*k�J�9�����'��?�q���������/�`P�f&�9��h:��p��L�����}~LeP���4U�#� d!p��t�����wDB,o$�����ZW-����� ����c�Os��RJ�����C�����!���=zJx� uK����o�(5��J��������R�`�}��T����#z�
��zKh�gd/"�
���I��J�%�H��10BZ�B��e���-�!��	j@6u�jf�.����D���s�n�"�bn���CC��<i�*6��,��J�.�����u�r��x��@R�50���qk%Dh�3�v����T����x�j'+p��c�O���(�m]�Uh
h���v��V�B1`4�p��7s�B�"�	�-6F�I��2���=��F����O������<�[P�v����<����f�
&URG�,0g8�5#!�,�xcV����^�b�&�Z7�J���!�'�����r���q��S����"��������4+�S������H_)X�T�R�	�]�F�������M��3����%�'������/�3)���G���Y(zJ�f�L���� �$���Q~J�����l�>�A8x��m����%=I._O� ����T��A��*�pH�)J8VJ5{�0eD�46�����cy��e������s�i	l�z���W('���[R�<f���mn��s���&�����&�Y)��lg�x�
b5�"��9Ou��B����g���S�4�
�dt|<.��)�P��$1�e6��y@��#�O�����	���#��?��������������<�TO.�-��bN@�_`i����Q�/��Gd_�����2hl:`��q�H{y��
�����a����Q��P��sS`������P TH����fa���;�m���R��#�Ph�RZ�G1���k�&�8z�(9O�GG���~{���?;��{;(� ���~���~Z�����Ho��$�����,���Y���X��e}L!�� ���a��h��?�k6%HR�%�0K�?��d����[�,��!K�
�?��w7�����\��{z=o�>��l0��gx6��}"���|4����E2�Y?=G?by]\#"�������)<0��	#�]N�}#��x
�����5���|zL��OG��GD����!���F@'��d��������f���l�����7O~D������&�����%�NO_n"���>W��D�:zZ��V��9�]^��eL`��0$
��#R��t���O�8���OH�|	G��k�9��W�m�a������F�<�dC�bK�����e���Q�f��a�WP����(]������T,8����JC����Od�����z�u�"�,
~�Dx�tHVx�P���
	"���������U[{�u%��&�qF�h���`����`�
U[)��c�������4�l��2�[qFD�3#���
��-OJ�#8X�7�<���!�����$�b�&�U��h������+��?-���BpD�s�F�J�#\$�1�������
��%e���F>����U����N}�brK�xs���h�(�������#��scb����;���9��� e�+���m/��g�=*P�J�awL�+I��	���%	S,&�_�l��|�Z�b����6���wt���D�T\^o>RC��:���8��sC����k������C��"J��V��)��IG���K
X���]E��C����z�V.�JX�6M)n6Sm����^����j����'��b��)�?y������q����g/����'o��?}�����F#zf����gK\ ��X��x����Qt��������oQK� "Ju�<��oD���9!��b����@F3���N{`Jq�y������8�?c#A5������A�&��&�q�������,�K��Z��:�"���+����;�$L��-�C��6����;������-�`�!���iRQ8|gt�����P(=����������B��NA���4����I"k�+�c������o�����>'<0z�Z|���dsP|������m89��������OU(p�!Xq��=|���-E�r@�����TY��7��2��(=�#���IB��H�x�2�$�)/[�d��[�<-��Kz<AD����A�*��2��-"j���x�!�)�G3F��������v����)[
�A�%z%o�d(8�j��s}	'x2�,���f�xV�L/D��G��
����L��+l�q"%6��7�I�\Sy���G�@�c-�TzX�h��W�����
u��z��:����5��)f�P��{@[%�l}���sf4='����{�XZ/��@�#���%��D����N"~sI_o�hO�r��`u��i��E��Q�G
Ri(b�S~������Pl����B�	����C���&����
�h��h�N�k�7(�2�+��X��O0y��z%^�(#��w<"�����d1��G����LZ
�
Ej��������a������jXq�	FKg��Zg���|�)���j�a��.��L������%�;!��+���R"�>�N���#)�o
MZ'���a���?y��Og?���C��2v������>�3hy&�E�21��j23�tm�"%������4�����l�Z,�P�n�]=D��2	N�_��et������H����Wg��f	�d��H�D����������>��ni�+��w��G2��6��kG[)c�������}�������G8�^�����zN���E�'�m�~����
�����?<e����m\��o<=����,�6sp>�G�"�x6JG��b�L��0��}���jo@������ y�%�����(nR������#��3y��X�����o7�d�_�o�������X�V�a�L6W��X<�LA�yu,C���>�t<�8N�zvx��/tP���'��!Y�:�"��\k���?����']���Fg`iz1I����$M����`���Xp�*���?@�1�gc�O����4��,��O���AR�^���`�B�fP�=yuz���9���)�W�V�\M����=(_�+^}�D��V�7��u_*��)���+D�������?��OTeM�4��%j�������$T����4�����.884��P8�g/^��z��� �h����O��A`��f�L�r�����O�}|�f�����4����'+�d�&5����W�wp�48i�Pt|L��i�{d8�hT��g/�?�3�s�9��MP,C�r�\.q��r9�.���R��r�\.c�N^�<���a'������;�|��#�c2��@�0>\�&�6�Y��Y����(+7q�Nm��m��Mt��n%l#Z?�b�Y�h�f�c��F�:���8��%��A��F���D[J�&�n�j��.��9�r���LJ��%�rX.���R�\���R.��e�D��Qn)pQ��~$KB��H@#oA�]��s��`h1�X�26�N�p3��#��'��p|����(.6]���K�\�,]�)�lX��5����G���???y����|��5A���#���8����������M<]�7P���&�@wR����{
s�^����,&()@(Y������� ����G����c��6��N�K�-@��r�9`���
z��'�X<����p�G"5P{�#�I������#����-�6���.��^��ZE?d�a��CQ���q�)�A�|�&q���B*�������P$k7�����Y�����q����F��iC��1aUna$&�/�#����b#f�w2�)mh�HbV%f${-��\:m�M������|����hp�h��^�gTF}�\�h�V�^C�z��'��u
s���B��0�@�L���@Eka�1��w�vs`[)��nu���Bd�����������Pr�����Q�b��<I���~�')-�g��$c��G�� yV�x�Z(���y��Rk4��{�%=�.�'���;h�!�P�<��DJ��z��[���/x(��<��C^��	G�����~<UhB�.`�x������G��UqQ��	��+�P0���D1"��Ub\,7�X�&<Y9��+�9C;��xk�\��W{E�W�=�`��Gt�{��}eW0KWd�|����t'f8��Y=_�Z��]nfEmx���0�����A_S�v���{D_�6�]{?��l����Uq��j�����U`.�ea1��bVo�M�6!+cV�%Kho���L(�V���`�f����S���YT�=
P$�]e��iM-L�~Y	�^��O�>u~A��I�V�K����jW?����%.yw�>��@y~�J��Oo�Ox�Yy\���<�wRV&��F�����iU>c��
f��de��l zA���V~ `����x�-G��7���w{eG��>����%�\5����J�Z�T�W*���N�6�u*E
y2�{�t"}B;���:���{��"}Z;���:������E��v��u������Pj�5���:C������G��Hj�5���L��bI���h�M�i9�Rv�*��W�~��v)�+]�*������)#���}�<�|�;D�S&���t�����]�6�{/?����Wd�>�;�Z(�����}��n��:N�7&6�vnv��gGc��n��6l	�\�0Wa��9�����"�
��Hm���3��Y,B{��-�����A�v6|G�)��������m�.��g4��`L���I���f��3�:��AY�+��<\�	6�=:��S���0;�Si�������	4Jsh>��C��J2��{S�~��g������Wk<P�(��p�alp��!c|��}��J�f�
_�*4�
��CO���
��R�sV�����)��H�*����n���01���%p�/����\l[-����&�!�=�A�a�#��I�~��J�k���^��DS��;���ib���i@7#,�B4��@+�U��}|�_���#��4�g��X���d���"P�������K*�����-H0�x-Q��-ZU\U�,W�3e���R����Df�<�7�w�2k*�&�j���F���n2�$��9�QeV�vc|�n��VT�y���=�}��s�L���U�4X|p"jDE��G8/�����eQ��&Z��I���0\[�e�p�����
	E�4��}�I1�f�����<~{�I����i����Eg�����J����M����7>���d�OtL��n�d�D-�H��]��b�,fe�Z�-���|���X�L����d�O>�R�F�9vP�K~#����}��*J�y�D-�z�|_-�IU��&B�/M�z_j���t���	Q����*�J�����E����M�����d�#�e�{�����-:^,���� ���1�=;o�}]��g����?��'
ZF�������7��F>�D���N����Nn�����D<&�����T� 
�����J���t���T��S������_��[���+�y����H�FJL�e��g�������WO+�g�~_�X�����]\�{A��_�Gn������5�ZN*o�R���8x<�y<������{"{�MzB��?����L��"z�klz��y���e�wm��5���mW���[M�������7��-���Nk
���v��?���<_��_���F��J�3����'4����U��t���V��k��R��&�K�N+���[
i�2���Q�oB��xh?�`�fE���M��^&g'S��c�9H�B��P%{dU	�o���
dA�4��Y���TL�hJ�.KL�����FbTGU���s�����<ae���<��|J�4U��S��jE����G�C9>U���"��!3��+�w�r����p��� ���&���+�
H������XD�rP��,�>�eP��9dP����`%S1:i���������;u�$����b���/���b6���O�enyC���T�����ru��
�;��+�����E�D�(�"z���j�A��3L���3�&]<B�A:���tl������v���������;m���/��W>#�
P�@���2���9�����.>�R"~�v�_��]Dm�}�j&v��j�j�j����'�zF���I�������[	��)j%��ji-7��*�J�%e��/�:������;d���)y�������]�6f�1	6����Zi����h���`@�Y<J�mf�T�c<����{�Z�1
��N2��S`;���D��R^d�W8G���r@���y�$�����#�[���������2O
�l��M%^"��:(�F	����[��v���C��b6� �v��2���yg�����6����t��8��-:����Ko����V2�����JS��O��S��T/>�@=�W�����xp��/BC�iJ��������W'���6��l�:C<��2If��-f>6���������8	^�PC�������|�e�msr��aF�RC�����I
r�";S��m��o;����a��g�G��C��y��+\T��
�W^�wI��l��#o':���N1�d|�\ml�����3�[`��/
�WvGwI>������D����e�f����t;�m	A��f�Rm?�M;��lC�
��K[����f0���[+3���q�S>�:yM>��>�?R�cQ��
�v�Pw�j�A,��I��6WyC�6Z���*W��������h������)��
�X�t��K$���������<��+��rG!����=�u�6W��� ag�T���]�N�:V�vHY)s�I5�)M�������4pBT��6Y�*%�:VX��l�"�3����5�tn��r�qTuH�����a{�/q����C�YN�pT��8�T$w����]NEuT�7���A:�;�i�,��<*O�e�� ���4�w�S��g�2���F�'<���T�8�h�����
+xC�Qo�k�jG���r��$����s�y��1�r�����& */Ys��������mCv$���OY��|���AO^�<?}����\�����'���"�����p�l=���+���>�u��j1���G�����fe��Z�d}�evh��V�WcP������G%�t���I����lS���y�U7
[�v��W��tj<���v,5�
W�8�9��a���^�v/���OW������b�=.�V��+jlM�{�UF4�y&z��O��y���u���J���H�$P8��,��	d���o%����>��`{f�|Q�gE�>�� ��wGu���Y�����_�4#�Z]���Ng����~o��;PK��a�����8*�R)qJ7��p{�V�K���R,���R.���s�F!��B
���K�2�T���^j���2���>�K�t8\�Q}��L���'\���GT�v�*�k���0�f�o�a��
!�:h�~������-�7���=-��{��Spfp+8&p
��n�T�
��2��[ai��f�����%�FZ�n�(�u�L�,i�/�x�6��y��@�4������'�1�f$�fhv�Oq�g�vvn�)�Y�F��^��J�Z�R������R�q����>��a[�]���4������������	�
��������B7!��J�Q�J�����0�1E�m�����T��K�U��!��1@vj5C�p���;����a7��3����,<�Qh�v�����
=���[��+�E���.b���&�o��A�]�.s���uyh	G��������~v�9� m^�Ok��������3j$X���0����jP:��8|$��lU��;����J1�?3gL���1�&��!4�,�
*?���x{��=oO�'������[�*�Z�_;��8*�B��x{&���o�����)��$�p7��x��r�a^�����$H�#O 4C3����a���
8�7���SL!�7�^!���'���A���_��|�U=�/[��[u���r�]�G9�.{��������J�`bh:^�>�Y�J��e�n�^m�v����Y����"�����5��[b�.|D��f��M�TMt��>���S�y�/���.}�AKma�{_�:Z�^�ow(���W90�=��b�U/r��jC`��S
m`915	&C�!�bXZi��~�����(
>.��8;�-N����OR�k��������x���.��T�����L�^��E
��Yt5Dt\!:�����b	s8���������:G���R�|��MMu�E@,���S��<�mjF��ZToro�}U���@|br�����|E��oP�*�������
�	�E$�`�3�JTM)oB�+��)|l�S�C������D$�#F���cB�L��`���e�A�0�"4
S3��/MaO+%C��Y����������5�m��DUG�I�������-UD�JI�H!
um�n�"
NDAS�aM����?/�>�4��I�����Y��>t���)���R3�)���CC�b��.O�����p�
���2�,<�[�m2m9	�eZ�][S����V���e�2�)�r�o(��<��2:�W�B���m�=(-s�~c�����Z6�����}��^��PS����4R��h�eW���y�����|�#yDHe�����| ��t�
�W���T��s@��n����p�H�9sv'L��dG��Q�����k�mj�j5E��J��I���:\C��N���W= (���-q�v�u�d��
S������)c�g9�Q�H�9I��6������]�cib��"��>q��0���;�i.���v;X���$�������]n
����;&�~Nm�%�����J"KRS�WC�S����GV��e�
���/K�T�cT���7=r��G��Dk�&B=��
2 zQ������S��IZKt�j�R�d������1
?K�W��3���M��=��|�Z3��3��^X�`���LG�C����9
W���43���GUa3��^x,�Q�J�h�\c�&h	�bd����W1C�����c�W)K3���sk�_=���&[m�6�R�h`w�f��/W1f��A������1���0;;������Q�U	X2�X���kiB�V�����_�R=0�2��s��Bc�F[�X
}m��d!��^x{�Yg!�(�M�V��&sX�M��P�?��S�X���R��c�X�����C�FA��t�
�v�������/�k{|5u�M�V��VW�'��0�a����������:�_�}fo�2�70����;X��dfct���kn�5���{���Y!�W��B�g�r's������A�{P��z������=(yJ^����z���5K�}���]�,�tQHA��h~��7�����hE�z����e���<Z���]�.��%�,�@�@3��Q���vq
:����m:��w��]����ur���"�%�#-�5zy�\^�U�C�9.��&���rM�)�0-m���#Z��r�U��� �U}B{s���0w�/�G�qw����*��n�F_��/awG~��u��o6����a�W�����������uU?�<q��+G���<Xx����&[?`�	KVV�e����M���Ey]����8���t+Q�aR�-!r�yZ�2P�y(���
�l
����tu#._HS�p���������R���F�d�Y�������
+XX�@GzlEz�Dz��lW�#��aH�A���[dH�M��*��^����x�
��2d����Fud����9�����0bru�l]#�����K5�
:��9U�M�KZ��[Y�-���L�_�iot�Oy��)�E��@����l���X�/S-t�����Bj�Y�4��`�d�e�LHJ�bqCw��V�O���k���'��f_%}$�`�/���F �x��+��kRj�46t�1���IQ�b�����g����\�
\��^N��]o�M���X-o�EW��RQ��W�4�\��������7�Y~*W(�����E�]���Z�Q���t���f��:A�s�����j����4���>��\�!b��YI����]|�;"��x�"�`XwM��r�{np#�M������E����[!h=��+�b���Z+Q[��+�m-a��G�#�A a���]��*ku.�}#��z�bDMf���sJ��z
�G!�8C2�U��`\��0�����\�����c�e�E���4�73�DOG�Z:M��]=W�4���N:�k�^�A�y�+VW��%�h+@�9�|{cD;���j
���h��Mv�b/�5
X��"�������0������;�6��/���kli�>s|���5��Q�Qi�*���P�@��aXY"N7@4�#��x��*���Y��Y/�,��`�����f2)�_i&Y`��t������Azc:z�{�p�[���{@6�M96�TN�{����*1M_���]Q/;�s,gy�M���#o���1\��7s�z��m�uq9�6}���D�����j�S�W;Z{6~������=N������9�f�J�������$�


�t�iT�&%4�2~%��q�g7^����t��6�sh���;n�g���T8�Y������������E��3�hc$Hc�
y�G�&�$�,��%������k�v�noiN�L)�;��Xmp8�Z�IbN�j��jc��@������PRiW��d���xY
��$\�>����������a�\5��:����D�e���V������+"4z`��`����m:�p��r�{���T���C��k$TM$�(��&�=���k��(����P��OhBkUO�����lw������y���^m���.����f$]���A�����#Tw���hyq�XyxEj. �@��KT�����W�0��.�}di��"���s(�X9C����LI�QJ�,O��Z��Z�l��B���l��X�]j�j ss�$V]��z���+�9hZ���)�E�(����8�8>e�[���.F;\���V����+�L���r��zN����(j�%�L�7�b��`�2�����lI�^��yI�Up$��+�i�����o�\��`]}�r�&��	�a��n�~o�+o��O=��d���a�#�A$�zUN
�Q$�:��L E�����>��&E���M.�9g�5�(Dv�����qw������6��]�4��:��`k�X��������Q.S�0���1����-FMd������(�2����g*��i�
�Pg�)���e�r�:�������tx_jNp��d�n��N�
�����o^�zh��Z�^����]������7^�<|�HT{v���q����YU����r
���I��N�������N��=�������s~���W<]F�]Zl�P������At����d
������[^��H��>C2������uZ���{U�������C����bX ����-�T
�J�[r�"6W���mR3��D&h`�m/6���M���U��/���{\�md}js���v�
��Ikl�h��5k���-W>	qX���k�n:�C�����Cz}��L���8�n,�a�G����!�	*���OP6�f��:��J�]�#;�k�n:�5AE&,a�mT�o��i�����)=Kkl�x����C��C�����i��i��i��iE��2�P���\fZa+��t4>I?��	�$�a�M&����Jq2i����70smu�.����s������0a|�59������V8�,r&:�/��x_�CT���i��dn��Ec}�<���]/�����eQ6�"_�e����e�yh=��pJ	���@B�a��Em�7"RZ�����[�)�b����>�e�����;iD��59}�l���?���)����?B�H� �G"?I�'�?�?� ~�O�:24$��-@%\m��IV�i%Y5��N->F������1:9?G��6�]���|3'��Y�a����������2��]/��9l(���B*+���WW�
����W��b��P�u���g������B,$�x������z�|����7��4�|����+�'�9}���l�m��j�m�3�3p�a@�^�>}�={An�*#����u��`��l�._������|�Wm�mA��o��u��kheM&�6��������U'/N����Xs�����o������]hfX>S����xA��#!{���oEs���u����]4�aC��cu'p��S\cy8zk���������c��[�
C{x��>�E�2��Y6��7T�N{*f�����sp�So�����y�����}*m�zj=e���L��U��W.Z
G�b@�l3c�]l���gUc���m�!��R(i���jD|7����@�M�^|7�l���|�w�@�C��p���)�T�_�F��R�Em����B6���~I[�G
s?6�����t�����V���t'��U>I
�����I������'I�P�|�4
�Z?��$��#�O�"����
�$�Du��I�S�v~��������]'����Yy]����9
]K����x~��}�Da?����a?����m*��2F��e���+��>_o����$el��f���y����y�B������	
|����n�56�Qq;�����
�<�E��2�|Qt5c��Nz���t�`��;�-�u��M���/��$����'yy��xk�}���m�v���61jNK�%�#�#k��r��4� q�`&�`���y9����ANC6���
��j����K��������6����vS�������qW�6��]�-�����w�����iW�����=2��]
��vU�������������vV�iv�G�)m���"����U�;��,�T"��S���N%��;��i(�����$���� ]�I�'0�_�t��{/]w��.���r���9�}H��^�C%L,I�x �����R�]*��J�v��>���\�}T)�����RMs��Q������J����G���+�%�2W-K*e�V���\q,)���X�����$s9�v��wd����)�T�v`p���l��?�V��<�!��P�q�0�4/�~�dwl��{����6e�~�[�;u�~�gD���+�s5����#|�
�'uf�+������x�Ir�?��;��jtz���_�?X�f�|Q�gE��P�6�H�(�~�a�}�MsX�����b�`���6��<y~����<\�=>\m�
�a����+e������`�j���W�V2�������E�k<���wJ9��=mL��k���]��{��=he���D�����X�\����������I�l�}!����I�@2���!Y�b!	nt%�[Lh#/���]��nQ��f��_/
�7
 .�!�Y}����]8J��s����T�m4���Q��}h|�o���m7�3�3�-t�!qT`�(c�yh�iS��z�[Y�����@��w_�|#Cd��'g�����+��OH������[P�����]�PFA�"5�Y��)*BO
���b"���=)"BO
����!��p�V�{ ���K�Y��:O�.���:n�&�m��Wg����]4�?������,+�nv�6c5"ni��F�--7����������F�}Va9��
�v:��,�gt�����������l��GPC:��~8e@;����S����w��n�7�l�o���]!�U{07��a��vf�vX����:������:��Ags���N+���A��v(��<R�M��Gj��#x��A�� ���O�y��"�$���8�u���b����nB��]���6]4e3�hKvU���:y��i��;_��h�!�E���d�[��>5��,< ����A,�P����-�(�4j	�B��^C���d��������&��a�f�����ELT���f�����Y'
7�(l�(+�������B
�(�d�,�}t�"Nf������4������V�o,G�r5,U�j�0\�o<�4;7paE:t+���\Y��}������o{�9�r��1+ILr	t{���g�%�L-�
0m-�%d��_��tHS���:R;yBBK��J���EP�ngw�t����a���^�t��6�Xf�f�3�3��Z�q��h���aN�aN�aN���*��Z�9��Znz��h�%^�p|_��B-�f�K�X�����U(8)����G-����_�W����=l)�-���rUj��R.����e�{�r<����-�al�K-��\+������r���F�%6�g}����u���].�Q2�wf����L����<�� �@e��^�}����}�������LLS�B�+q���_�����.|`��~�����h3`C4�gu@30�O��O�$��n*H�.at����\����Z��?��f*a�/�����(I�?L���_F�
��a[���YW�������}nb�T$_`"�(??�x�/��.����J����\-�$����tysu�6o������6g^|�6|��	�Gl�G+��EK���GKF�	�a'���	����L�������	��zy)��14C���j�_S0�X���qW����zQl��G����������u���K
���e`�YQD�d����-Lw��l98c���a9�h$${�VR���,@2�gca(��m��6�?MV�ir85L�v(��Bm��.��H�nC�5���t#��g��4X[p�w��d����3���Y���	��,<v��8`��>�>i��b��!�X�F�����(.���I�����W?$IH�y���D"#{��xR�9(K�
� ��	�1A��OFC�SC� �6�����p9X9���^��nE��@��#c+��L�w�dt+�R`�����1ICq�����R�H�UYv)y<� ��lY���oY�,y<��q������w`/K�c�#)j��cz7��w��{j���xn.L�M�����Y����r�'t�8�s���K��b}��h������x�d�?V�G����g���[���
�{���S�V��3y�<b��g��9MF����h��>f
�K��ur�o�jm��Y��M�q{�o�Ry�����s^�-H�������owh�n��qf{8���XB����]��+u��pN�_�nP�C�����dom��L���!{E��i,��7��!�cKS�����Y7|p����K|om-����ea^�([�x�11�?����$�
�x��r��lXR��n����r�����oW��|Sx��V��k6Fr7��������h;��VZ��K��]H�P�'���<_����	QB�|����
�N#v��=�����"���`4b�n �6�b��NB�?�����������Q����?
�������6��X�G�w��E-�q�Z���`�M<
�FK/���������Y��z��}�������`���.���%�9����2�[ni��{���P�����w.>�%(�r���0�pKH����������-����1]��rF�H�����p}��U�6�n���Np���w-<��y[��~�|���)Wl��S��6�������r�o�>b9���-`9�'��F=�}h
�Y�����U�^����;06��sH'��p������Y}u��%��f��%/�; "�Gla�����8��t��V�X;�1��r�-���vXv�XR��z����E��D������!�������E�C�1��%��:����+X���,��&�w��S��j��]�Mu�pnR�i0����/������_~������T�Ix,���3��/�dbx�J&��������h[�a<[=�
��>;��-
ic_W�R
������%��z�����el����`v��7��}e�����%;��:c[?�F�������@�*�1�����^��	������mj?�FH����@�)x<40��}e��K$��F�_O���H��y3(��Vs__H\��'�m�]�LO�p�������������^�:gC�����xb��u�
xl���EO���mn�'}P�G�m�G���qD�nY�v��p�~P����#���cl���Ac`l�K���}fg)}p��[@�d��d��$M��/&�r�T���8��1
�C��	���m����T�0�<O�!.
"��*������
]k{=������g�m =�W��b��P�[�����Q����'��Lx"���M��t���>��>�n������������Vt��s��c�!�O��8��9���[|�n%�J�+�X*�����!�����@��j����(R���3��[���b�>�S��M2*�L����8���S����q{�U���*4���J����x,N'p�1�i��Od�p�	N�r5��b��.���=L����v-n��kY�m�)v�\��2����v�
�������p�����>QB���b��n�>���C����b�|v_�C��C��C��C���!fH`�}�{�r�r�r���]���m1C1C���q�r��o���N��e���5�o��+ld�����=�{L���e� wz���|b�Pl�{}�jL�k�
~rP��������*����D;�s�YN�^���9�io�;&i|�b��+\�,6��E�9���e���U�����UW{�0����:�����W�6�&uS�~?�p;9���H|$�����#��v$~?m�?I�����_	6�?����>�e���Ep��H��!�/)���j���{t02�!A�������Dl�Q�?�i~��]#�c��N��.Pz������w����]���{�����������^��qp4���Q����=�����������QR#O���[�����S�z�l�^��,A�.���so>S���W�B���Ls�����90?q �V3r�Q�._������4��aJ�Rs�m��$����������d-������`���R��;R���0���M�Y9l�������U���������*����MU�T+VS��W]h5T���*�M�jf�-J0���W,H�Z�������I�M��6k����{�M�����.&���Vm���!�!k���!�!��4����������oH���z+�!�6 ��M�����uo1X�������Q�"	�c�m��s�]
����6����z3�����"����S�T���:�G�#����V��p�<�o��L������1���5����z
��fG�1k�Ak���>h�<hn���9��W�AS���!��nC�2�K2��/�4�^���w�2�{,��$X�
5�v �6���Q�����i*��7��!��Nc�����17|p�����K�y���_�������_|TI���4����J��!Vr��E�X��*��!V�!V�aW3����K�!Vr����[������]���m�������q��|�b%�������oPq�O��;����?��_�)^��%��z��y��2��l����Z�r������%j�h~���.��|�%����^�����E��"�;h�~����Vp@u��{��A�����o�7/Q�w��	�A,�P��>�E�8��aPK8�~
T�[����E��8=;�	in�	��k�����LT�vY9�)ml�(�|��}���'
�'�
5x�p��J���0Q��|���S�����|��!���������r8aV�r5,U�'����
�������������e
P�rr1%=����W���:����u�}p�G|k����7*������Zs<u�����P�r������J��nIo'^����!Yz��Zj����)+G`I��m��" 8��d��Z�Qn�^��:�{]�K�3��+������a��a�Q��ow�]���0��0��0��aN�aN-��F
s8���[��q��������������(�F��������?���6����jQ%��/����D�)���J���tA
`�
�
�{�>��K��{e9�����r�/-��op�R.����rY�����������oyc�^j)7�ZI���C�v����Z�#9�Dk��S:���M��X����O��l@��>��g����O�]���?{��53�g����0G0�)���g���W��po�?1������������1oo��!�E�/r<ED}rIS�
�2]n��h+,���������p�>��7J�^��)�`N-qZ��Zh��-�F���4��2��0-���;!�GR���BxK��6�w����zl�zW���6T����z����~4��X������o��b
D�X�����]z�l�����,
w
������`�/���v�:_X��;n�'0�y'c�>}�f�����\n���I����[�P��}�?��y5�Wa��Ky�v�}5����MJ��5�M5S��8������&�C�_�_��<�"?�z�����U_$s���`
q�*x_:�;�j1�zY�C2W��i�VM�d�������<�'�r��[�8��y��t�����[)F�Q:JXU�z�j�}t)O�,y+��;j�y�k�:�����8�V���
�#A��n���o����������Z�S��j�M>�����d���*��P��=u��1`�
�?e�����p�w�)[�?������L�^�wL��x�����z����/�7&
�F�S�m��L�`g.��8��������qv��.x��6������	��&�Gv ���v���TV���@�=�v�WH>���q�N�i��ak��vcl��	;�>v��
���18a��=�s{����	{X}�!���;u���	{T}��{��9�d��=�>�`�����&N���`O�����O��v"}�PsT����O']&I��8����48�����I6H��{���/S��7Kp?��{���en���In{J�| ��In_R�����y�y��zH�u~����|� �o�n�Ry���u�u��W���J�~�G�H���hF��������KI`����#��,��(�N��&��cS�����p���y����������N\�4:Bd��
{�I�<��wcU,r��	�W���*.�7��������`���'��������|��J.z	5��o�����Dk��a�4�@@��]Z�����ho�o6����Q�
�j��,�L3��V�%��4�N`�vg��nX�E���G��0��E���$C�KK����}@���	�����
MXa:�����p����D?b^�3U�x�j��)����V�L|o�2�*�2|�f,.��:��WE����(w��|�.��>��d�d��bm�f�}�V��0����8�G����y��
�a����B2������$C�!�K3�1��T�9�4��.�ZI����2�Fy���bZ]�)��iw��N��Q?��)�R;��&;�R$5������-������md'������*1X�>��7�{����.#P��e��4d���w �4�6������k`hal_�\��1|+IJ�M�
ymrD3|d1��m�oB�@���u��������#�mD�t�Y�Csf����PcO��C1�![aU}1+���$/u`�v��i�b�����:�������:�������s��-��MVr����W#5������$$��.����uW��g5�qD����U�������E�R���^[$�`b��k�W��-����Uj��:wy���V���w�nR�!w�T��:j�T�I^{"�L�__/����SU�����������
4c�h(�i��PWRh���[s �{����d��
S�l�nl��2����Gj�:� Zu��G���^������7V�f
A;�|o:�:���b5���4��|��c���x������H~95#|�E&���'T�S��BQ��(�b�%J=�U|(N���V+,
i����:�������|Y�e�!~�4���)(���*�A?�&���I�35_���d�����������O�fs���2N����G��i�/X����Z�z�h;<�U�l�u��?5�m��l�����C�}%�c��5�(�C�������V�K��S�<��f�MY�Wo�P4r**�@�x�M�h*�[V�2D'�R�X�{��;zh�����3�0mF��H�
����
5��V����w��t�{=��+�2�@|M��o�U�W����*�B����/�8{�}Ln�ih��9e���`���G����PM������~�Q���x3%�~P�|�FTe����5��6K�Y��
R�~$�
<b�
��$)q�����D���j!=9*Iu�L-?={A�L-���l�5��%9M��������:����c�AdMlS���e�����
K�P�/��8�>YH����c���N��y~F5m~����?S��,��E�e�V.��%�rX.���R�\�`�R.��e�����d!�AE@�_��TB[o�(���@�0Bl�����$[�*a���$�(����i=;-�b}�;��.��.6�W���P,�g�`�dk�3k����
�%wT-���J*�$0��lS�b�Z\��@
�n�j��.����}�f&�Jw9U[�.�*=�V�er�����y	�UCh3	�"�����������0�z/��E~M��bE>{�CcC8�E�))c]�TXfAb��K�4tK�.sN�j�����C�����������0[c�[TS������$�3N�	eZj���a��*�y��/S=H�c�$���m�o�m���_��9$���\n�������h��[~�B�~�tE�p�����o.��n�+�;��N���{��
�����$����������?�O�e���T}*���M���?��Q�X��5_�UA� �
?6hZ�<'k����t���]�'rQ2���Q�z}����y�#�Yd%)|�]L��ow�����Z���*0�A�#D&K,x�:�������Fzw���0��Y�,�����E������1W����yNGR�,bKA�).E@����M�PTP1�A-���!�R�V��r�+�i��X�#6)!�����v1�J��!Ic;I[���TV��B���$m�UG��N�VX!$���tj%�(P%I�VE��"�p$v�e
���l��Aw<0��%w�������-�,����#?9�N�Av���Jn�,���&.��.5�M=���s	C�THr
"&��B����zg,�$}��m;�����%���nGOK�<@����Q���"��4��6o�];=��
�O��}�b]�b]9��p/�;���T|'L��j�n�;��L�'r����~�ne�d}����2�,�^��m�|��\��
6t@T��u#����[�5�-�%e/J����\���y2nv�Q����F��Q�\��g��c�����X���"0o���)���Z�j��?��]���Kr��%�x�i����YQ�n8��1����;�4��Loq�vf�F���� �vOM������M'M������e��������f�nSN�-Ou�d',E5f�Wh�S�10�1tA�X%}�������D��,��������J�Z%	��������?��%��`�^���d���B����s���/h��nY��:g��>�i���IB��/���Cl��F��,�^�O������G_�y�-�L�t%2g���6���?7�5���B���@B����-u��$�'������>���G�����d�*����s�Zt����F���w��+R��i�]�}WR��3�O����%��Wf��y&j���b)��M#���,�cN���)�-�v�zJ��"c�>}�FOe���;.������Lf��(fx��T�aY�D.����)�B����.]�5����k�%a������/z��U`��/���hP�s�����3P\��@��H�O����$e:�����(iQB*Z��*��������b#����h��+&Rng%���]���>hAZ����9hA������.�&��6�������]��}���q�uO���ks��������}T��0����X^�{R���V�{�V7�jP:������������,UCt�&�)M�KpN�}Jv�U�C<�|]�V@�����i:�F�\A�Y\U����}���u>x����I2�[���n�t|�����}�Ww�{�[J�{�����~�N4�.3uw�����3
��,��i����.��NJ���{C���i���xu�QT?;����||���\u�w�q������G�qw����]����|?8�{�1���.�F��+����g��[�u�_-iv=�*9��@��Z�'+|qy��x���D��~|D�z���'����?�/��~{���?;��IM����Lg{�{�W��PN�W�lc���Uq��6�����1��|p#�a��h��?�+N��$�Y�
�d0�C���ocn�	�?���l�kg/�����),rir1����0�'S2
��`\\\d��8��/�����1������k��P�����0��#���1B��#�7b�����_�`Q����P�O�������	O��gA�V����{��N��u�����f��dI~������'?"�p�+KF��)H '�����49��I�s`�����`A�r��������5����&x,Q~��.n�f��.Z/	X�����-S�UH�C[	 �|V���+��]-?�7_�0���*��P&b-8�oo���f��{�;��!y'$��,Vo��e�T�H���#�a�"M����|�p�o���m��l#�B�u\������sK�������xQ�nl�N�����}������LR��L�J��Q8f����8'��|df����2F�J�x+�>�������,!�9�`,M ���t]@��V)��{��7v��J��3m��@<��"�ZY�;x���b�hl{
�[zP���������
�8���~�a�e������l."�zs�����]�_��z��"��"�-
�����$���q�
`�=���?�{@(f�$?�DO)�nEO~<}���d�Y���k���#��gOy�$��g/��<����&m__��
A	��Rl���n���h���A�b���������pg���Wb����G2�]����jM��������G���������W'oN����'�$,���9�=x����>����b�4w���k���F�.����Z��i�*����(�y�!k�|%
������&��j�T1���C�h�Q����j3n�;��?���`��U��Y���B���7K�����3}PN$Lg����A'����ph���2�\���,�P�/VEIX���@� C�"_�-V/F��q��p�:����E�7p�-jD��������%���#"�P�6��\Z���^m@DJs8��R��Z����j��w1�
�����bl����%m��.f���,/i6���4���r��a�o��(US��a�6gxy2�����t~����fq���K_/����o.H�H!�
��(X�-#2��$��h��A:����NJ���a�u�+`��wXI�i~q^\�_^����
�]����A���tI�	��	�����fU���q�F^J�F���s���l������.Q]!A"f�Hz������z3���.0dCfw�f�p�@H#|�^��2^�-g���������t�+u��'X.����(^�;�x0>`�G�a������w�!cz�i,�|Nf��,(�~e��G�����l���+�u6��Z��n>{HD����GX��EVB4_U�(�������z1�]Mu'�@�r
�t��!k�X95RX_.��j��7u=���{�u�����e�.�ROM~��Ao���`})�)���^jW��`kX[4q�IS){� ?N��	����:h����!;�u����6`4D�9U�	��p�B��OD�����e|M'��A�)Y�����h/`���>���<�tF��$B�S��|<����7����F���%�]�%�f�R�&10MY%�qa��qJ����2J*��
P(����$�a�|U�%���@�@���u�����7�1y;�HE������V|�kxM���ubx���`����\����{d���+�$'�r�����e�!����&�F�����Z<�#��K��j>e!?tb�7P��C���S��u���:\��p��u���:\��p��u���:\��p��u���:\��p��u���w�.���%
#22Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#21)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Fri, Feb 16, 2018 at 12:14 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Appreciate you taking time for review.

PFA updated version.

Committed 0001.

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

#23Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#22)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Fri, Feb 23, 2018 at 7:35 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Feb 16, 2018 at 12:14 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Appreciate you taking time for review.

PFA updated version.

Committed 0001.

Thanks.

Here's patchset rebased on the latest head. I have fixed all the
crashes and bugs reported till now.

One of the crash was related to the targetlist of child-join relation.
In basic partition-wise join, either all pairs of joining relations
can use partition-wise join technique or none can use it. So, the
joining pair which is used to create targetlist for a child-join
corresponds to the pair of joining relations used to create targetlist
for the parent join. With the restrictions related to missing
partitions discussed upthread, this isn't true with advanced partition
matching. The joining pair which is used to create targetlist for a
child-join may not correspond to the pair of joining relations used to
create targetlist for the parent join. Since these two pairs are
different build_joinrel_tlist() arranges the targetlist entries in
different order for child-join and parent-join. But for an appendrel,
we expect the parent and child targetlists in sync. So fix is: instead
of constructing the child-join targetlist from joining relations, we
construct it by translating the parent join. The basic partition-wise
join commit had changed build_joinrel_tlist() to handle child-joins
and it had changed set_append_rel_size() to compute attr_needed for
child relations so that that information can be used to built
child-join's targetlist. Both of those changes are not need because of
translation. I have added this fix as a separate patch with those two
changes reverted. When we will lift up the restrictions related to
missing partitions (I am not sure when), we will get back to a state
when joining pair which creates targetlist for child-join corresponds
to the joining pair which creates targetlist for the parent join. And
thus we don't need translation, which consumes more memory. We can use
attrs_needed information. So, may be we should retain those two
changes instead of reverting those. Any suggestions?

The extra extensive testcase advance_partition_join_test still fails
because of plan changes caused by recent changes to the append
costing. I have verified that those plan changes are fine and the
queries do not have any bugs or crashes. But that testcase has many
many queries and the plans are not stable. Since that testcase is not
meant for committing, I haven't fixed the plan diffs right now.

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

Attachments:

pg_adv_dp_join_patches_v6.tar.gzapplication/x-gzip; name=pg_adv_dp_join_patches_v6.tar.gzDownload
����Z��kw�6�0�>_��_zEJ���K����W?5���q9��9�,/��med�#�U������ .@R�����L��r����x�����}9[������d�����4���4"��i4f������0������\�����^(��z��?���u�\?�������"_�$'���������d��}N�7�z�>~X��q�xq������l3[.��I���]
�o�,��I([�!���x'�����6&�������=A2>?
b|�w��g^D��x'�Ct~v>�y����}J^�w���?��!���G4���%���)��wS���$�M�zY����������G?����������x��yg���?�>�n��O�����������������������������������������)/���B�EY.��`�x����b����=A���,��wc�|N��m�f�\��<�)y��/���l}���*y�1���QZ�z�[_,���'x�gRu��n����7��}Ip��r����#in�^�&k2����z�������7������O?����/���?������'\�:���_��p?>��A��f��������w��=��������	�����t���u�h�������$��Ww�N���*y�}_�;�|��o������U�W����������g��~O+��c���S�FwO���l�p|����dEj��
���*!����O�La(v�tF�����~������f����Ch�^�h�|�����lq��e���;�>�gd���O>���u>?��������A��^FoD7����}��'���B��Z~��_0�A��<M6_������O�N�0>]��dA2��Im��7�Y�&������|�Q�-��o���G��p��d������I�x}! ��l�!��� y�?]��L��?��������9���XJ���&���.��g����e*��]���P��x���[�_�y���5�����@���	=�!�'��|����~���S����x�|�;
���y���z=�;� ����uZ��������hc���)��C��Fm�!�er}�����+���������x������)��=���5�����L@�?��?�J���v�
P�	�1��G*(�	R=��Up8�z
���?���B	��.�O~���>\}��N	�G�������7h6B3��G#�!n��p�7�&���!��c�HY��Dh���xl#��q������
������~��ti74B'��&W������P������T6��l�N{���A	(��e{P�n��=(�c�}���{�/�`�#f-��@����^l�q�)}�[
�Oi��w���r��

~�������CB���l�w���{����M��h�og�����aR�KJ���O�yJ��>^�tqM�r��xU��?�^\��p��do�ln��r�I�#��7�����Y�Fh���,������E/���4Uz���q���#�c�!7
0���A���X�iX��n|iQW>+2H)��x�
4��������.��q��^6�[�Q���+�3N/m{#�W�a<R�����~��6F�#
�G����4H���H�|���HC2�����i�v#Z-����[E�1��;�;es�����#�}�Ec\�x��2�t�����L+�?���lZ��,��`�>Y����i�0�Y>�Y��I��o�<l�3�7�2����<9���*yHV	��#�0n�������q�|0���~��������6fXA(Q�Dn���h��Y��d�%���*PX�{B������k������
)/��g�H�iRc�fsQC�u�����g��^w<'z:��5o<'���9�S|��(�~�91�:�=o����xN����P�3��V�vh��g��"Q20*E�G��TS���q�7.����G�
��$�f�Y-���U�JY�!��0�w�A�F�)��i
��-�hA/��0�4+�
A^���4�h�`\�I���?��uBZ'da��\��'�
�|\�����:�d�����El��;�X�U�=�<Wc������k�}�4����������8�r�,����q�}z��M��[�)��a���C��4�-���9��9��#Z����h9���I��I`;'Z����h9���I��Ih;'Z����h94���9�m������)��@�����_�-:>�o����(}�xI+��f �����&�!���%;���yx��Eu��<�K���{��9^��9�AVO��������t��?)?$������tw�U��T��e�l��m�1����g�����,��vs�O�jY�4��auY����f,��i����D��R����x~
�"�,��`@x��N
�����`��&so*]�����r�S��S)���SO:�����x�N���������Jy?����9�-�T��u2����v�4`�T�;*�4��i`1�*���954��3�!3�R�S1��tNC�9U�������`aC;��2q��Q^yZ�M9�=�� ��9�?(Se!��cx���qtdGf\��u�����9@�<���J�q��
��)���G��g����8$pa�X0�6�����%�W+���9��6>�L9b�E���+���*��J��/��j�kp�j.��^����U�D.��S�2���|$�|Q,>/��(���r���#CLG�TL1z�X�QH,��T>��&�>�F�3�.�����g�m��e�U
"L
b��H�of�I��C<_'M�����61�6�D�5��	��8���Xm�RuC�4��'X�\�#���bY�`����k�
��:utI��
��zsf�r���O��|>��f���_g��|y$��������)�<��k��|����|>-�W���s^w�:A�NHK�|/�D�O�������-o�`�f�����A%�T�oj&h?<����:���I�k���@�b:N"��&��~��Z����M�F�n���r���D[=7���7�;��Mf�������&V�d`?os����f�nT��t���L��l��PFe*5��6r�a\���&p��]������fdZ����9fF�����������XmL��������=5>0Vb�99V�f� �����3 �mN�
9naR�|R�����6'��K'��Osrls���kfGJ���������;�g@b��
6$���?���	J,�H��$�gc��l,x����q���r�o��P�������U��������9���#~���r�e�t(c\H@��| P�����#En�O�Y0 �d�'�!�Zw�M����~-���H��ChQ t�J���!�(�5���Hv��p�G,"����Zy��<��0W�w�����<�N0d�4|7�$�����'��At��yap���ix��T��p�0���g��U����\\O.yB;���&=��0O��&]�Qz|nT��g=��-�}���Z�E�z���U�k}nM<n���(lm�:�Y���R��)U�j�6q�$��5}�Q��2:6�l�D�e#g�{R�p7_�F�`��]V�F��e��4^;�Cc4��A��_��6U�N��W����AA����7�����]�FA{����O���e��4^?��x4�F����%������25#�pP3���g|p�&v��b������/�X��|%]K�J{%5�����{H������U��."���tJ�U9�OG�+I��D����T�K�r���9U����r���q?���c�Q$�8�s`3o73Z���������`�W�r��9�����T\[Dxa	/���7�1�L�v���Q	/���Jx���1-w����S���� ��IH�"0���oa%����<�go�-��n���+L	x��g��go/1u��;����Lb9/��W��&c��
���
�� ��8S2��=b���>�-=��t����4���?\]�����2�e<��R<6#�i2��J��Nt�u�cEQ+�� �
��qv�$\I�����<����l�<�)���2/un��
e���U���FQXk�a���e7�������.^�W����������MP���8-�:X��[��j4��Xx�x�5���������eS�oy��Qoc�M���&<]KoX�M��ad�7��&��W�mMx=�h{�"��M,>��|6����}gZ_}.D5�Q���
4^�%`�z,LY.x��� RY���He����7�`e�^�����7�T
��e{��W���((7Y�������
2���X|�z��V���L�qp�����qp"����qp�����b��8i���N�*/B1
lU L��
D*�U��� X��J����W`��6B���l�������={|�ge�J�Y�U�y�p'MFBUc��@�j���6��}5�W��!������0#|������4F�����?w1�/�I����0�n��N�CH�V�Zg�}��[&����Xn�\J�������?'��D3[��'��=S.xNO��R�V�j���{g�
������b��	�P��c�`K{�1(|a�:��=$fw��/�y���1�0�n�����d���N<��V��z���,V"+�:	� ��N�i�	!���������0�vf�888��
�M�~��H���W�u���BA��6����������X�����v
D,�o��p�d��%�m?���l������]�MbX�b�~D��39K�S���#}{T�8�\,��m��,D����G����p�Wxl/9f����0�9�������g/�f�����p������pe�%��2�"n@�f�����eC[	���F����Q���4��4\���?�^N��=se�pOR����f�"?D&INA���>���f�88I�f�88��f����������&�X����(b���&�q�5�-�^`8�I ����)�)���Nye9���� �#����a��`95����`95����`95�����I���(wo:5Y��t����gVS����� R������`U��F
�UM:l�(Y5igB�~�(����S~�w�*��V�o��+�U�
��D*�U�
���)�U�
�(8�r��[U}�A�B������V�m��+�Ui�;t�
lU��@���
2V�d,na�X�����
�,5��K�S��0N�S��0N�S��0���]F��}�����*H�O V��
R��6]A@J�t�,Hi���)������8w)�RB@��-[v�)

��KNA���>���f�88I�f�88��f������N����&�X����(b���&�q�5�-�^`8�I ����)�)��������1RFw���lof��
���m�0:yM�������]�8!-��=�ND�dT%jG��&C����>�k%��#Ly�����L�[o�6@gl�����
2�R�T� U���<���Uv�65�>������u�����i�T�uKR��i ��������0��9�!�*4G(���j7����Z��C��hj��k������8�GU����5%��k��W��0��_�!�*��^l���nHU����������T��@���!zq���R���T�C(3�-�a�M
�C�Uh�����Bkc@�K���v���F�C��d�Q��K#7�h�nvf�F�P���$��z�5eU�!5k��f�>��&SS�^��l����Z�J����qOu�����v�j��N��l"��?`��'0��'0�t
?`���y��q����0���?`��~$����V����0�T*��������*{������m�=$�g�}��c�Qe���S�0��t
�>`���y��q���f0����>`���}$����V���f0�T*��������g/5�`�����%���&�F��O[V�x�Q�&`�q���~���Z�vd��D%�%t��,@&��UP[������9�q�)�@&��U;P[���������M,`
2�� �2X������A�H&����M?`2�6!�2�����^���R�6�^����w�C�K��$�6mBp�~=v��&��I ���R�@m�N�,@��`j�pl?`��T�O[���@`��O�2X}���^X}�RcV��V���?��n&��HW��j��=��v�H�vz�?'(��f��x��mf���4�{B�5�O����|E+~���|Y����<����X�Y���
�g�	Z>�U2���wG��x}1��`f	
b4[lFx��?w�s��{�WC����������v��\��b��� �����������o��_/>��s�����y�n>��X�35�0�H�7�B��p�q
�������U���F��f#4C@~����-��������;�fH�=&�do��u��%��1�>B���W�H B�nkr5������Z�o�.S�4��u��������~]|��H�T�E'���������q!�b!�U��/dT]�@\H�`�za������Ds�lTP����4M�~r1�m�.�]��`���f�F.�U
[�hfR�G6#���[i���i��ni�z���B�i��7�)�=C����W��0�z�O��i�:-#���xV
c�y�����*G��l��Y��<9��0������]R�D���`g��M��������N�t���]���54����<������3bl��lNL�8�sblX�`x_��;�,M_�?�CD!B�B9���x�2�_�gy�2���y���M�h-����ZY�1����B$�Y���6�O}MkP�n�GzQ����Yyo���:t�t�y^@��N�d�	h����:!;��B�?iV��w�#\��;))Wc&_���l+j=�+�_!Rj��!F+gN�)��4�)J�����?�Y�r�n�i�����	xK�~������}�����Y�	���Gi��������S�[��U���pZ3���h���x��^�sQ�x��w	�4�q��x�vG��������#�-����!Rn`H�YiU����U��e�l�
�F�����3foK�2� ��������fG��#C���Z�R��d%5����^�`�M�*B�(��YN�!W�=R��	;.!Y�BSa�zOkD�;������Y�8��r�8*N�;%���o�~X��]�Y(��?(c�QA,��h�s��`��8�^�}iO[����9%��caLk@4���1�SJ��G��g���ng�����3�j���w����#�*f�/z���0]���L��#�s�
�~����3���0�_9�s�2j��5y�&"�b^9��X>��|��+����0SS����h�k?=���Dkp��T_�z�w(����q��������x�N�N�g7%��\O�D���	A�����C\�eD�K���j�CW����NGQ�s�s�
����\�;��1��k4������G�Y�1_	��<����|
7���j�"��;�>�O�����F�W��NP��R!���9V�S�<;6�,�����l�t[�;��
�]M�{�U6 `d�xG����eev45�mKl=�I__�������4�`�O��O��.��;����Zm�Pr3�j�E��L�-*�E�����:�#U���ieFbv���r3"z`�3�
����"U��Z�P����Y�{�OF�(�[:� �6����0)~>)v>H��I�A�SS��!
���E��(1$���X0�;��j�x�M��R=j���J�MR�)]�>���p������Y����e����
���5�XOJ��o�x����%�=>�
���mr�����Ch�Gl��
�S���c�
�&yD)����}yTq����4oS��i��<?x7�$�MB��}&����\�sS��oUN�������%��'��oL�+�~�'�z�.��m�����/Jq:1����!Z���z���U�h��l��@���k�J���].�/���[jv��
���q%r��y���o���$�#���b��=�`���Y��*U�.�V�����7
��x�,�l1�w�����Q�wZ?�@R3�(�T����b���_�� O.�/��j�����?��^IUK2�4c��4
�|��
�F��!����J��
[��Qa��@x�1�j��J2�94�������D�W��5�H�q�8��4o7S����XDx�z����c\Z�vyCL�#�Kx���Z������rq����u>�����������@�q���P����FK�S�>89SNN�DQ�c����PR{��t�����y�Q`�����}��a�!���>q%�B,9J*�W������:��6�*Cw$������K	������U8����J�|wq6���;�:X��5�p��G>4�����
�-,�u@pgg�!�������o�[�i�����jvvX�RM{�!�E�n��+�Z��F��W�����. ��!BI�1������1ke��5��Q�e�^�����3t�6����c��#��e����w6tW�`���t�Z���l�Tm��W�w\U��{`�*�����Cf�@w�m��,�]i�6kW�����,��
RAyJ��O	{�u:�o��'g��
�Bn{�I`����GR�\{�|$����6���#�v�+���&d��b���:�"�����Sd�;Y�B��'��bG�b�99�@�]���3��'�!��F�)�)��<��
&vT5�e��
vXJ���
�M�g�KVA=�+#����%����C�sKXmw�]&�/{?�a
.�o�ik�	�������n�����|L�I�R��m�:��2�e���d�����xg�8�+�� )�����}�t�%w@������=��+�!)Z����gp�O(#�Oe�>����f�~<�A	����I�d�2.PF��=�k?��A	������E"p�m����(#��m�a1���4��f��LWA�+u����%��7���J�[	VM:l�#�Y5igl�~xA�����']���
,�]��;t�
tWf��
\���_:�]�wX+�����:,tW������.�W\���v�+u���B.K
�����@ue6�7�Sm��O�o<�[r��C��L�a)���Q:lU5J�b����2�"=�C�.���	���'�2R��o�H�a?nt��/:PF�$`�@	(#������u��ed������c$������"����5</W	�<a� ��m���9��=8!-�1��ND�d�G���l2����o��oG]��KU��w�D���
@%�XFV�t`l�Z�+�.HE�����e�]&�M
d8]��e;P������,�J��c�f��h��x3��0�B���d�#k��
�8Z��=D+�����Z�����A�:@/��=��=f�*�a��'�C�Uh��*W���*ZBk�G�K���v��>J�C����R%Q��5�k�����^Y'Mv�����&L��:��<���<����k�:�	�N��N���P���T'�:9(�I��^���Jsbp���b���	(N�:��4(Nj�������'�81O�81�(N@q�P���do'{)����Wz���o&��z��&���&*��'���U�	hOZ����'��$WD�NQ@�bRt(\e�������Pl���#��!�I1���2hR���^hR�R
MJ�4)&���j�$�@��&<P�:@=�@@��<�
hO@{��]kO��f�7��7�z�~�M�R��Ic��O�A7��\^���}5���L��L�t;�'��|�^��fF���xqOw�S�9A�x||�'%��f�U�xLh�/����,���t��/O�BY��O�g�	Z>�U2Oc��C��?�����/&7�t�A�f��o���}�WwO�j�~�\�|����6������^���D�)|O��x��6�����N?�J����#��<Pc
��TP|(�z
��N`�TO��P8�>]\�`4���F�����F�[2�����������B{L�*�$��d5K��{����!�L�J
�����jr��[�:�n}��i�e��;��/[_	(������@�[__��>�������q!�b!�U�������c������.�a�8�
�X���(���O.�Y��E�j�GT�z^�X�d�M���5?j��f�5"��`�4N!�����q+�i��pK#
���#
�HE���H��m���SC~�]4�a�k�g=�����,���2�<����<y�d�!
#�e�y���r��r��{HV��.�#d�����ob��5�����$c�����5�m��-g[CGnRf�XEk�1V�v6'&$�P�91��t6'z����k<'�N1'_����bx4��`p4�hU��P�\2��C
1�7���e����h���d�Z"��1��JcR)�#9�p�~��mD�"�����)�"���,CI����!��u�P�H�����%�4+���:['�uBvH��L��0������:����)G�W��TR����BO*�~���>K�b��m	��(����@Xq�H�kv���(7�ckmgs��vI5Qn0'�&���$�o�P}�9��s�e"�s"'�[D��%��Y./<��(�����S�v|*{������n@k�����������|.
�/��.��=������{���S���y��ev�0�O�0w�i^�R�/�f�m(5��(����x[��Y���n�f��m8��Y��$��ft�a"����d�k5wp<?t�xe0 L�e��d���h�A0R��7���tf�A��8�������P��'�S�bNUE'sj�FogN}fN��bN}���s�bH:�SC�};s0s*exsH�4��SC����
�)�����%|q�3P�lZ�M���}W�������HU�����j>�}��q��Er_�����j���$��c���1����O)��>��?��];�I2��j�_��	�WK�����k�s&}�}��l}���f8�|���#�_�
�~����KH��H3_9Yf�r����x�&"�b^9]�X>��|�l�����p�&��%��Zq�J��B��	���PPY���"9��='E��|�4��nJj9��N�FSU���r�)��2�n�������U__u���[�*�L��������|^M��g�=�<:�����HP���%���S�)x>������i��|Z�����t��.�u�j���
�6^�����������[^1�P}f�����A%wP��j����d�xG���������
e _�NF�������������mN��n�0�v(��\%�
�q/�����2q����`w2���c\d�g��XM��aJ�Z�d$};x�����%�d�&��$��Dc��[������L+3�3������L��y����L)��q��E�5�e+V����]�-(�4�<6>����)��<-L��O��;��isRl(�tR�)�4�<6.��G3;Rb��)�7c U�mw�R���A��0����XpMW�*�����J5T�?>��GL�k������s�P��1�3�������F_}�A��@���Ii4#l��L�Y0�5d�'��!�Xw�M��i�~-J���Ch��w��fXv����>�
��L�4R�k�-����@VM�i��v���;��������H6}�;?���(<������$��+��������oU��������%O�&��oL�J�~�'�z�.��m�����_E"�Q$��!Z���z����nj]�L���I��l��YnIm�P~K���V��E���(T>5}�
���*[0{������T0������*x�U���jF��k�v���Xi���kt��
�i���d�|�H[���2^�~~���r^[	�i���n�(�����;��$�����("�/�Q��~%]K"�����ue�3���jq-��i:*��U�X��;.�*��OG�u�j��e����V,�+��#���7-W������k�������i���L9^�k����Z��r���9����[�+�b^X�u��U�������HT��t��^��_����u>�I���������ea=����<�WW���.L��%L	xuYxu�Ee�\���V�= �E@,�?0o2��X��?�o�:,4������	��|�%o��PW����:�;9�7�$FS*�v��j��8����%�U��[+�������X�]����e1����.Jh=b���J��Z��������f1���\C�w���B��5O�Xx� ��zC�i�5�x��w�`T^��joG�f6�0ZZx�X[l=UB���a� �;��}y-����CX�@��S���<���7���e{�s����(4����u�_�.�(p�e[��w\v{|0�{�d�����5����6��q���6������6p��c����;v����i�YO
:���o������C��Ag�m��]�Y�dw�d��C��ph�@7������!>%MF\uc����n+r�w�%veM��#!C�c9s
���{7T�����1�l76Z��Q�r�/v�/��s���N�8��b*���Du_��BH;G�P�]m�v����9Y=&������J���r�sZ"&��H�Lt��6�;��g�����N�=��$���n�X@;���QM��^�]���J^��.
'��B�4h���R���a�����l�/�a�i��v�)������`����@��f��yZ�vW�u������������-�m?�.�l�������E�Mx��~o�������JRi������\^*��Q\��TVxl/4�W���s��u�����0"^�*�K�Xr������#^�*�P��������T����p#
7�+����BXw�R$M�8@�>��Lf�88aLf����������&w
&����\7�8��"�gL��&0q4��xy�q��C�i����R����$/�q�
���-�q�
�MKX[f����+#������o��<�"�e8��t�6
��k�a[3���I;�����0��,>���1��������C��Agm6����Agm6���1�i���u}���F�:����Ag�m���k�Yk�.{W|0��b���7kPY�;
�5rYjPYc
*k�aK
*k�a�ifz��0���L�%'k%�O�]��B���VaC(�t��l%c�v����6��a�%�d�C���&0q�&}z��8��qp��8��8���|���Q���p�`��L��o���}0q�����8�����-���E��-k4x^��y�h'@�5)�������'
pBZ&#M�ND��N�� �&C��M�����v�0��&�����rd4��8����dh%W� ��C@V��Uv�65P����8�u�����)T�ue������gf�4�Z�b=$Y���Z�R��T�Z�>z�VMM�{
TR:�"'�*u�^������F�P����$��z���U�!����Z�>��&SS�^e�l���I9�J��F�
�]��,�����ZIV��b��vC����}D/M����jt�����]�Du��Icu��*����W�5&������&��@3�Yy��y��A�@3�Y���'��w4����,hfA3�Y���fVR4�m��`4�����b�W�Yn�8�!!�>x��^����N�^�������eA/zY�zY���^�����,�eA/zY��J*�^�����^v/Uj����Z�%���&�S��nn���5���`d@9���z�#��%*����S<EP��T-WT�m�og*Z[$��H�%^'(jM*���������3E�-��EU�x���5��Z�2�k���^�k�R���^�k��w�C����E�Mu-x����|(j��Z��-�h+�AE�V��������T�l[�/ZP��Z�R��m�o/��{�R�lc��O�A7��\^���}5���L��L�t;�'��|�^��fFn9C��=������l�V��1A�����l�K\a�^����k�Q&�E����2� 
}�m�PymW���w?^_Ln.��B�������������)^
�/���7�����zr���A<�^��M���S��?^��M.��D?���p�n>��X�3�� ��P|(E_�q
�������e���F��f#4C@~����-��������[�fH�=&�do��u��%�t�#��qe�R ��&W��������!�vk2�M�/[�zM|��H��@��
��(0�E��	%p~�x?���F\E�X�l	g��xZ]� ������`��>���\�����*���G�r$y��Yf�)n����N?`+GG�+(PN��>8y�����AF��h�k�6�
i+���(/V�?%3����v���n��j�@��I�j�-~<v���3lZM��iy0-��sYK���
Z�n2T5������<9���*yHV��.�Cbd�N����l��&���Q��{
 23�rD������_����$~/B�BY��(��|����q�7.����G�
��$�f�Y-���U�JY�!�p���� o#����L�y��eyJ������ �C�JG���\0.��YA����	�:!���CZ.d��f��<~^���D��yI�JE������2�7r��*�e��I����U�@V�%.43�6o`�i^ s���K��7xX�)���@���~����o��)};>��E�[�K
X7�5i�����5O�HO����
�D
��N�i��{!G^�s�G��w	���q��x�v������������t�?)�1$��T��U��*U��j���R�}�����L�-�e�A�K�Ug�f��r������V	�{���TT�/��h��@�T?��"�6�FY.�3l&~(���*�C��T��
���S
�u!�;	�;��R��}+j��x)NQ:����o�rX2�]�M(��?(�PA ���l�s��?���u�����5zX-�S�1[<�U2���m��c������$��DQ���&�k��#���f��o*�Qc��c�u@w��d{hH�2�+�N��Bs6� Us�)��|�,B0/Y�8���r2��#Ct<~���$��)����L�{�M�4I��Wj�#)��xP{|�62�����.���7,�Q���(%�#Zq���y7���8���������{|yt���������KV���pS�|>��)�i�������_-��|x�O@��:!-�mT���������[^���Bf�����A%�,�5I������dnv��������5s�e�� A��o�����v>uJ�R���H�i�E���'���F�"TK��Vf$fg���)7#���F��5��R����]����S�!?;���t������;���y����)������FD�r(1��X0�k��j���M��Qg���J�MR��\�>���p������Y����ed���^����O�'%#s�M�8�g�G�d���A���3s��8(=W4�����F�
��iD)����}yTq6���4oS�!5�{~�n\I$����;N�H�Z������������.'7��K�N>���YK�0O��&]�)4��%�Yo���ZLO�M�C�^3����nC��P�k5��zG��*Yo�{t�\��K�\\�H z�N~���I���`������C�(��:����T����e�&,&M�Bf_i��ag������Z��@��WVe���k�O@���WR_f�bT�U�o��LS�
����r-B�cS�:}*U�Uu�/SW��b9^U�8����_F�\Q/������fI9N!����fj�r�Xu��\V=-���.��b�.�����Exa	/���W�0��y�T�x�/^����[�_����W�l,�vR�������mf�V��%	D,�C,1��7YB,���7\b�AlP�kR�o�����b_�:�����K��3�YHi!��$xI�@�&�{���5����;Z5|��W)��t5��Q�a�U�.3e�T)���0m<_��&;�1���J��.*c����'���o�V�I�*-K=���;%����
Z�Q�U����qi	�U4����:N�$g�#gW"'!#�`8O�Nt���7�	��o�9��r���C������'� (����>�������I0{�ToM<l��2�@PA���J��]�I�sl�My����x�u:�m��O�����W�,\�m��l���Oyy��iwt����i;�
��3u��zu��s'=�������g����Mh��)�g�y��#[�5�P��*�J}��$������k��K@����`��Q��>���K��&��.t�[��]RL1�b�S>�o������c[Z,	�(��Q�)�e��Sy1��8�+u�����1��.�bI@HYY{�D)�P�b II�����v����h;@�Q�@���6�A��noX�q��n��mh;�m��;����oI�!���;@�Q�����Pw���su��I�;z��0Y�Cw�U(9�*��c���A��]T	tr��$����r]Ae�6�BP�����g� (�UAy�������@'��[X0(��r2����9���^����@�����9���$f�F����U�	����9��?���^��mG]mh;\��m8�am���������o�����v�]�q|�;�I�n�<�-��l�X���r���F"�I�m_hb���L�>�R8-R8Q�L�-65���92s}L��>F��G�a�6���F��c��s?�53�����Ud�f�<���)Y��+����&'���;zC8�I1��9�Jn�u��i���6-�("��'��q
�]rR��v�U-�>U��f5W	W����������6�.9i�u;����UU�v�������^��Mv�����&l@ �"�RAJUV)��Y�RH��)��.M ���f)�T��3)����R��IR���T�QAFU�@F�!��jY
dT�>��
2*�����~�����Q[�s�l���Q����v_;����-���j"9��<���iC�
�#���E����6��B�bD�}�"het������[)��G�\����O�=�]y��O�c���%/
�X����n*����a����dhQL� z��a�<�����mn}�!�������{'^���XD����/�f����t5��������a��tM��'��63��)�����e�f���	��4�{B��<�G����!~�o�Zh�������U
�������:I�&��-���r��M�nB>�mb��H0��2-���2�]+�/%�}��������_4�q�7#�W��;�9^�=��!�er}�����F��\��b��� ZY��{
���k��������=���U�<D7�`����S~����@����	�p8Q�d��J��t�~���
�������
FDH�*�����? ?�(fyKVe���y�3�t����E��7��:Y���������!�L�V
$E\����`��>���\�����*
QF����G������jr��_fu���n��&���&�i7���$��I�|�l;I��9�H	G�N��D��������N~����4ka;����q/��y��/{e/����(���	�+�q�jL��EA�s�"���Q�3t��7_8���4z��2f���6n��F�<p������|@��HM�����BO�7�G��q����3�Z_��kyx-�N^���'Y��O�	
���
Z�n2�=�<����<9���*yHV��.�C�d6
|I��q6m��E4�������
��<z2G�������Q�{����q�7.����G�
��$�f�Y-���U�JY�!4a���� o#����L�y��eyJ������ �C�JG���\0.��YA����	�:!���CZ.d��f�Q>��}���h���}B��0�I�N����:'e��w���U��K:�X4Q�=%[�����1�]g����V��
�ZaMC�*y3�@(Pll������7�fh^���^�\�;�+�!�7$��%(/D8��(�.����S�v|*{������n@k���Ik���'���8���:��b��<�K�(v��?���;�de�d�<}�o��H���2WC����W���T����>J����gR�+��Y����������$3�[������d%5����^�`1U�d������!����4ru��`��	

*�*4)q���n�T�B
�uN�;6�;�;��;���%8�b	�3�)Q������"]��������P)ULe;��m������G�O3��z]$��=�a�|N��l��V���j�
�S��T�]��?6ZG�����Z���T��Z�aph:�F89H��P'����N�9�6���})��N�Bs6���-��������	Za����y���������$��M$k��r�_�|>S�'��!����\A�����8�Nn!�����������~��y��t��}DU�W���D���eS�Yr�,���zL�p3,e��K`s�������]i�x�k���
�|�~�,_�����j��{Vi�NUj
9��S�<;6�}�+r"�l�t[�;�$��0�&�=�	�t�����A49*�t<��FJ��:��t���nJg����
LY{e�:�����;J�R��h��Y�D��h��P���07�~r�4���hZ����Kf������	Td�0�4���[-W|J�a@�@��+�f4�
�M@v�������&�y��7�=c����gL��4�C|K�D'���XpW>
c�1a,x#��.���bC�j��W�c�Ij�����i�P��1�3����������8��@��S�Ii/��o�~L�Y�)�+p)��K�o��/;��z�v��xiG_n�t�A��r_+o��G��j�N��{:>K1���"��NO�������x8�;K�V������08���qr:}r���xnz��[�������zr�c��'����C��I����3"�F�%�$>�W
�i���p��kF�^��mhupj����]�o�]�J���].�/�����G����E��J& �`���:���Q�yup%���| �~%a4�6e�&�	M��g��&�����bkbK��H1-�f ��[b|��%pU�W���' �T�+�^M�������������U5E��*��Rn��Z�tT����*^f�����r����e����G�Y�b����cM]�r��q����������0^��iL,�k;��]�dV1����^���Se���v3MkT��t��^����e�
��f�NJx'jx��c���<��s���>�+��}�?�x����*x���Rw�/��8������R2�� �&�����#�����
�� ���},	#b�B������'����N������%!�Z7	��4������!]\�E{b��U�J�K&2+�'����~s�1�Yw�h �:�`�(2�BV���v�d�8�r[�Z��m�Y��+]4k�E�� 0u��
�����%d��L�p����i�?Sb%h�\
p5��Wc�,p5[�U��4S��y:x��4��?c�	�g��,�3�"S��4CS�$�����(WcZ���\�K���l�VS�'��,$���$PS��c�^eOb��d��4�dZ����F��0N!�91�f`�1���[�w���f��2����+�nO��2����+�n�{���e�c�#���k���7�1g��������d��Jo�_`�2�>�1��}�6�}����R)���I��F9tO`�+����s6��d�7��c�,�7��,�%������#�����-�$��@���x?�����%/2�p��
$e={VO�A�N��bYY{R�<A7�p�$eC{�4�
#F�a�FnF�
���ey�1bt�I4�D��2N���u�iE`����.�;>�j��}���]�A����j��Xt��m4���{����a�@������x)	��@��>���	�a��m�m��2��������d����������f���iv[\�~x�S�s���@K��iE�g��3.�?nY����S�/�4��W\�]����4�-�f?��������6�����`g�	�`gl�m���.�v����Cw���spt���7��f���>5pt�a����+Y�^�3@4���>P��Vv�>;��,������-A�g�Q�#�}������E'���F�����s����}��kA�w|�;f5���y:[���r�F���*A�'��D^��m2�&W��xXpBZ&�7C)���	)����MFW�3�����,��B2jM!)���I"�=�a�#�=�H!)W�VA�1�*���B\�����+*�����HU�b�������k�jS��AYE*��s�J�,����Zh��+����W
n�gm�'dL��,�s],�M���0o�@��h ��4P
��*t���_�r��Pi�tP%{9����J�]u�R���nZ*Av�MK��q7�yce���.`ubT�eu2�q�z&N0s���@V�Y��"����2�� +YY^d��wd�6����r�$��VT6��`�	�����2H� )���2[$e��[�r5��������RpAyo��43&�z�e�0��g'�r.A\q�e�A\�'����,��r��	�r����������{+.�tq5��m�e�+����6����iA� ��� (��� (���>�{)����UA��������_./������O���]&����������u�A�5	Y��?���uq�F3���	��y�,���63�t�e�Nh���r�^���l:OP�. ���cB����U��������fr� �-mFxS�?w�s��{�WC�������������\��b��� ��,��=���5������O����*}���hp|���K`�t�~���
�������
�81��U�Gf#4C@~�?��-�g}��g����.<&�do��u��%��q4��7���SNa�o75��\��_��v�����l��l�|q�H���<��B9,q�|�����e�6��:��>~�q����g�OC��(�80*0�~s��&W?����Z#Q���b��Q���4�T�hcG�����p���^`����cB�(���������;���_
=H��f!/�<y����&��F����/O�yr��a*�������#}4p�c�����g��>�]}��C�&J#I�r�Uv����qJ���4��i�-��r!S.��L� +0��������r>S���yL9/+�1��bcR,�#9��Y(3��xl<]�J���4&�
1N��~�sXH��U�u,L5^wH�D�Y|��%R��D��t����x"�+H���W�����O�4�4��ya�����<���y������*|V(�r���u>���y|�`��gc�h��0Y�����#�-�\t%���w7@^�R�/�f�8�*�
���gl^o�2� ���Zm`�nf��c�e�f_lo�'+�A[<��!��`@��e�C2k��i4��6�~�9��^�Z��;�i�9O�0��~�E�������Q3E�_Y��K_���0��u���z�5zX-�S�8[<���\���R��>��?�wV8�HH�y�]kR�d��-XIk�l���g�;�M�\w
������/���n������EXL'��������M]I��|�Y�Z����q�>�>|������M��K�m�8��������4�$�T[rfv�0�r��!W6��\� +pe��e����h.o�-r+ww���%]4�j�%�U]���yvl��������,i�T�����`����
O����f�����7:��E�Z�H�����c�����Y{�?K�������$��)�yQ������������HP�
)6�r�|��d����"d�YNM?�w��g�i�d��������U�o�i���������?��i�;�����t��j>�/1�i���$���%sV��
��3�����]�@�E�8�f���Y�����R�����	�{���)w�����������?>�N���S����;
��`���2>;�&��\�sS��oU���������B�O�7����C��I����5"�F�%�B>k}*\Wc��:�z��B��;��VR+j�����*Yo�{t�\��K��R�{�6��T���8���+;�k��)����Y������L���)�6{�9gD�w���#��2�R�+��7����1g�ft%��q�2�^�"+�@��Z��0�54������4_
�c�yjh�SB��8�"GRq<)}5f�����g�:���a�n9}6��)�>���	4���viI������1:��K��`_�2X���6d0����`'������O~Km�N�����lx�$�D���i�(-z��_�Fpx��"8���vgMMt��g���G�[8Rl���i@�����7>���Z�*�q��p���"����zw��5E$`O��{�=[C
�<�k�{�i2�����ot_�f{���g5,�a�M�:% {����=[C
�B�f���_�tx�x�'�2%C[��o����1�8������w[`�t+���&	�-������6?h^6�t~������q�m����:����QN�:t70hg�_5�.�[6hj�������]9-�}�@FR����*:�#��+9�k����Q\:65���-��X.�J���%��G�C�i���G�C�\)sXr��/Ea��|�R�%��KQX�!`��Oa����R�xH�+ECDI
��h`(�1Wj,�)� h^��
��,�@
�No���=�l����Qm'�n	?�$��v.4��@����H�v���-]�v@x�@����=�)8�4p��7����M�@N�������	�sWHa��g��rPu�i2�����o�hs�������+:������G�������`���$�`���(h������� 	�v�(�?�$��$pL���[>Ogz-<</W	�<�O"������/�	i��D�R8-R8QT�M&�MM��=�5�U��d1��F�*r��
�Ix'��W5��{U3xY!�D��������6eL��*R��$����H��$�*�N(�,�:],'�Z���g|�Y��bT+7��HC.S���O�NZr����Vi��������x�&�-�k��B�A�pP5	8��T=(��;��T��3P�D���A�S���$��+���S5�e�i���O-��&�]5s���f�r.���$0Q�*�D��&���}f�����1�R���� ��l�(�A��h��>����
�'�N��}�Gz�e���������_./��_���'-���D:@�G/�j3#.��=������|3{�'�n9}^�w�[�i<�xqO��3�.7O�)�����!��.G�������w�{�u����S�~Ga}����e�^��x����st��"?^_Ln.�5��
b4[lFx�?w�g�~�\�|���7&�u����^��t��A?e���s��?^��M.��D�������Q���|D��h����A& RX~4����|3x9�f4���J���L
���?����PB���x?���W�p�����6{����(�����xL�*�$��d5K�rdf����!.G�NZ��C�A���0��:<B'*�����o����dn}��?�����{�o��}���B6���|�K���}_���0���/���n��m��@�?��m���l����@��~��1i�-0��D�^���O7�X��N�Q2M��o���:����ab�'���:
s�D�z	��_��yM
�C���I8�e��i��8��8�,��A:�i�t�S
"��V�2�F��X)X�;���0;�����3C���$�����r�Z6O��H��L�;	���T���k��J��NvT*<���]�T��T��T��#��g* 3���5�� ���f*W��?��
��
f*$3�R�5��0���L
�PP9��tf����y�����x/=�.����:m�@{�����?�mA�!��#��y�������-?x���C�h�<l��u��C#*TyZ��c\���d�,�cN��Aey+NMt�1�[�b�<J�+u������������t��#��d9x�,�4��Z�(�d;Go�����7�FY��g���X���X��,R�%�t��C�
����C�Y���,���K7��D�~L��,P���EE����,���s�����S�s�f�<����,�@��e��MD��;ZC;�d�&�f�,���sE���
3����t����&�g���(�uosxT[��;�Yg\�_GA�*��{�g�1��o"�oC~�
��s ���m���G!3i���m��>���\g2}/�E�?��g�pY��q������'C�J1j�������m���H��,}��w��W{RC{1�n�:Y��7��k�"�I'M<e-��x�+�$�e�N,�����x!�:�����<�"@*!s�����8�L|3�B���9��9e�g.R�R���0�-��VP�v�43�������sj �0�ZaFL���0�����0���yN
��T+��I"�D
��]2���+~�b�V�}?�~�w�8�~�w��4�~�w��C�w&���}A�������;�^ k/�0#��,B-���l�3�����-��w��r~��?��u,� +������}�~@C��U���~W��iY�y��e1.�@���y�e�� ������yvg��m����Q��fo�[��d#�������`�h�A���
(}0 �M?���N�a�1����J��m6�7��0�7�:�^9��bN-mVs*m6�>7��0�>7�:�^9��bN-�Vs*m6�7��0�7�:�^9��bN-mVs*���}NCnNCaNCnNu\�rNC��Z�)(�.�4l���S&%�g@���/�YT��%S�8�� ��Na�������H�Y���^����D�=���)k8[<�U2�7���4FJe�����3���f>�6�^���Bv_�P�:m's�+]�:=����`�K��m�l��@n��J���\.�/��o��L������h;:e������ri��.�����8W�|e��>�_\�*8*��|�����c��dm�|e�T�:g`k�E�|������E��\�v�����jA�C�����M��K��F%���\�����(�����f��z���������9NlWX{�k�
���U�8��\���;0$=�F�gEj�u�)i_�������l��h�aO$��
�$����G#�_�����I��y�C/��������	��P|C�y4t|:���
m�XhU�Z�����N���N�)I���������7����V��qo(q}�F��	o8+������@��]��}��vS��>%��c;�o�vV��S��(�E��J��~N���YZ���b8k����\�����!�L1fNY�������F���jk�����6p�j�����5v��lj�c5�����{ju6�
��8dmaj��$�Z�
��g8��N�Q����%�^�W����\%�T��T�s����_�������G���bv%h�������c�e1g�
����.����x1���9���-;���c/�c�4�n�1�/��x1���9���-;���c/�c�4�n�1o[��6�}g����d���2��?>���x����P�O��q@'W:�Wo��?>����{���/f�?���S�������3��z��z��u�|}����v6�������Y<�PCV��P�������i��a"2���C�:3k&���(���s������j2'
��������f������i>'������&s�s,�`��W��&y��,*���N1����+��/�I_������ck����4������uvF
��_S�'R�� o�F���i0���z3[�m
��t4O�����OWF����\\O.9�M>�����>L���C��M��R�G�;Jo*�Pd�	)z��J�	�������`��k�Y`�~�;���4g�u8�������������>�*#[0ru7[2���aP�;T�(�*�<Ko�!�I`�c�w�
���m]�g��Rmf`��T�-g��wD��n����=J!��].x��H���X3�rl�,t{�N�J�D���O�����~�e���L��[.��F����$����`�������_p�>:0�G+��KqA�Q[���4ID��*����N$[	����(�){'�9SJ���(+�
�v�/�H4��k���,�G�a���:*h����F�k$8��o�u�������,\F�QB���(<��a���o��=i���%uBe�� ?i�F��o�H�s�H�N��[t�^m$}3�9���u>���ms6'�����q:�Sz��?���Sy{8�p*N���������
�Os,t�5�c��c��#�sL/��r�-��r,��rl*��r'�8��;�V�2K���Kr�c�1�N�]�}�[��YDV�_��������>�k�9�-��r�-���mrf�LYW���<����T�Y����E�Qi�j#�+��p`
��ZI�Q���5�ZY�Q��>5��vy(�
.�ZDT3f��~�.e{67?��L����q���U�F�#������Ql���7�Y���F'[�q������op7U���6���W�i�3��v�j7�#lm��o�b��6�!F��GU��j��<�=2o�����?�*�%>�#*�]���?�"����^IO��o <�	O�r�)��O*N1N����S�J N���)��=q
�)�hqJ��)Nu���4�������q
lQ <���gV�'�@x2H <���H <�R?��NyT��@zr���� <�-
�))�����8E�S	�)�	�)Y��8�%�
��aJS�7g���<��*m
^��%�����Bo����u]�m$D
&y�����X�c���1�=*x�qs����\�
|pj����N�?�Gl	I�n=1��F-E�IPg���5��'�v��9)�k2��������v�[Ax��6�lYN%��I�>\.�d����������-��lW?��~k��������P|&	�Nr����&������9g�vmA[����v
����]
O�]�((u�V���=���'�;�
�g�|�g��_��M�=|^�a�2��b�������HP���;�,����CY_9��^Q5I{m}�Y\�u���Z���V����Z
�Z���H`k���m��cQ��z���mlL��*:��`�G��Zg����l��Z6� ^�kYk�H���n�������f��v��Oi�n�����6S�li��HRw[Gv�r�|����mK[Z���:������u�V��[�: ��$J�Q�s�|�P�Z���������K����~v���6&m]���Z2��B��M,I�G��M(i�����0��u�����
��Md�|�(Oh���n�f�6f�����/�J��n�B�K����!�%tb���A�mu��uB���u�%vbI��x��X��n'^r'�:�������u���~�� ��2���4zi?�~�?���.�G�����@,�#����\E'�#��z@ u��6��$K�Gr ���K��G~&����0�����$����������^��2I��
Gp$�^���� �#�A+�d=�8����]G����I`&��'�����
�'�K�)6�8�'��S N)�S���Az��#i�-HSp���8�(�@x��(����"��$K <�7 <��V���)=������@�b�Syq� �8��"�8%K N�7 N��QD���K���X/\NM�^KS`��	d'����N ;)�N���A�+��,*�N ;�nL8�*:A|U�A�R�W�&�d	����U��u@�%	���d������ �*�W�+	�U!���u���%RGZ��a�G~��K�$�#8h�����v?���V���N��A/e'8��*:��� ����	�'Y?����\Z?�#0I��G��������%����%��Gp|�;&��n�<�-��l�X���r���&9"�IQ�YN'��a�a�]����n�I�
i��}��2&3T��Z?�me�D��r�M��:6u��S%{e�""��D$�AD]�AD�����"r���\� "7?3#Y#�j����j�&���7Z�	�n@+��U
�[c)M���sh�x.�b�?�zk[�J���5��Vh�k��;Bk��F-�Qk9�����|�v[VL-[Ot�m`U��u�{���ZaT��{C��V�-#�����#:�����1E���Z�e��������p���R�w@K&	]��*t�;�N�
U��h����v#��p�;��B�J������\���B��F���i���a`�0���
4��l�����������tl�m�J|nA}
�S�*�>�)�O���&�@}
�SP�:t��<`P��$P������>�)�O[}z(J(����������[��q����&P��	���<��)(OM+���
`�<�)(OAy
�SP������<��^(O��
t�{�;����m��1e@w������U�I�Ge�NE�j�US	���A��(
zT-(���������G��
zT������G=*�Q�U����Q�G�[=�d3u�o������������4���jV	t����)�NeAw�0����t��;�)�NAw
�S����t/t�oZ��m�N��������"[���y~�w|��.�@�;���	J��Y��%^mf�n,��iv��fkt�<���
�����;y��\�g�y��'IY\d���h���'h��V�<�|���w?^_Ln.��B���������_&�7n>��o����_/�h:�^
��%e���s��?^��M.��DQ���W��(�c�n>��x4VB�� ),?���Jx��L
3�G��������:p~V7C��Q���#D�<&�do��u��%���q����!.G�HZ� {r5�����]\�o�\�Q,�_�\~�����%�l�\�|���T�%�Ha�t�~��������G�5�5��^A"�Oo*nM�+(���OuG�-H�������C�I;F�����[9
[����1wXj��tH�'}�������R��8:��f�I�`�Kp��_��}������]��q,0�g��2�,��2�+��2�*��2�'�X2�$�%#��R4O6EP�}�m�0���'��f��U��K��5���<���b�6�p�i�V#l��m_'k��V>[�5��M-��F�C�~���y��h�����f�X��|5�S;������XBI|	%�>��
��5{��gmU�1����`I�+�Yc�5f�<R��Kq��=��qUH	ZeY��>��f��MD���-L�e�,\�+A2��-���W�D 6�QP�b����p�W������(d�Q>�]���s�~=%�lA�Q*
6��@2��+y4&�j4���r�9�a��UXCJ���%6�AJ���Nff�73z4�7k�����H�37Y��/��B�����������T����r��l��x��^�sQ�y��w	�n�q��x�vG��������#�-������C�;J�M^�R�g��LusR�\`����y�e�Z���v��]8�)K���x
�X�3������+NFt��S�=O�
d2�l��d5��hXi�)
S���( %s-�S��zUV���K*�%���6���d����qN��,�-�oF{#��~%x0�������+��2�z]$���s�V�������y��j�����R��9%L����?:j��J3R�'�_��_b6�j���������H?��UoQ��d��,��p�%��}�%�[������=�f�eH��[N�$�(�^���S������N���S����X@�?r�7�s��[N��{�����{��>.����sR4���I��I�����Z����e����hA�sAQ��1����l��4�,~��m������&���,�1�Dq/fz����$:��D�B���������d�D��(-���E�/���O����H�mUkGW0�F)�������t��]Rr-~.}�����,"�l�k�K��&bS3���������SSt1Q�]:5E�I�����jdp�0?�0vg����m1aZqG�b�PP���1����a�U��hz@��A49�����1����3Qi���x��wnR-�gO�GM�"s,_��r,_�}+'���-oX����gp����6���,��s�3��J��M,�u��������1g?Yl(gOu����*y�7I�|���"�����?��i�K������t��n>���xj}�
�|����]s�����C�
o���0�mp�
y��w*`�}����7'�p9���������k^�F��[z~�n\I�>E��3g�C������ v�
���������Fh�	���l�!��$^o2I%%Y��b>�FU�.B�z�����w��6N����pl�s�J���].�/R-{[��49{�4�>[���m��Ix��J{��zi*/G���ot����q{�Ey3������*��%��\����f�����������@���V����w�O�/���L&:H�?0v�����S�B*I9^@A��y���`R����2J��")��A���O&.Qx����/l��?Ep
�n�2�
�[��HW�����0�m�b��������GV��?��,�tc�:��$���r��lvp����ne4J��n��nu
v���7H�dtxn�dY�� ��@�K���
��2,��H;�b�8"�Rb��r4U�:qu���o����_1eHi~;WJH���Ek�BZQ��)���@&�:FIwS+�K�x�+h���K��q���J�N�+����l~l�i��4n�o��\ZF�z6�n����lZF��FMO��%����Z�*�����Zd����_��V.�+�3Z��leM����1�� (C }��X`1��CW�
�]�#�0v�a�Q`���0�6������8���=%�b'��x60��C���^��'�;'G�a4�]��;D����5l����������v�����}�Q3*��KO�����'M���?[�nO-rI�C��@z�gbw4X��f�Sv���IO�0*u7{����6[����$g%In���4X�(���Q>��&\��.��� h!^��IgN����P���M�0��v���������n���vc��N�?������^���\Xil��6f����3By��P���+a1�����
����2��d
�b2��W�c;��9���)�d1��]x(���:�xWA]�+�k�li�4f��4e-�\���H��6�����h�k��(�L�H"����NY,�,�
_��s��M�!�������Yh�@-�,	��C�DA�����~1��
�=� (��m�rO�kP���	���"P������o4�^�@��O��C�����w���^%�I��w�wP��r�
��z�1Q�@�W��h�@�W�@��j�G��R"P��������X�6����X�8���99c'x'���)���
`.d��B7N`.vO����	w�A,�]�b�z
,�n��b��������F���'�/���B�k�-t�|s�E����- rR����p�b{��v��Ui��U[W�������p��(��v|�����$P�1A��o�{�^�rO7NP����ro�DN�c3@��=P��=��U�����I:X�^���@��V�{���c$������"����5</W	�<��0@�5)�.��v���k��������0o+�1CE[����me7�F-�E�����`#;X�(#AE���oa�p*���22WQ�i|���f����������u�Wm�u24�����_�P��B;����B��_-�%nq�6�,����@��t���>���b2�W��x3K���^�,���:C�/��-a\��F��dG�I�����L.����K
�8��U{��[R���H� ��wR5�IT>��-(�������N�in�Gb_��n��A���H u����H��]���C uK��������}(R�����i�Hb!�Zec�@��dn��
��k�
d��H@�f�� st	dn����������&������}�! r7�
��G��T����� x�������@��[�%�
�
��Q�@��T��1�����E'���r{1���6V��o�u��
"����]���; rK�����@��O r[tD�vD�7-4�A���������_./�������y��?�2����>y�_���63%���|7<�����'x:�nV1Z���Q_~G�5zY����<A_���.�|�_Z���|@�d���{������w?^_Ln.�yC����9����_&�7n>���������x4~/�A�����9�����&��^|�����W��(�c�n>��x4VB�� ),?���Jx��L
����������0�:�N(�W�.�o0���Y��F�������,�U�In��j��$��E��C\����<A��jr��1Iu����b�$��-�7��|H!�l_���L��/��O�'�^��k�������9��+�A��']���:bhjA�TT5������\�v����%x;�Zd��,�IU��*�Q��j��qR�_��(�5k��n�@@�+E.�;7�����%����3�rnS�b��J93�s�I���M�1L9B_f�'�\-��1.����d�,��:Jo�}���Vl$�F�6� j���/����Tc�{�hW������s�f�X���|5�9iz���������q�f�r�ym=_w��|�\�|�o��>E�
�����i�|�C�"F�x�"�5&Yc6�#����L��*W���P���#�m�L�DD�����X���e�$�L���|�L�c�0�0
�R�s��
!~�`1[1
�~��t,���V���X���cHV�7Vr$-�nJ������)_� �r�s�h���j�����"��*��
Tjv��l���/��K^�k���B��26fu7_�Ex)�Q�W~^���*�7�/-���/�+=�H"����Ho�9��H�E���\$��C�e�����8���:��
��y|�<-��8�s�Z�s�������#�-������C����W���T���>S��[��f&���D^f�������h��r�� �d<-f���';B��f���b�d��|?���8�@&��x���nHVc08����z�*���Hi~��TD��
�����{�usn62=��t��n-��m�i�M�@<P��}�����g�����+������&|��,����"�/��a�|NI�l�Xx54 ���1�S�Gd�AS����Bg�'2�R�H�Z..��Q��t�t�:q�Of���30�V>"��R��x���&�/f�q�/����-��������'��`1���T�Q��|#l���%�cG�72`-���%1���<�)�0e��3���$�k�{A�'�����f���?��u��ty����������ti6���4 U(v.(J�>��[����������+���Aru��Fm�DVn��2=YIOR�rCv���2e%�0e�#Y?�lD2J[��:+�0^�1�������������`*7�R�
1�#�g�B����Z�\�Ps!�����EN��d�M�bK����Tqn�*�3��-�'���d��h��B�N����������v��7�=�\._4��jJ2�jr2����1.2����M,'��R$*�O�N��P^@j:���D{lA�H�9��Y#nz�#����'�<7��S/���T�H����TP��� ��,�VI��R�E�T2�|��L*���J�(2��!�V
�m�Jo���!��d���5&�
��&��'�������a�q��P��Z`���U��Y����9����??�f��2���qWo��?>���z
�Bc��#����b��2�g��8�������h?��H}m�s��������0�mR���������#��-��q�
�C��] +��y�[�C���N�w�J�����-��#Bl<7=0'���6�rrsq=���-�|���(����0O��&�PS�+��>�El�"�����b{�~�����5����4!0�J�����\|���R�(�:Q`���1���#�| �����:J�����jH�>2�1nh�Y�F+�H��5������o����:������;9�r_I����_�'��L�s2D�Y>+���Qa��D%�x!��3�I�
�i�r*����c��/���?�HK��:x�c���@\��� :��H���/������������M�QxA<j���}e�8���3�!��\D�Eu�"Z-���rzq�:��$:��r��lvv�����o�(�c8��c8��5��c�� ��q����B,��Xf�N^�cK*��`�\�*�N[�#�,e��7G�%�XW�����Zp:���
#� ��NSH3��-4P��Gz3�Y�M�"~-n
���2y\#�����.gp��&����5��j�����c|���^��������/���-�X���q�!E�w7SuJ7�`��>�N���0��8xW5�r�;0e���*0��4�K��K���N�>�f@7���
l��}f:���������!���7��A�����Q\��vsX�����������3�Z�6�����D���f��a>�3��p������d
��pj��{�����F�\^�I�3��>E�i��Q>�sv��|���K��9�z��V��-uq����l'��S��K}R�6f����&�	\�e(������l����U����$w�1��[��b���"��j�d2�����]�v@{I��Z�t�e~�?!!�UC)n>oU�H����(���p?P�#���'�_�2�����X��a�������jc���?#��~�9�5���`�B�Q�gO%i����@�+h|��>Z�v�K��k�iw�hk�������#�4��T�fK[��]�5���t���c���������_�*�\��t[�P:��)����F������c)k�$J��E){���!pH���a$�o�u"�,bA*�/P+�H��j�P+6���,_�V�����!Hb�T
�E7it�$�[����N&����=0���B6@�N`7��z�4�X�@�@z`E�@z�J=0��@���O�=0awz`^6��g�:�"��E0(�w��!�M�Nx_!j�!�`zE<��	?Nb�VX`��
�^`'����0�CA���
�nqza�Q����N8���p?8���
�v�	@�:�����18?������������#	��9l8?FRG��z����������cN�������=0���B6@�N���1�=0�������E���$�XY	���-���.��=0�F||�;FS�n�<�-��l�X���r�����D^������3W�.`��BZ+C�a�V�zCE[��RQ�VH�D-��biw��^8��fG�6;jS��h�#;mvd�����f���i���w}E�(���e��U[d����P�4���cezYE����Z��%�"4,l�k��ljY�T���v�Vo��2h��0i,���4������R�r��+���
��e���N��]t�I����:���A�������	��e�m�H��  ��hVDA@�� A@d����A�����|��k�v�$n���f	�C�)���*�� �@<�����!�����o���p��n�G�)����Q1�1l%�>�� #��2"dD�AF�K #����&��q� #��E����]k��~��`A��C�A:��9H�Y  �t�'�������t����_���/��|~_������v�H�mr�<���
z�W�	����=z�?'�1~Y�����c�h�/���-���r��M�	����L�$�G�e�}�m�P��+�����/&7���A�[����I����2���p��N���'W�����{)B�
�����F�\�z��~��jD����h0��=3�D
�@&�U�|3x9�f4������O�7���Y�O�F�������,�U�In��j����Qx~>������'4�=��\��_�&���Y��(�M�/_&�}�|�2I!Fd"����������BT-�f(��������7����{Z]� E�)#>CWo�_.n�~qq�eF�#�
	WF�e���9"���O�u�X�b�:Y������LuJ%WR+�!c��N=��v,%K)EU<�J Z��Vrp�
����|��p�q�L��w��I8@�'��������rr��g�0�'�|�d�lD��/O�yr���U�����]R����
�u��@��}�V	o�#hN��ar4���7@S��I����B����+�Yc�5f�<R��Kq��=��qUH	ZeY��>��f��M�
��ha,Xd�2^	�y&o3�����l����,�<�oq�����!��Q���|&�����
���U��1�P�!��r�9>gtZ����fJ�2�>�"h3�3�4�84k������H40����.���r�ti �c���\t|*�E�\9�@Z6��=Kz�N�N��9��:�����<�K0�~��?���;�dep�/O�G�[n/�MA��".�^�R�g��Lu�@��`�M47�e�Aj�{��y�t�m#�M1 �)Pk�R�';BG���x���*���{^()��`@T~@x5��(��e\�����q"�-�SQWx*�
5�d�.�.e*Kb��V����b���)K���Ql�V���e��s�5�9���
�e��H�K��=���)��-�*���/v�E���Rn!�,O����H�u�F�������W������V�q�:��T<
����1���TD����bJ��v��M>��]��\2,X
��1�� =�y�"#%��r�<�FI=������O{�8;�x��Z���r�.x�}$���M��M��]��w��+��:!�Ez���\��ez����d�b��o�e�JJa�ZG�~"��d$�zq]V2aV�@������z���*��`*V�R�1�#�g�B���$�~.���k���<�'s�3}>M��z������,&
8Y|OFgv�$(��Z
���Z�K�h%-NG\y��M�tD�h~z�QJ<�I����G����s�zL*���1X��7�X�����8�&OG%>.7���K����1������Hs�CT�$>g�.��'j-����
���j�w���$1�9Yl(�I�����*y�7I�w���"�����?��i�K����q����7�^OA{4��]0��|�xV��c�X��p-qb���:�l����z��F�=�p�����__����y�*S�s�������������x1�Ok�M��>���\Nn.�'��F�O�7e��C��I��d�q��a�|���8�X�]���Q����p;Z�J#�S���x}��7�=�\._�%�+��6�U��<�K
�����m���d(p�:�����yhp�,�G�Q|����,��%��\����f1�����)����@���V����w�O�/���L�l�9����mv���lW��r<K_����������e*��f\R��?��L���P<�1T�?^<)��o�qGu��)�Q�zT������������^�j�+J|Y9N�*�������x��cr�n9}6s�/d��������,�
�f��4����%��tI�KpKUk:F}�A������Y�)q��%1<�����{u���o����{d�\%�sb�4F�f(�
To �B$�#���:�e��k5�����g��n������-��I�C���hw���`��>�U��wHV
�[�Q,h�(&W9Z`}cL�"�t
8�N-�\�8]5p�rr�B�U��Lh�D�@gwBg�o����
�+��@Vw@V;��@WwBW����&Y��PK�_k���
��GH�l�3��y��$9�J[C���e2w�i��������n����s�z���\�l�,�m.���2os��vI��.F�����=�'^@iR5=�w��~{��D�5%��J��o���z����(��%ZV���e-+��A��U}�>B�73�!M�t6��V���o�������,�L�JNS��h���)Av��/PK�%GMy�}����Av��/P�yH��$'G+jx�_�8^�JT��dg�	�]�J���]�JT��O@rwIr���*P	����n�7�	���A9,J������(���%hj��&m��2����Y�i��
�+��@VwGV��}xt��Z�A~���*������R:���r����:�����
�+��@UQ���`�����o��
.���
�l}��K��w;,�d�K����h�JT��dg�	�N%�v��������]�����A%�VU���c�������"����5</W	�<��/@�5)��;���Y��BZ+�����mE�VHKEy[!���S�����<�yw�9f���"lS�!^���B;�Fh���Rl�V[��nY��/Q$���z�S�yc�����������Y�|.2����HV�t�����P�-�u�Vw�3�4�?����%u����Z��d�@[�d/�v�R8�]G-�9�������x0�)(qw"�89�m�
��n���t�
H7 ��t�
�@�9h��P�Sn��jA�9�]��f	d�m�)���*�6 ��'�m@����l��e�7���h�?v��#:��h����;���<;;��=��S������s�
8c'nxR��,�k������m�(dz[0
Y����a��� ����_����� ����O���]��f����H ��8b�`��D�� ��h�_W���9}���O�A7��\^d��}5�O����e"@���	������Kx�?'�1~Y�����c��<�����l>O���_�>y�_���Z>�$�e0�d���%w�e�N��h�F�%��\�g�9����8A��<.~�L��0��p���
E�wG��x}1����
b����������/���7�n�[�zr���A<���� �����}���k��������=���U�<�����h0���=3�D
���#���o�������uj`f0#3�a����f(��������7����O�7a`����F����m�1Y$�x�����,YH��:9?�r!��S���[3t�������WiD�M�k<*�e����h&W����������m6�t����o:������N
9"��m;���|9}���B�wo
3�/*������t3��v�|���m�����������H�������QT��9���
�,C���
�2b��������^5U��rU;���z �ihU ae�J
�Vlr�����L�D�f
�������4��K����R��F"i(��L!$���\d��	$��,{�������<y����&��#��~yZ��c\���d�,��:LO����P7�
������S��lo���o��h�bEb�4	����!P�=��_���1���R^^�{��I��BJ�
(�����6{&oR�+G`�"��J��3y�	kl�L�b�2@�|N���_!�/Cl�9(`��t,���V��56+�
�s�>a+�s�>�Rt^��f�`���L���t��L7)n#\��-��%�5F���E3��3�75�7k�����+�^�c����%�7q��-�}������Oe�H�+�H�F��'y��p���u>%��y|�`i�g�Wkw�4����zy�<B�r��.?}�[`��X�yu��Ju���3�
��{�af2�X��Y����f����g���T�����U����Ee���b�d��|?�����@<������nHVc08�����k���aY%Eo�����S�Wx*R���W�%K�%7�%#�%a�>�Y�G��S��|J��d<
{��WB�?����9BM0rF�W����utxX-�S�=[<�U2��E����?�����[����AmDE�2'Ow����T�f�l��K����Y�
ub��.5��l�fKGqE���V�����`h��U58��-��� ��:��93l^����8H�x:�'�2�X�-��k�T0������i�!�"S�*���P�>��7�����@?H�6��6Q�6��v~������"��QN�ri_���Jz����
�A�)+)�)k���d#����yY�P�9�BN���<����b�VU�tS�n������>K��%%3��"��9��5��Ud�zf��Y������k:~�Y�����5m:��������o��F�S��
���Rj�;��D��u��N4��:���1���6QU?a�(�,�i����[T�!S*PsS�����-���wA`�.���uK`��	��5���&0^gf�����3����3�|?>���x��h���)
�P��1�3�����������W��@���)(����4�g�'Ad{&���_�����;q�Nz�=�ND��/7Zx���.��U���@j�k����,E�����^tz����ON���#�l�k�@B��}�M	�L����\\O.9"�&��o<�M��?��x��d������;c��R1D�5�V\���v���Ff�&&U�U��$��r�|A��W��jh�UC�8P,���$�*������K����!��$�����l�L\���b���Ik�S��S��g [������no�+����"�}%�#�~���g2�N��H\fq��*?G��b���������'�=*Lu��/��U�x�r�pZ�Y�dB,����������"p�g���x3� ����^����"h��?����*V��x��X�7��q�|�g�C~I������E�ZT�?^iP�������w��W9`�x���@Tw�������B�w�{}�#�;��~G�m�������?�����A,;�X�%`���G��`��Y�8�'q��%�<��H�����������N����A>� Ric�eF�� TZ�������F��m��@#^�t�P��w���B����@�wr�������#l;Wa�} ��V�i�2�u{
$��a�@E����+����7Y[-����7��wR ;dg(��6��)f�����96L���g��K�<Q?y">��f�@G��Cy%`��h�!��w�uJ��'?$^&�6�!�SL_	��.����]2E]�G���y"� ��e���9�J[C��BL��lm����	2D��F�1]�������35'�,l�l�c�6��L��i��.�:��+[�����u���+[�����u��y/�:���94M��z&�����L�&����J.�����9�U�(.��c�&�	(N@q�]��8�m��I�(#(Nz�
�]c��M���c$�k6{dY�#�f�=�Z�{����X���3��#	����d�v8�(;|�ZMW��P��Z�nd�s��\�$ZA&�R������!H"��j8Ui��,���f4{��V*��E����6�C�����f��z��fOW4{����=��a`�l�!{�w�q���f�����=^��k�:�������w�W�����w�m�����t{�c�����]�=P�SL0E�n�"��w���Wx�����@G��C�J�u�0�C�
����7��]0�SL0E�n�"��w����x"��s��[�:��'pC�
�����a��ln��en�����u"����pB�#��G]4��m�;d���O'�� a�O:}����=���Z��k
�j��YW��z{������=]q��{�V�����=�mx��Q���@��=�v5{���cR������"����5</W	�<a ��m��u�����A�VHkeLa����������
i�(o+�P�VXl6Y\M��vo(7e�-��������O��Ikx���
������
���Ok����1�j����B82���L�*�&���U�d����(���
�*������e��e�!jD������c����B~[#��F�����U0�=ljAP�u�rWU�;	��
~[��[�
[�Z������Vn�%J ��d/-�v�Ra���Z*8v�QK���:j�@1�� <oA����10r���\n��)�mI�� ���oV�}�A�w�>�� �����>�� ��A�?i����}�;����$0���If	�}�%@�a�}=��� �[a�A7A�o�� ��"��iad�����/����-���^���V���/�8H��UA��$~��[	��2H� ���Q������?�$��%�-^�-���B�,����}��A��+��od}��A�Y_
d}��A�oW�����������_���/���_������v�H�W6��O�����$
�3���������_�'�J�i`���K�����2['itB�Xn��r��M�����~����\ps�1��m�����!�er}�����xo]O��z1�G���R�V�>��A��x��6�����6?�J�G�Ct�
��G��J���O�'�^��J�>]\���	��^f8c6J�C@~�h�E��7��:Y����8�G�q1�i�����AO�&����]��m&p:�e�����&��H_���<�G6��J����	��&w�D��'0'0����`��tQ�U�����jTa���^�U��r�Q��2�r��Oj�D+&x��*-����q=Z����r+���8#�������9���c{:w����-_7��9���/O�yr��ar�����]R���:[���@�������x����w��7@a�1d$������r.�?��~Ei��G�?L)r�R*�J�T���I)�+��R>W�#�<��GJy\�(�)�@Y~��t��3Y����V�T��X�`h���!��r�9*b��v,+���*2��:�"925h����pC�Q",��T��d��O�J�>�9�_-���y��PN9���:��<��<�K0s��?���;dep�/O�G�[n1���:�C����W���T���>S�@iU3�&����Y��n����wa�������_v��';B��0x�8,�O��<R,��� �����!Y���4Vh�U�W�����^��K�l��O�0����R��`���9���HI���^�}��\����9E���c��l���	�1�SJT2���9�5�Yed�6���TU��l��pT=k��Kyv���l�����|�jp�E[TV5��jUY�q/Na�u1���/�(qs����l�i�Y�z���Ze�r���x7�9k�eC �~��m"�������M�Aj^#7��W���c��]d���T ���R�X�KyB)�v�,��H��*�O����H��mU�A':�wG)���EL�C��b�����f5����G�d�z���|����������GE����k��,���E>�/���a:�B?�x�b\n�cP�Q����c��������J��o���o��j��_`�8�0����y�rY�����=��sF��z��=�2?k4H;�l?eL�'�
%�T�?>���x��(i������P��1�3�����sR���>���| p�9�hL���1��3��o2�=#��������7���6�(*n�&	7]>|�+Pn<��-@����O]u�����8��� 8?�����	�����n^�sS��K)����������kh�	�����?�a���M����/V������v��=D�5#����p;ZQ��*��j�@
�J���].�/���iU��Ap��_,�����*BR�� $�����D�����_)&:����2i�@tQ�7��#�����9OAj}%������T:�UO�k�ZN����������B=,���D�RV��E�H�.IV���������8�JQX�V����R�RH<����('�����y<�'�n9}6�C+X��k&���� �z�v����)m�r��2)a�C���w�3����,�e���<���>�-u�pr��������m{��J	�0+n��y,��n1��S�wF����dN0�b�������8�h�	%��qx_�����3�:�b��o��D�g7A��?Tm]P�������bj���oU?
H������qK����&�~Pu]T�hg���
7������aU�:9��������-;�{����{+����bSe�nsd2���9��
;E�u�W�n�:���!}�X��])�p�qJ�G�qG+��@<���������_��{�E;�g�g��txl�
H��0P��?�����z�?��8SVcp���u�@]�W����<u-OY�C�}����k�����O8�vi�)���w�@��k��CL��i�����vO�8@���)����a���[�M����K����U��%�oS��i@��b�n���9T,�vu��O�	�����{�v������AY�I����N������!���oW9
1b�
H��08��=u�q�J���h����@�z�pT�x��{8�H��!a 8���cz+���y:[���r�F���*A�'�Yz���a7�T���BZ+�
a�VFBE[��RQ�VH�D��X6�]ER�E��d�)���M�)���QWb��lXm������n�!��fRd�t�&~�B�]	���j��@YE�������u���U0�h6��#��e���N<�-@�k8����Z����@���d���v���5�� �v��P���kG�x����H�W�&�l�����&ZU,���lb���aC��%n/h���t��$�L"0��$�&�A7��$�e.��<�Yx��=*�Gl�wTL{�S[��N��"p��)���b�n��S|��H�9�-F�]oA����D��	x����#�#�-s!]��?]��L�ry����j������D:@�G/�j3#�q�{4����/��|����e����)^3��;�Wh��A/s�������.7O�)�����!~�o�n��w�?��
.��k<����d�����;��Q.����=,W(���]���\}���#�������K�/z~i�n����O����'�r�\�g��
�����v�J��l��6OI��'\?���
����%��Y�i��f�O���Z,�pZ
�����
w1��c*W��3��2���C���
wO�u�x���1��lA��N�n����?�����brs��[2��������������)^
�/���7�w�?���nw��% n_<���������o��_/>!"]}3��oF���?�7
h��������
Z ��������7����O�7��G�*�����?��/������<{�}��g����!����"Y���v��f�z0�h<��_�jdw�=��1���jr��'�II��������w]_�.Rh&�����W����u��*�Z��f5{`b�����|�&�����%�oH��+J��������]|K��t���j�|�|�U�����h��IW������U��U�$l���R�k����_�4^Q�BY��}L��}�2.��W��Z��f�+�� ���i^^�S1��7�?7x����E��@+��3gC��%N� ��j�Q	���qr���o�g2�~���a5|���4��o7~u
�V�����vd
D
��i����
�c�8��p����!�������ISR�x o�C�pB���z�������} y�~�} p���������	NO��+�����S>�7�c�@�.����1#�<y��������-�y��KT�2��$��.����'��~�(&�g��x�(��33Ar��b;��|>��������7�D�A*@"lI��u�	#������'��Z%�*Y�%u$dDt;��q�
�q��_�s{t
������Ld�n�~\vfe�i6Ov$H�+kB��/�9���)���sfE�$sfK�ts� Y���$_��|m�a�i��=�������� <�G�P
Im���<��	}Y>���^���!y�<�o����G����-�[�1L�2���3�>����#y��}����c:?��Q��y����3��|��*�7oG�a����/�<�rQ��?)��K�*��=���s[4�":��l�L�,f;OUR�Y�2f�n���h�t��NTXk��z�������s��s���r�����$RN����@s�^���(�+d��|����!���������kj�{�])�d���>�	q���)�~���'�Q�''A����zdK����<�K���{��9^������n>\�?���� +A�6�������C~_��}�4%K��b�g��7a|p���+)�!Q��;�iH�,�|��6�F����"��)R�'q�D����^��:4����f4�,fF4��Y
�j�I�7���Y4\��W��y�l�r�4�^����v��p$��r~��iKEkZ�V���,Q��%]"_�D~�%�
��w]�T�-���%
��(�,Q�x��%
\�Hu�����=�����x&�[b8�����������I���A����y��}��<�����4;���Z>l���d����3r`�U��2)c�m�cMg+� �\�f�/�)-�7!�M)�V����R��J���].�/x���b���X��6�L�����84;Y����6@���:D+�����6�B\��J$��/<
��P�1P���O��O���N������
��$�]$���D�^!Q�}$�����#Q��n�?$���x�i�/],7�����	����M-�������7��+�Qo�����;n�����-c�0���T;b�����v��a*r!�c���
�j�U���x��%1F��������e�JMu��Oi�,By���~���F��o�)m#��GiH RpH^J�Ycv�������z�X���w�'��p���G�Y�06�k\T�������q�<��f�������,��[\^L>��,:q�Zchd��:�Ak������nt���x�r,���G�3SfDQ�II�h����xU]�(G���_��������4�A���%���u��g�#��CP3����uJ��7bF:DO�7L�����?e�[�w<Ye���y57��z��;����a�Q���2�0��f���l�.�7c~h�Y�=Fuf���SU�-^������[�_��=��s�H�{�?��q!��5b���������G��u�]�CV����]��:9W}eMU�_�[��}K��58�h%_F]v����j/�n�0�QI�a
?��7�y��{{��b� �m0�f�A��IL��?���������i�����I��d�������#T8:1��j#�~	������	�4�I�l,�}����_�� kH2;���+��h���j���Uv+����p��|��n���Y���m��Q��X!.�t5.k��8���H������s�������7��l�[�&`����1�[������I�7C#���f����oy3���xo��eR�u ��p9_���\���-�������������|"g��w'��I�{������#�?��yb�T���s��nxz���������l�D�Gc`�_H��yb�T��#O��������i�������*�i���)���'fK�{�xc)�P�{:�e�e�xr�A���lWH��2-Y<`�O���d�e�
�xr��	�N�]!��h;@�A ���R��
�z�H=� ���s �H= ���3~SR�������!.W
D/��y����7����\������7�)�s�]����MMf����C���i��phX�PN�#5Fg�j���� �^�*x�^20� `6L����Wu�3,��b=����E<6��x��j�_9���e�F�]
�	A9mwX��)7�c��u��������5c��v��������
�[_�������7�T����c5f�u�[�����cK��XXe�j�
�����Ca���}S����.������D�J� ����7M{D9bdT��E����iT2�BQ/���c��������b��3���dA�*/�����9#v��eE}�/}F�������w�L^4*�� �@ +xg�y�K���K��.Ap��dNI](����}�.A�3�m��U���H~a���g�	�h�H�B����������g���w���D" �	i/��4V��x�����i�5��[<�n/0�|�����&�x���`.�� �i;��A0X���?kE���^,+�iE���V+��E���V+�iE���^�*�iE���V�*�iE���^�*�iE���V�*��E��X�+�����k���RM�����o��!��u�k~����N��'��:��[=x�ma�Y�5��,LS	������~kQ�p7����\'������o��z�Q)�~�];��z/x�]�bo�.��L�X*n@r�K������Lw���1���m�X�Lkd�=���Rr�d��sc���D��[���Z+��������'���M���M'�l�:�D�Zyno�e}��������L7����ub��$'���d�6}IVf�)��l�����/�
l5%yM[���%YiM_���%9Y��$#���d$5}IVP�������H�d�b�U�b�E�'FL=�����P����O�	�+K�{��(��ht�.�� ��R����2Nj���?�[�������J�$-�7Y��%��X�q���&���^������6��q/��J�X�S���������&C�X6�����X�nZ�;cX*��7WM���mg���eLvYx���g��_��5��lg|��%�|�����j��n;��h���R��v��T�s�e�.��Dc���l����vn2W�P��h'&�`������]�g���f��b�Y�����Y�S�bgf����|~�����w^W�f�*�L]��y������#����>>N�?���S�o����xjYJ�K���bK����3����������K�M7:���������jq}{�M��N�{�
���L�(Zs�Yv��m�c�
���VY�E��C�e3�U����g
UiX��;��&�&�'��F��gv_n������.R����QC�b]/�S�KG�,Ty
Y��l� ��g%�L�$�#�W��T�@�p!��i��RQ�$���R������!��E��U�Ee)��h"s��r�&�
)���Z��
l���1�~�d��x�'}�fp��H#d��Rn��=�;7������e��R^��=�;/���D�.	��?�������c�c�N�����rQ^����e�����b�06X�N��5Y��R����i�����j�W�)Y'�q�6KD`�xM@$w������s~��E�����x�^�>��M=x�*�
L>�]����5W�>�}Ko"��(b.'�80��H�J��%��I��T�2����m�D��r�"�2�F7X�v���2�������`�������
\�[!�x�9�X���zd������Oe�J>�&��S��RR��:�G��Pr.q,3K�FRC�����*)X��O��>��C%������$*)H�(<��=����Ya�ZX��_%c��Ma�1AS�����M�3�����=e�\Ud��<���c��D��@��H�	��@��4���-''dy�J_��e2D�$!/�T�,O��*��� ^>��:)�H��d_+oh{��6
���>
��Q@���/[5�l^�j���Ej$4�>����Jvm�(lP��������-@�*@�r���Q����d�ni���eT�W3
q��$��
�Q��TI�&H����u"0=)e�DOe~�����v�9�XN�,
s��1���x
A�'� �q'�(	.\���M��	s�4*b79N�A_�����	�	j��T��������9�w�������P�����������H�����D{����Hv��Hv��Hv��Hvp�HvD�Hv�Hv��q�����jc����p��Ih�6b��88N
PY��#�ItT=GPr������-�1j[
Z�3��rr�f��@�l��Zs����09��p�\14-X����#K����G����a%p��.��T�y���/D:�
�B���7_�����tE~u\������������������9Wc��&t
[?��O1��b��F,Q������k�Orv�`�x�	kLO����H�j����'�oJ8�w��3
��������]mH9��(�$��dWK�]'��&_��e1��o�j�@�v�M�-O~�M~��|�=���7�f��������|�i@���i�E��	�?��k�����
+ya%3�������V2�J�X�+yb%S����l��/�3���I���x�V�C�Jw��]���7���U�H�z������������C�i��A����"��?��x��� E�u2,p��"���G�����u�����nI�����m�����(���'��//&�nl���a%��w>��qAU���M�T�Od�Y_&Z]�u�����L����k�x�<��'{-L�5D���<c�%M��������
�b���������7�4^bc�)M��]/s����e�]r����4^fc}-M�X�H���=`�_�����5�	������
�I��O!z�?�0RDe��	��?GB�����A�s$Dz��	��?GBL���(B5��������PMo%��jz+�'T�[�H���J�
��VbW(��w-
�2�u/���(R��_T����<�4��7dI2U��7���������2J:e�����3*)���$w�UJ����Z��.!3</bI���[u���a���==7����<jGv��4�rMM
v[m<�����D��yK������_p��G{��N@���v���tG��`��H����;�Wc��u�qc�+��'����V�p[�A��<�F���J�JR#-��i���7X}W����^Qi����S�c�����^����F���K�r��E�Rf-*K
<���Qse�^����N�`g(�$�s��;q�X;_���v����\^�.������
m-�`ek���yp�l�
���.����U,%�V|�����_�������4[���i�-�>��6S��:�o7}Ipn��c)�����|��X�9
����o��j����>���ljbRKS�>s�z��~iz������o�7�\}�+��|YS�	�/��{YS�	y�-��z�/�^zYSRq�A(�]y�/��x�/�^x�-��wYS�1���dld���e�����
=p���������(�=�bbm��o�2�G��"�1�T���Zi&>:m�k���5��.�U���gl��O���k�[��f�2v;���,c����:���`����6?���vnq��(��T����7����s5t�������v����� ��v
L�m'���6��pz1?�$/��c�r��x��/*\G6f��qEYa^<��.03�����x���9�e�y�\_T�m���c��%i�%�Dz�d W�T�VM��P/�"��
n��%��z��!W����s�lQV�G�k~����b�Y���XhV,2+vbV�����Y�s}1��Au�j�Wzw��X��U���b5�V��4+&������	n�������p#4�
7B���p#4���vs#t���-���E�!��H�l�HV(^��99�|���NJ*8�u2O�6T	�j?RM��j�\�2
3������P���-��E_�rh�8*}G5�
K����{\�/�h������X7����<�s�z�A+��\�w�Z�_�1�'����sS^�Vq��^��K����d�9F^���p���������S��
zf�?��>E[�S��A:j�D���P�����yKsS�?;/Q
+�?
M�{�yj���C���p���z�
|�+�9��x�`8�0%�p���z�Wf������ s�Q�AH����)=>)3��~PfD)�����Y:[��U�*�5�*4D�*U �np��n��v����>'���jk@S��M��h��hjW��`H*#��,�Jt �y�j
��
 �������C%��	��k�6	��+W �
�q�2qmqUe��v�*�(*PT{@Qm�Em���g1���������8A:i��k`��`�������k!��b�����A�+x��u��c�
���Vh�0�([�h��V��
a^!sw���*���-�
���AQ�D��3���f���qrm����������cSCNE�
DL��L�TT;`�Q�os�@L��'���^:����M�O�s�\�.�wV9��3s����r�T��l������,#�[a�j�����
1^��������������8	Ob����z�M��~��9o���z��1 �Vd���p�_��fJw�Wq����Mq�_��v�W;8����jW���S/p����l�5lb|�80 8�!�a�80 8�*�a����]t���	���2�b�7.���Y�.�-�,�Km[v}n�g�_j���s�?��R+�]�[�Y���������2���e������F/�4����/�|�����e~����#.�,�ym�N�|������
������5}.������K��+;�����s�����	����|�@{��k��[V�9"�V�{n�4|����
_���������jo���{S�=n�v����I����\����V���|���*>tI��2C�fwe��:�{zk�����6���=�JS����K��'ir�W��8p�e�5�45��]���v+:L���S`�BW��Lo�@�'���j��NK��5g�����4��6P��',��������xt���T�
�*��IjI����I[����jIoJS�0iM*M}��N�U����������M)1;W�[����fPbv�����"�1����u	����'�qj����L�h0M�Lm����-{M��R��K�n��R���D�����^+w@k������k{cs�\/{���KM����76���FL<�X�Q���v�YW;�&��DKS�XF��d�_��A�-k/[B��^y��0�}n����iu,M}B���YM��������l�Vy�v���>^�N����b��>�S����4�E�����_���/���/��&B�/������e"�;�)�� LiV1z�W��
}y��=���'��|S����N^�,���t��[�bQR��O���'h��V�<�m���w?^_Ln.��C��������������2���p�g������fp7�^��e���)�����&��^|BD�
������O�~����(��sJ�R��>]\���7i��Fh������[2�y����������!����"Ya<~���<Y�#�1v�e��VCWo���G�#oL��\M.�Tl����������o8��l��@5�.H��t�~�����N��IX%_�	�u�oV��&��L��wu"�Xp;M?�D���[|2�n>�[�;�����7^_� 
�VS"��0`���U?�����Zd��Z()�fdmg���c���?���������6�&;�:���q�}���K��C�E�jd�������c���a5|���ir
�������?V0�JCJH-���+������S���Z ���3��Jz�v;hLz�Q�Lw�x��������CBiju�3^P��'��~�(&�g��x����13Ar���O�������w������	}yZ��c\���d�,��:�8"���� 2��j�o�':�'��~4<0��:�nZ9������~\���|P#���/�������D�p��qb�i��=�����}'|R,��G�Im���<��	}Y>����GoH^2��-���1L�2���3�>����#y��-}����c�c?��Q��y����3��|�py�|s�Sn���\�l7��_�x���*�� V%ppJ,?�����B��k��&3v�14@+q�������
����ky��W�|��e�75��8��^����P�l���f�CL4����A5/8� ��d~�F\�l`1e��Xlk���-^���y|�`I�g�Wk=������W��rdk?�JPCzy�<B���b����/��|��#'^��4�w�������HyR��5)�e�Rn#o�[����xM]�� �_��yM
l[�d�J�����j{w'S��05M�.`0H�h��B��V��o������2kK��h9�R�����K`�)�J�O4��1([��#����z�C �a�Ch2r�������(?M�� �P�:y��X�u2O�6%M�f����sA��<%+��5[o�h�U��2)}.4w�X���&-W�>L�%4�S����u�]��5�e��FW6y|\%��F���������;������
��)3�F�K�q��'s��-��;-p��,���,p�c,��8!M�`F=@�]"M�����c��$�����>��D���������r���H\o*�7C����	�p)"����un��tZ/XC_8�eq�d��~������	i��	7�5������W�ox3C����/B{ZV
������Sx���+0`2����H
O6r(7��z���\w7�'�zs��*_����������������T��>LG��X�����M0�m���(=��Fp"/����)I8x;����/���pOX���I�7d�B����_��x�$c���T�����b��f�f��3�R6A$�>�E���������k��o��">�5b��9��N��� M����\I*���4^[� D����-�o�S�)��v+&��x���k8�Mn�)���w����a�	����%����'��A&�g~��<+��2Q}�:������r��\��$5�1f��5�)5�������E�?g�����p���v���R�[Oi���,7v3tT�O��H�of�I��=��u�[��e*�x���1�����	��o��f�G�ru�%���\���|��pD�$��������$iN�����(�G��l�>:5���=��[j�z���_�9���}c����������Y��te��4u����;��p;6�h<��f��t�+�R��o=�:��t���q�x6��5&��y3��Z������@q�����U�-�e<=%^7�$��x���]4�������G'����^0S������l)W���*�y����?I�b���;O��r�~4Vf���)���'fK�[?�T��n��iI�]O�w��-����WN+��NG�<1[�e��K�����,{,������N�]!���}�x
���]!��H��!����S%����n�9�����snn���b"g��E)���1E'���Di���#S,�����"?���a��b�<o���\���o!�#\��M�z��31���P��6��by�j@��P/zNK��T��Z�
�3 ��B���2=5�YwHyOx;���Jk$�T5���v�m�2��S�]
�	����0��w{{���v�-������
?�va�j�
|�I��	����?���?$�c��h�f�������i�X���a�s�<+����s]����sadOx>s�^,�sY��������3,X�_V��XV�gxN������y��E��UCA��@V4���y
�H�589+�58eD�h�3�I�w@:�����a����>���y���1'�1��E�����d�������;�<V��;0�B��6�ye������
�����o�3�>0���c{���m�g-S[�2�e1-3[�3�e1-[�2�E1=�Z�2�e1-�Z�3�e1-�Z�2�E1=���!����l �]s�45�Si�.���~s���q����8��B�e�N���������b��sm�ys`����,=�}���)��`����;(������)a�YW���N��:Ye�s�/�RH��.������Z����/tiI3�X�.xOt���y=���.>�.��iM3���ZNxot�}�y������,���s-�W[��w�%Yv��$��jKr���$����dX]mI����d]mI���)���5%.W_�er�%����C���/N�\���/���|N��!��2�e�n�W�4}��Q4:q��������|r�J�M���\�0e�^��(7�n��	s�#�����.��x��&rR_tL��L�u$_�y�����1=��L��;h*��T�f��s�-4�g�
h������nj����L���ZS��o3��+�Z�&~���@���h�A|��`����N��*�����f�"�b'f�N����;���TW�fz+o]��y�|u�j�-/V3oy1a���E5��<�����������fH�\R���u�h������_�V2�R��
F��P7���P�#�!���8���n�	ro�;��K���\��S�&����X�|���X4�!$�^6�P��/\ ]8�J
7������%�%���b�f�*[0��n	Li��<�=��b��dpZ26�a��Pd�B�?����PfN��s�+e�PT��r`��H�%V���+�Vvbk1�P�C�r�~�dD������w�-���I���K�6�;��]l��������7��X��1^'k{�\�q������z�\�������yMV�B��j�}Z�9���Z�U|J�	z\��(^�F%zz�~}�#����
�������l���T�S�^9�E���$a��c�����qL1����k`i�����f���v���p`�k��]���0gN�5'5QF���Y���t�|J��'�2���`��G�pw9��t���T[-)x*�WK
�����Oe:kI���IY��Q�x������Ya�ZX�3�O�1����h�>�)|jS���pz_�a�D�d9Td��<>�|b��D|�pb��'V�X��������=�I��}����ou\�����Q��<��|x��x����8D�#��,���������+��;����x�� ��m0�B��W����|@Z����m��t�������\������V��|tj+������4^�>��>�e��F����{���/�#�R~U���G`��NC����b�db�|h8�a�)T�"u�D�k������'8E����O��v��p��H����d��G2��#�����\��S��;��r����d��G2O�#��������i����nc����p����S�m�-��,BAx���&R��x�;�V1�&��1�0F���R�@65W��D��������] ���%�������w���F
!_8J��*�|��9o���+���E~u\�T�W�_P5E~����E���D�Gm��d���c�{^<��.re������	`|�SH�L9wL}T"��������{y\�WWw>�nNX}�9_���ruVe+���H{�Vy���[�#�Q:��?*����Q2?J�G��(�%����,��R2AJ.H�)� 9#T9f:�1���h�<$�dq������z3[�mP�[�T�\Nn.�'�G�����*��Sk�'��_�K�0O���~��g�Fd��l$�"(��o@�%��YcTH o��w�%���2���.�S#�O�����O76�N���_�.N#H������`�\&������)�����k��/���oq=��nUN��M�^�#>�����Z����j�d:,�A���Kl��&�q.f��CL	*G�����?A���Sv�H�#9�$�)8������H8�$�9N3I��#N��#�����H8%�9NHI��cS�y���R�[���j�*��T�V9������,��Um)��w'�
F!�a+$���	:E$��c�/*��vz�Aj��$�E��Y��J�Z�B��)�(�{?��T�����D����
E��<R��L{f��s>�,Iq�sB��q�����Q�k���n�v�^j��[2��w	���qs�����?Qkm������R������d��+����y�i�)WN�M�)}�
�u)�����b��E�Q)�E�R9��o���Y��R��[���!��nR�Vn��[��	nji���CF	n�[���!�unc�[�$�D����C]4�YP���C��sp��:������J1�u�R��[���y�;���k�l�
���;�f�hx�w4�����y:[���K2g�E�B�������+�9����{����
�GSA4:V��bQ�)��h��1��#�F�� ���<%���(���B�wD�M7�ru��O�Ee�ZBmL�[���G�7x��d�HF�}���d�F��c<O#�iV���}��t�W�|��5'R�flk�w����.j�l�3Y���@qmj���R���������)��yK#����7����XG�t��_���T����{{�53�*�V���)�c�yXf�t���� S�Q�A�����)�;)3R8?(3�����LQg,I+��*��������W#������_����'`�L�5�Um
��/ _-���@���|u�5��z1�h�2�D�A(P2��%J��y ��=��xI�&��:f���fh��X�Hs_	K���`2/ ^@����������������h��L�����9"E]��x��t�>\]]\�daO��4��'	�Gu�V�-��B@+�y�@�BD+��B�W�
�3m��UA"�N/�]h���Q���q��*P#
�2:�*�����l�6g����_��3y���M�����F��i�����T������9o�����*�}��Ym80n�l��C���h�(I������8��F@�t�����']��hN�M������Z������ ��L���^��O&_�f�3�������2�����?��Rn��=(�,�KY"������/�������2��9��?��R��;(�,�K�$�%���������!�M�]l�J���.�����������@"i$���4�D�y� ����7&�4L2�����@"�Kc
%�:	����v ��@�����Q��V���N���~�[U�1�[Q���3���9����P��6=#
�L�������&F�w����5�k��AL/K�v��c�=^v��W��V�e�!������<�����E���6�y����� 6}�"�kC�o���o�v��X��^�<Ku	��\u2���o��)�P���U+���tV�-Y�4w�i�u�jg�3GI�U��N:jA.�
uu���-���<���0�-s���l+l������m���E����;��ek�A�PD���7`���*�m�� ���#����=�mA���(���o�t6����&gRM�
���9�nl]��v�������]����X�T�:���;������F��+;P T���s�}��K{]��6�uo��yu���O�A7��\����rn!�s�R��?�2����N�}�c������I�e�{7}�^>'c�U����}����Q(_~G�5zY����<!Z�������'h��V�<
m����4�b�����)���=,Wh/4K�/���^��/&7�"�A�f�� ������)^
�/���7���?���nw��% _W�����}���mr�����~�o���fD~|�|����A)������f�N)�3�s�B�p������H�ew6B3���1�Y����g��o��������E��L���-�z0�k���:�a���x�vd<���������O�wS��|�|�������M
�d�|��������_/o�u�w�.�t�%V+C/��n��o�D�q�v�?�D���!�}b��#�C4��p�]|h)��b�K����w_R_��
�f�tI	D�G��U����+M9e�,:,�\\C[�U����@��L���&zd�=����-C�K����J������$\���������k:x������������)�Z�[?�� ���NS����2�C�2�V0UJ�j��;��~jr)u�P�H�b�Z�T�j)S-Zr��9���tn�K���x�����.P�xxH�^�$uF���������������� f&H��\l����9�Pv>\�F��B7HH�-�(C������	}yZ��c\���d�,��:2"
��Qd����7����:1��5t6w&2e7�L�F� �yr�{nD�t���k5�1�u�y����d�l��n�$���������-"4<M_�G��Vq�w������!��<���!}:�/�G��+�x>$/�����}�������t�>��[�1}{���S��|$o1���~��yL���1J�2�t�c�xF���]+/h.��-����K���(������%e��r%�)1?��hF����d�d1�yrq����-;[��n��1
f�Fb������"e�������1�m-��,'�rR�u���(V�O��bU]�,V��o��0dsP��3j�l��5���=Y��|2��wm����j���i?T��������C�n
]=��G�j�e�%O�9��s�Z���O>�|���; [�AV��m����#�m�-���`K�Bi,JJ�r��a���.�t?��)O�}�&������by#��t�,O����Ss���b�?�IA��if4�,fF4�5��C5�����E{�,�a�+���Z�	��(���zy s3���:�u�lD���r|�-�hY[]"_�D���t�|�����7\"�u�:PQj��KH���\�t����(0\��u�T�h�	�#�E80���L��pH7��1���/9�4�O���5���L'��W�"���=�.����&���7k4��xE���,t�����ly�[^�Mi��	�oJ���mN���-{���l)���%\k���IU#�qhv���5�m�RA�u�V����m������/y������!sX��YO��N�9N*�9!Q�`�=@��E�~�H����G�^����>������A��/����Rr�x�:�:!~�������7��a���q�X��'����;n�����-c�0���T;b�����v��a*r������
�j�U���x����s�������zrY�Ri��Oie�'M��,:~h@ �&��6�/y�FJ������}��-���z�X���w�'��p���G�Y�06�k\T�������q�<��f�������,��b\^L>��,:q���0���!M�����F�X|���F�e"E�'%m����kjU��Uu����u�������"{���j�l�w������(������j����������
oj?%E�O�n���OV�*g^����^+�Nc~hfXz�e:��#����'�'[�������y��g�_��6�D�|5���m*��Cjw���8���Y�2=sg���(�b�"�$e�s2����.T�A�R�;�P�������w���:3j��������b���m�������Y{`��S���/�`F�V��JR��Wr�N��
.\�<�.�����*g��,����'���2?������x�-6�x�~(���"�"l���������[��"�=��}q0��>}��v4�B�
�gz�A����@�T�3�d3��A����jk���������������;����a%���M
�dU9�x_���=�E�1����<cx��Zt%�Qrg*��>,����(n~��y����e�*W��2p���Bh]�q�]�_���:�!�/����� _����Z��3�4U�G�:�3,��-��r�2P����k�dtz2��{IN&g��8���Y�G��?��'�Z�������b��hz�L��L��X-�J_�xh����$E�'�Z����ql��u��9MG'�Z���qb�����*��j�V���7�V<��N����N����8���
M�P'���3T�Y���N*����8����
M��D�RO�� ����F�z�� �H��'H=��uRO�z��3|RR�V�7"�w#���U}��Y�A�a:uE�h�-���
�7�������F��-��?�"�\	}�a(�I��0�����g�.q(������R<�p[�^�vDe���n������i2�!��l��X��r�(�{U+�n��HK�V�%!P�QV�A�����+�*r�n�]�d�]
�����Zs6fkh��Kd��zK�zK4�M/?EJ�L��f�jv��(U�_'��zs6�kh�9��eZo�TkR��,1.��o�|�o�9p�j��9�&c&���k����9� *��4e2B43fY��d�\h���A'Ll�.�
J%(V���S=�'v�$:�$������T�I��_��z�1OR��T���l
R����S6)��v
�)�/�r��:J8�}D����Q�I�#�W��N��h�`t�yK�������y:!MF<#��Rb�+� �)�A�g(.��xV��	�Y�a���m��D�w�� �Y;�3|�3�e�]<K�xf�\���y��Y*�`v���YE2f�
0�(���b��`�]�b`V���Y�.f�
0��������Y�]��`���15�3i���jHY�����_���B*�������bp_�c�DT����FE�q;Xc�v�t�Q�I����Q�Z_o
���]�7?��b�%t�%��['-*���r�9��^�\,��=�s1c��*��Sq$�����[,����<�U���]���&��D�;���NJn�����o�3��Md���XjQK���9/����N��\�<3��D�����6�T�;���.Kn�z�In�����&r�6��)Hm%���f��e�HNd�B
���J 9y�
)�kvH^Z�C���R��J 9Q��������f����f���=�����Q����C^��%��GGx��5C:��t�����-��#�`4&�e���}��Lb}�l�������|�%W���9�����q��Zb������2�U�u��$������k�����b[z������=c39��:DTO��F�����9��Z�9\�Y����y9��l����p�9��sI2�*�����r������ ��,�TX����S�r�T��_�)��Sy9�$.���]4f�X�����-�:�����%��DS&:���p�q���UF�M`�X�6r��M���n`�n`gv�D\Ae`%����2��I�LX	�(X	��20A}|��Q�'���Wb�.��-�{���XlH�,��#[P�4�d�3��]u9�z�vt4z�F�)i/�{|���Z��ZNbGy���L'l
���I\zY�M�p��R
��r�=mnL����v�<���A�����u��SLs����������'���
i������Z���S&JG�,$�<��SH6�"�$��I�rJ��#i�X^,uD�p������XT��@e�G\1�(l���!��U�<Jd�;�b��Y�e`�#� .�d^����$���<6��o|9�2������b����N���/�]��@�=�]c�|�^�K���]��@���5��g�
lZ�l��q�]��78]�r�d�x�����f3_-7����=��/�Y�l^����_s�h
f�6����U�]E��(��*� 7vvp�pGs	�U�eP�D��~T>���w8�rn��I���C
��2�)���D&f�T0��R-������bb���U8�v����z��`+�|]c�J,\�	�-e���.&��f�� ��#�s\^2U������qR��;AR���Gc���Mu������&S���P6�H��T(�K���5�������(TX�O�����C5����p�=��?�O�M�3,���X��#��s�#`�0�<������<�c�<�@E�ZU��"�J��G�T����W��4P~i HA��4���[AN 7��J�k�d�2IB�\�/��Y������5*��d\&��M�HEvI����G�lSch��S�"Q6�Q�(;��H��j [����#�DB3�#������]S=5U����&MU4m����*:3U�,����#M1$#6K�D�PNFU{5��!8�&8Z!4jm��"����[9�F���Z�O�TG�n�P�	Y�����'�tQ��l�+��(4���D�`���Q\���W@����	���M����h����>�!�z�K����k��+C�{��B�*��9E�t��=��cO����9/j���T=��aO�d����t��=��`O��������#/0)�W+�|��:�q������8%\�4YG����v���s�B����-��r�6"��gp�r��f��@�l�Pm�'g�+\|�*d�k ��k���~�TdI��c��=;b�Fx	\<QW���z������R�D:��r%��x|���
�tC�<���2���/x"C����y�P�N�O�s5�AJJB������)k^
vR�%"W���� ���]*X�"�`���|LA)R�Y�b>�S�7���w��s
�'�J�pn��5���+�����+���"?�����
�uT
�h�"�U�0�S��	��<���C����j���m3��������E��)���i���Tb�����6r�Fv��b#Gld��<��)6r�F����c)p�"d%[D��&_��Yng�����|9�F��$�+�9�x���M���g�"�-�T>���LoRp�����DP:���OPD���8�0��fC�B#|������*j���q�o������0���W�.�|FS����I4>�g�j�GZ�7��}"�f�&���b�#$���:�%��D��h���{�����!�g=;k.�e�hawl�?�M�/��T�\k��
�����;�L�e�_��4�8�4�Rm�mI�*L�C���������%V7�~��p��/�O������SUA������)D�"���F������A��S�7hzJP�OO�����)��?=%&�����0�W�aB�R��^)��	�R�	z��&�Ja*L��bW����w�l���"zCE"���|����8����U��7���R�2!]Q� ��Q{��H/bZ��ir�I��z��V��r���"�V	���������m���\<D:4�jWK:&��G
}~Y�x��!Y�D��y�*R����k�P)3����O@�b�����#����'u�~�%gAZ\������F���F��6��tl>�	lT��($T�7�h��D��N<�z!�y,�^ai���a�u�9�7�B�Ca�O�bRJl���M����Z4B������TsP���]�N��1�N(�89b/f�J��)��0�����X�,��6�{����Z<4������{p|�n�B]'[8V��D�=�����_�y����W���}	�C��V��Q�z���4f�����������~��F��
)�KXo��%hH<N�[ogj�e��\m������5��G�K��J�;�R_� ���%�\�p;$����yn��^�!���%��RHyi��3^�!���VH!�e	$w@V����!�d�V�q�=!�n�����Q����r�$�o��B9$G���
���z�c�C>Mz,�I�p��F =��	�c�v�BQ��&�A\�;*��s"/g�m�jq9��8�=U��B��v�s���w@�/�T^��|ov�������S��5�9mw9��U��v��JL���,�����\������S��
���i'�*��������W��$0r�n<h�f6rNp(/��~}"��m�y�����$�4�H�z
�S%��i
�^�"A�W�b��q(��r�������}$>W-6�%n`���
l�6q���������q����W���V�7��)+�[�o���M�C����+d�.;K�CF��:d��CF�(d�f=l'#4��g����2�Bo9X8_.�u�-��t�^=���B8�M��g[�A����Y���U��D�P���j�5�n�9����m�������x5,h!������/X�P��93p��n�G?f[0���|����j�k��D?/2�'@���s3&I�L#�i?g�l��7r�vR��6�N� ��LQ�O3w��C���}���^�
A:J�w�QRC����Y��pS�??+QK-^��z,�]����)��b_=.�6��Y���+�8�&<��p�m�p6bpw�G1+@���)+�8M�1+�[kr���)��'��J$)+��w�
���?��Y��������T�����M5l�aS�VO�Tw���F���z�4v�����=5��aO
{j�z����=�-��h�TN�oXZ��������z��W�z��������X7���W�9BM��5l�as
�k�z������y��hvU��hv[�5�5��aG��!��>����u?4��l���
����
e���"�A������9���o��z�D�g���_ad��X�_�F��Z�R�����[�F��1}aD_ ��cpo
d��\�|�p>(L�x��|O|�qj���T�I���������������X�*�rU�B��W�Fc�����=��O0{�<[��;�����s���Y#~M>sU;��5#�L�\>��H�d'��C��Z�Nnp���&��O�g]�M��'���RIxR�N�>�����o�W/���d@��7m��n����
Ld��4�N���L�K��1^����Yc��TJ��^����yi�k�4�^������
[��)��^1i��S��pO]"�+�����oB�7E�F��?Y9������OV�4p$y.���3�I���d�LSG��?Y9������OV�4z$i.���3-I���d�L�G��?Y��
������nieK��O>�������dt}eW���]����U������j�����j�&��c5_E���~��������&��c
_E�w`���������	����$'���^T%�_�Y�ZS����U�Eo��L)3$7�VW�UVg�������jT�������q�$�R#�I|U�WYg*p���5����W�b����0��$���V�����5t!B9��6Yw����9Y�Rv��s4���_]�����z"��M(+S�Z*�j�C��5���;MQ�F4��[
�M��9JZC���.Q�J�U|=
����	J�'��l]=���Y����lM��K�����jL[���xu�m�����;
��F�`�t+h0�W��N4�O�j2(/�7����[Ayi��DDw�����V�����Z*����-��`z��e0��\��9�|c��mn�Kd�5*{���Z�L�Lc,���2�K�
���������
�P�Wq�*�]�m��i���P���K�4a�W0����6��l���<i���]����i���P���K�4Xb�O�i�N����~�.��}�
��K��2W��A��v �Zmo��l���8���<�����z~s����6Z�7��S�x�7�����6��/`'��]�7��b���V7Q��������>;�<��8�|-W���f3�Z�0	$8y����2�oW��y�����'���u~������E��v�!�3�_�n��Q���������
��7�?\�gG/5U@y���������������������u�|���j�jR\��TM����W?���������W�/@e��#Bi��h��e@cV�}x����?�����0���/�5�#.7`��7�� ����_����@]�NaG�������� �;�?�n"��Ht��f��95U�NDb�����H#i"����b�]���`K~~��H�"s����Fr�W����;��_
������{�Z�W�W�����)�0M�~�U�|E�iJl���`�4����\,I�U�SM��9���,a��|
�h8)��
V94��gz:{�
.��;�\*I7%���G2^jGG��(�Z�ll�~SA�h�����LVkyA���U�$���v#�7f��,��K� h���,��E^��O�L�87v3�E��h���*[��Y������H�����Qh�w����������������Ro������8i+E��3�`���,��]-�c�s�;��em��@��zt�k�+T����m��E�BY�@$�B���n�3 a��������LP	wZ�o���C�===��1~�n�S@�����4�*����EO�����u<I��=�n:�R������������j��d��+�I�����b$K�E�\�/���jx�?=��
!��a���?��H���b���@=f�a��h����k���������6�,�a��})A
-'��I��D8��c�n*���"��@|������NT�����[�[.d��	>P��o?
���:�(������d�k����/x�AhP����{�I$Yl������_��G�j��H"��,��z�j^�||1�O��@B:Of���	�%�E��P4a����=h%dF����2�
�69�#mi;��K$8���f��uX������'i;i�6,����<�;v�9�&��D���!{��q;�$�;��%l�@���2����6�x�/���m�W���f��+�����:������&��W�1��fV�a� (Z���r�Ei�6G\��Z�aJ�Nx�b����v���p��c�V���\��y��?~\����^�������o�S|o�`��W�~���cQ�1'*w�C��^��W�wE����
7�h).[a��G��(e���8P��(e�4�L���I�����"���Q��"����~�p��r��	M�w�e���j��N������,	|���*����
�f�U��+��&����?&�4���R����S?i�����������r5M<�O&���� ;���9�`G�4��h$�-��l�jsW��q6X��f{�-&��7�����)T����z���������{��'�n}����A������M��=:�6��m�9=�����5��/���&�2�TO*uB'�0�� ��yu����a�I_�[���s����^)�J���8G@v�}��lRr��2�pX�-�,��q#29p��;�_F��',GM����xq�I?��\=�-�rAP����N�^����~z���p2F�����3�(N�Fr��YX�
b����3f	Bt)���������r���w�&j��L����DA����SX-�+I��_�I'=�T�u�����b �a����y �6�kHd
u�B��b����->�J%Q)&�D�\m*4����}��9���Om5�EHa	��S�[�?� 3�%�@��?�-`L��"w�vv;_~��Zo�?g_�^�O!����OhTN�LO2h����M��U
���pgP@4'���:��O-!~|����E��4��O_h2��M�.����u�$eN��[�
K�:��}�^�VWr*�ZN�����*/T���/\q]�8��
��*U�N��N�v���c��Z��s1A�l|Rr`�L����lC���k�������O���v`�v0MO�C������4��gq:���i2�rT2�$j�VKF���'g���U3F�'�Z�������b�gHK��I�k%�d��*�
�F�Ob�X++BT��4���b����cC��D�PL�i�+�dV�
C���*���ro����"%W�G?����~4���=��0����G�����������'t�t���+���c�p��X����e6Rq�(K���B]��;���������b�}�L�����Z3r�`g���u�mM\�'�����
�f�����l ����#a|eGVUl%0�P�i��������=���Z�'�3�GO�
�4��M����<�����9!��m]�KJB�l�Q�_�k��7-d���h:f��S�^�����������C�H��#&��C
d2�B{$��D�a�1s
�$�X<:v]�iL�����|�2���u���)�|J�H��t�HI�_-C����~�0?JF��1��)��)����������n9nZ��6�18=������.����y��se)��R�R�J`)�l�q���S�N����fe'(+Y@Y�Hec!��},���#������2PVv�A�XEec��E,���!������PV��Bu����2�v�A��@_�
��_������*^���������t#��J��_�������4�(��H��5�z��~6	����"��j�`�������=������8(���$�D���h��������o���e������H^^l��F��
85�>%|P
�����e���d�]�o��lN5|P�B��l:����&�w��L�d���
�1�68����	��
.����x��'��V8�����,�
�g(-p;i���I�J��xF��6����}��Dj��' �{� ��H���fH�X�.s�W�1�Ka�F��`RM]K�{]��e%, \4�O.����e�#���-&v�w+EL)�Vt��e���)��d�g�My�0:,=j+Li�o��o��p
�bm�U7#
��y��5���i�1�����C�������U�W]����6��m�o����A�u�^�j�!j�������	���&OH
k�H!i�mR����	���&[HJk�x!i�m����	���&gH�k��!i�m�����	���&sH�k�����UF�M`���
l�6u;u;s��v�D\�e`%�M�y.+A��1��������7)[	���~:��}����N���C�r~������H|��>L
aNv��i��i���]��|�!J{>���&"b:f#)�.�&%�R�[zS��G)^���i�p�?���y��	��_���������A�s��.~�^�Agl�;��p����t?1	������e6n�e���m�la�+��^4[������Ln���!������2Q���+Y��%��F����2n4/� �t�^,s��E:e�+� F\���q�c&?Qx��:��������U:���������t�����[�<�_�e/����D�4�PF�����I��I��]����5�Z����n}^�n�`J<w��6 �� O��4a�
��`	} W�U<x�����l�op���j�R�������f�Zn��2�<\��!_������D/�e������m�����h��`Q��U��m~m��6w4�^\�=5=����A��� �}�S!V�`��	�W5=
�*��of���dq���i�Y���N�.Ay_���n'��t_�N��i�a�����!��O���q���;K���xc�R�W���>�{�|���D���p2�iS���8�����S�!������I8�
i'��!
 ���$c]�P��I��6�����:'>����x�<�>�>�FqV�37��B�H�"G
�*
�S�p%
��#\*��a���������9�zi�����u\eo�.s��L��w���
�p������K�V�V��Z�����H�vkT$r�5*��:C����v�Y��) %<�qe�+2��MU�4U����&MU4m����*:k������%���|�!�����\�D5�p��X����_�.�����x���~�8jv'�<����'���4��o-o<���P�eA#w�����Oy�*��i<q������~2zR���/F`�8S��@=M��&�ROV��	����JRu����=M���&�QO���	S��$�iB�4A�j���+L
����1�i�i�F�$x�LJ( ����L6��&k&��I��	m�;�yc�//m����x;��?�v��L��B�a[��t�^h�7��\ �(�rx��F)����.J���X�X��L����L_��}�X_<�O�����3m1>j`��8.����7.���[�\���]��`�k��u�~����E���I�G��������cD�E�x�"��s
���-�%FB�jM�������;��w��W��]M>A���0}P�Q�T�U+�jeW����^���V~�
�Z	V+�jeX���b�b�V��
����^��1���E)Pmc�v0F��}���Y3#sfd����E32iF6���X5��~�Z�-�u~����,�3p���v��m#�S�)���_�z������g�@��/MnoRp�����DPxW���{YD���030h�fC�"|������z2��qlZ��X�2�J�P�����E�/�h�)�_g�c���7�/���|����7��}�&�1D�jW��Bz7��:�hM�:�G��3�bP[�P'����1:���6������(���O:�h�G�l^������bOt\�=�'���oU�������'E?��:�8���WN�Sp���y�S����������;�OO�����x\��/��O��S\��?=��O��S�Mh�<~�L�}�y��V����V����V,'�t���Mh�|Mh�
h��]�n,�%d�X�������]DQ�)��^]q�q��)!�.�on� ���J��o�����FB�1	��4�%��:�����ia-z�B�J�@�iC�4'����#;s���7����HS����j�O���U����u3��.�I;��:�K9M
8�'4��yU������<�Q���qH��o��� ��x*�B��X8�s��f��������Gu(,��ZLjA�-�CQ����?�[�F�>�C��Pd�C@���0d��z�x��B�(��,�Z��0dl/����w!`�hk ht�F7htC��0dYC@l��J`!`��S~��:������_'��	�u����W����_��h����/�-47����2_G��:Z@���X'�j��M��g[�4@���Y������D���?��+,�5�n�9g���m���)��	x5hI������/X0/ef�Y��8���-�O�:�����|��~^dO`OJ
��x������a^&J����u�X��s�d�p������FI08��Kd+�����z8���������������S
N�����
v�[�b`L��=LJ�q4�(��`����&W���l9V��t��@��`��qjNvxu�W����~6bp�G1+@���)+���|V��5�N2�LRV��)���Mt���v��*��m4l�=
�������mt��hk�����h)��mTn-��a��i�E���]4����E�����r�x��F�bi�P�������Z�
j�P��P�O�s[mN@u��d�N�v�\a;
�i�Z�v����yb~ �(�G4������P�>:�4���w����������>�)�F��

P�"�@��q:�Y7���o��z������Zk_a�Z��Xc�����Z�R�����[�F��1}aD_ F�cpo�������T�������uW7�����R�}.!�uJ$�@���t�fM"f���_�4|���q�	�_����_�vT3]����Ub�������g�h���T�l1�o��S7�NX���)]�W���$7�:7�:P�����Q9�cR�V��'�!���i����h,���$J|�����a����M:5�"���W�2���2wt>-��O�L��z����q��,�O)#f���Z����
9'���s�/sSF�;q�Z4����I��#����Oa��Q�h����+6�S�U��S��\�wJ;e�8%�R�$����3
����d�L+B�x�?Y9���]�OV��'$=���3�
I���d���e�������(�Z�<Q��%��n��G�)\xA9^�x��d
�:�^����5^UJ�I@�l�V�51D��2��-��Y<��`�C]ARW��ap�k����a�1��UE��&+�������H�?^�l]���W��[�:�4�!�*�+~?u}M5����5E�������������&iG����������N�j/�����0�eAO����HS�M�^����AX�-,h%�S7�4��
�W.�]7�p������_��#v�PV��D�[y�P�t��"p_w��W\u�S)W�n���O�k
F�>����1��6�T���~Z��������bT����fb����f&�!��"�M�����f=�����OG
�}���Y��`�3#K��
��T�3��
`M�� 8R84��)��@����
3���l�L~��V���m�������I��%�������n�M�
�.�?ZS��gAW�$,������gg-�c
��cx����9j]M<!�.T���G�5�6U�{����d��G����i9�k�8�Y~�.���s.7�D���m�h]ES�����Y��Z�If/6�A�����u��/R{���mX������S���N���������7����TJ.s�,QK���7��`����]�7��b�70,��~�V��M4�mo�u�@�
�+��z;�w����G3\����j��_-�(�5.�T����&_�K R}�����+cQ?
m`���Y�)[�n��Q��������7����.�����* f��~�����}���o~y������I��I��Iq5#S5����_�p�������^���]���9��y��h4��b���x=��'��=?��c����6����y��0s�x�Z�����x0L����o�����b��}O�OF���D��1��D��D7�j���SS5�d$��HZ��t0�&#�.��k:�����������+3�~)��o%�~-��J��\������x����nu�5<U�~�=~K�$�T�*���S���*��T�)��r������r:i�YH��qA�N����+;�(8��LQ�i�^:�Q#���+��-����%9�'-��B�{�v�:�~<*�Z^���a�4��� S�� #�?���S>�6����g�5F6"i���f���T:<:�`)�D����7�>|��@F �/�� ���&���7)Zk&�Z��O�1.��F�"�(��_N4�>����v������$<�����;���h�W6�]�p��.y�P�+9���P��o�[�H0�.="$�3�EgG��EOOO�m|D���%#Z
*����=�N0|r�#:,�J}VG�%Wg�MCL�A�&��3�Lg�-6<��/".�qa�N�K�;��3V�Oa�'�)i�Tyb4�KN�d���"�S}*$�y����0��%|9Y��6:3sF
����m�x��"b;�R�G��?Fr��>���\�$��*�Td��E6�����}��;1������o�l���'�0'���4��bk�H\<<t�@�Sc-����h:��5Ax������$��Z����1y9"sW�>�f���r�������d����	$$SI�j��%�E�fP4����VBf�oQ	.S��R��eG��v��HpB�5�T!��N/h���a/�v�����)�$��� ��(��##:xO����x�^Q1@��&����\>?j�w[�!C*Nm��6�l7Q��0h/�,i*|,�@��������V���6��G'+n?������x2~���Z_���?��q����|l?�W�x�x|[���|�������-��i�]���g�D�{����=V��I����JT.q|�\{T.i��%�S�$P9��T�����\\����Co���.[n����E�n��]����9D^`�v��UKG�'�Z�I�4��)�������$�I�z�
�u$�G�V��cI��d�H+,��$%O�_�v2�AJ�X�2v������q!%�,��'�#v�@N��90`�#���S��vd���*�f�E�m���ba2�ys~�����B;�=#?����
�\
�$�I��%~�^�6�g:@>��1>�6���W����Q��p�Z��	�	o$�Z�EC|u,��Hu�'�0�� ��yu����a�I��|����7������A6{N����B���R��5Z�{l
�8�O�x���,��q�2��;G��� �/�b�����#�l�'Uc
Zz�O,�COo��I��v{�q�`m���x{������g�P�r�H.8c�DB����l��|X,2��=[-��m�����5@��������0��+I��_�� =	T�Z�����bQ�aD����� _�j�a$o>�7G�������DhO�~��X������e*���V}�_��@6�mF?�,������I<>���g�X�"�������#6�z{�9��`��}�w��h��������$���)*��ja�P���_7E�Ds�������.��6����~�Y�iH����J��>�M�(��3`��+@R`|;/�nF�J9��>�u��h����T����qK�U����K=^����q+��U����D���1�8H�
`� (�f9�bER�~��<$�*��#��@�~��K9{r���	���������'��xA���dO��8����4�@�+�O�X+�%��d��8��U����[2NLK�x�+�
}e>�k��\�W
�R���x��+^�8�J�x����)D$&K�_�}� D�������~.�LAM�2����!Ff[�Gl�g��L������S��T{����*F�H��������
��5�=��X?�8
�-�y�)5��M���8��|��:��&g|eG��U��1�P����&�����q�Z�'�W<��B�
��������1�;����
s�yt�A������|\Q4�lv����2d<L�S�$�BN��o��$�������x��L�\@����� ��0)���G����!�Li�:,��)�RJQ��R:�c���uX%1�������(1,����@N�K#��K�S��:.�����J�>�����
�Z��
����%�U����������f�Z���f�"y���PV�A��3ec�
(+[��l,Yee���cP66����`��~1(�E�:�v��kIi;|�pU
e��]����y���F��C�O���iP�wr���}%�x��^^�I]t{�R�_R��?��V�v�z����	E}R�{=6]f��t�L���Ns�A��ms����U>WO!�UV�"���}v9�U�6���#�x�#�h[*��$R�p�A��3\c'Z��]Zo��ui�1.�9��A��;���93p��mc��p�f��X7��Y����3m8�e��q�
�g�,p�f��X5����v���r���wb��z��?���������>���t���\���$�G��Y|������u�:T=KHX,�h��\,�+.,�����-&v�w+�(�#St��e���)��de�My�0:,=j+�@i�}om�o��p
5`m�U�
��y������h�1�����CO������U�W]����6�m�o���\d��u�^JV+�.jYm���f�1������K�V�.iZm\��j�1������K�V�.i[m��n�1������K
W+��i(N���!�l�6u;u;s��v�D\"e`%�`%�`%�`%I���LE��hx|�����S���Hv�!Gy�=#����I�����&L3B(�I/���{E��'�s|�DP3�8���p�p�Ujvm��T���A������w���W�����t���h5�*���|\:Fgt�E����l�V�'T�E�N�����:A�3�;�ya��8[��$�W�J��&������=��T�����Y�Y
8���/xY��U�%�K� ��	���v��r�I��{�'��xoT>~���x����	;m�/��������l�op���j���������f�Zn��2�<\��!_��(m��%z�.�5��E��&�>���*�uD�V�������<��H@�\< 5�1����K/0&�%.��
���,M������U����O0���$�+j8�I��r�������3�p���\������T�,'�j9��zM�������d����3�%zd�%����JfF��t�e��V5���V9����d��VX���q��d7����'>�S�S�3`���y �8+R��#��/
�S�$���GTbx��s�+�\��=.���`�#7���d�k�8�2~M�\���1���?c@����B�$�%�R8���u�V$r�5*9��f��D���FE
H	k\G���<nSM��h�TE�MUt�PE�q���p����_������_KQS����*8eT����;���\<{���n�����M�D�1g��}"�����1����D�N�S�1Q/b<���l����`��������a3��!������i�^�4�-z�@�NN���	C����iBK�4A$z�p�=�!��6�$x�@qx���4	�;����Q��G��`���'���06���7B���x���������|�����n���6�q���:
!4)�c}�D_<����e/v����/}�X_<�O�����3m1>1h`���?+����7.�qkP�c���]�by�kQ=u�~����O�#���zu�\������������xx�����ES�|��1[�Z����Il�y��v��������T�S��XQ�[�{UiP+j�A�@���"�V&�
�Z�P+j�B�`��DU����b�h�`�D�#�cdt�����1�;F����H���|��"Z�7�:_�r;#t=�l���6������������oz/�={1sct`���4��&7�<�l����.���<��uq�f�y��lHc�=�O6���]������i��)���\��7��?\��"�&�-)�����G�7�/���|����7��}�&�1�/�j����t7��:�RM�e��r� �l���:! ���G�,���;�}pG! �<F�DK�me�R�O��{�����I?Q�<�x��\g~�}E?)�A��9�h4n6���#U�h���?-�s�?��X2��0����(���?����-���j�<�L��o+FW��idB��G6`n�����L��t$[Is5	��.x?,2�*DDt��j���J�)B�^M����C�U��7��URxY%u�7e�vD�|>����,�g�P�tu}amE��4��`��+�4uR�s���1(�
����C��`�p���cR��x�������Vs��$�����%�&��V~���,���Ct���[�C��9p�"�����R�/H�D���c���)�1kO����*br��+��k�Cb���&$��`!��Q��Tn=$�9�}!$���Bb\Hls���������mBb[A�TA����6!�MHl#<��AWHl���x��+��W��_w�x�!��������������h����/�-4_����2_G��:Z@����������&_��-�-�P�������\'�����"Xu(��wS��G�oshEM�^`��\�������/X~+��9�����-A�S���w�������M��"x[@Rp���p3�I��(�2Q�������"_X`����Df8l�/h����Y�DG�/p��,;�C�z�Z{�i�Q�W�{:;��tLny�z^�`��%/~�����D���s��Z���xjr�����c��V��������p:����8���Q�
�'9+
�����"���gg;��\������Z�
[`�w��F�w��@C��m�rka;�GO�h}7��a���5�����
K��8��a3���SO�fh}7l�a3lj3�?M�����`��=�d�
�V�\a+[a�Z�V����yb��=�����q�a���_�z�?��a��_S��~�������Q�n�����h8�j���h��p����W����|�W��O|!����;�/�F���0�/�����~aL_�������8��x9�4���+wf�"�s��v�����Y��9U:*����p���)��6�k\�/�q�}�_���+��N��������RTq��&�hwYQ/�!�^<��g��R�)�:Py]�����
_;^D�s��0]�!I�Q4����|
����YG��~�=���[�z������e���~���k���96��b��q�/���bv����B�)��1��R�_�=����27U����tsK�U��Zg� �����/7&�����	����$)	���31�$a�rQt��D���4�R�� �!b�k����8�+KL�>x��S��"��M�@0������'��}_n_��iFM�%�R0Nd�x�C�8�+1������"jf�e��z�N0VN�F���%}�uc79�jow"���w��Q���m4�p���l��E+\Yru����Ace��{Z�{���2�dMD���4R�_��'.X�P�f��^1�
��{lE��',��������R}�"��4_��]�SY��j������;������:������RC:���3�o�~���y������_5e������
5q�#�o�Y$hXj�q�;�r�MD��i��;2��u����j*�*<'�����_U�m�L%�_{��TTY�����{}�����z�=�f���=�v��T+������	�@���1���-�)�����s�
��
zD���������YM��W�,]U����T2�u�[�`�UZ���|��[�j"mw��PR��p�W����0�+����'x�T�
.?a5����e8�{t����p����.O��J�
�x�6��\^	�x���
�x^�iB�{:L���x�9�����]�����K��2W��A��v ��o���6�<��Fo�RY�7��S�x�O��
<Y�")��>����6�]���Z=,�7(�`V�}���s�6���;�<��8��j��_-��f����"�e�|�������Wz�~���XA�g����f�������/^��
����.�����* }d��~�����}���o~y��Q}��?�����PW�V��5�?����RMm������7���o?�z��x��3D����h
���%�F��� z��O�s�`?��|
���
�l�M8��C�
��W�_����@=�� =rk�
�V`�����o��C�E�xI�Oz������'�I���2��n���'��t]��Lz��r�i��I�c���k�|�N�o?�~������\K@�f��
ho��R�X^�~Q$UE�_�����vQ$�E��sQ�o�N��K��������!������
�k�";��������BG��>�]�������Qo��-���-;�'V�ye��t�����i����~(Hdx���~�J(�"?�����)/[�^A�=ayu��Y�������2v�$���z�
|r���)|R�@!y:�7����'�$�7�v�L��
�f7`N��8�~3�\km���*[��Y��Z���c��f����]����;���w�
v��;����kH<�	X�����{w|x}
���Qemt���CT�m���:��mg����������j��h����|9����T��m��\�H�� q�9�����V��P�%_5O�+o0��^'��'��lU���������l��������2���(
�|7*��	5p���D�u������E6P3�dZ�B!C��t0��a����
��p��!�����|�������6�O�-|
6+|;BO�[��4��S�����`���	z���A$)����-������V��X&������X�n6��(�%����W����;(�/���8��-vS����L������^���K[��
_�)�*b�DD���W�5�g����$�������vN���IYh��,�Ke�*�z���d#$��t`T.�W�'C~�46�4�6�8mi�t�U�U{������J���sGNE=�~�yLJ�����������E)���E6�oW�T�S������_�x���Y}�m?���O��+�R����C'
4�,5���P�`UD;_�� 4�	��������e�,VV<H�b���W�j>�J�>��r�����0����6'��#J��2Y�LD�%���=��
�p���9}�,�$��qY�x@l�H��3B���0\"����T���imt���^�)J4S�����q���S�����	�0E�f�lY^����T3Ei�)J�(�:E&/�:�d�
�V���|���(�i"Lao��+a�
��]$���nn��T��Q��d�8v��|��D}�J\� ^������[��-[n����l�uwt��K7a��[#���������u�1���r��^=����W���+t����0��>���&�Sl�p�X!5�I��:D/����_�!j%��P�L���dTEM������H�������8�H�Z�J�:q�hh�����iZ�,�NZI#�)Z�4O����zG��0i5�����).{��N�|�t1����7�@�#�S#�����h�`�!v���*��-�y��/��VD�}����o�
��I��;$fv&�yb�r��$f{bwI�Zf}����}b3���-\0
U3�4��<�l/a������W����e�z�������=7)�����W�
��P&�
>�6��g�J\���h��6����Z>��SVry�6{�j�����DqN�f'��������>_�'}���W������.W^%GC�`������L���I	b��:;��?��%[��*N:�%N�k��g�=����:��l{~+8G�8E��=���-�7�~�k?���)CP������_��*w�,9��f�#v�.3s9�6#�}�.�2c<��#�t�.<cF�D�����N��^>,�7[-�������5����`�!a���$�+%��������!���a��	5aK��`K4j��&����v[�qI	�)Z��Xv���y��\7��6;�>��lLey����%��h(�{4��e���|����<��}q���>}��8;�?���3=���M�Jd@�#T7���Cj���hNp��-�
`7j���w��,��4��~�a�o��!�YJ��*���O��x`i��]�>�u��^h��X��T"�2o�U����K=^����q���U<���D���1�82
`� \8��u����R��2;���?�mt�a�wm���s��������X�&����P���Or29O�I<M��t8B��4>��b�T���L�wG��d���d���j�V�J���b��)�=rc��Nb�X+�%��T1z�
U<>��b��q<�V<��N��b��q��8��L��x��4���Tq,^d�4�C��
'O_q2.<y��x��xK��)�����F70���
��q`t���}b��V�!d+#��')}����Sb�u�����d�Hb�2L��Oh���2
:b$+���f�q%���D�&`s�b�8*8�J��<��xZ����W���c��e���tbV��i2�f��l"hg���u����~[sO!�_��
��0@�V�%!PPV�A�����+��So�]�d�]
�����Zs6!jh�9�����[B�[�Yoz�*R�:��7����j���XV_o�&M
�7�d��eZo�t����0XrbWh��7l4�����sFE0�6����"2�dd�2y ��c,�����A�T��U�	�����&�Q��M1��T;�D
�h"�%1/�%��$�MN�����`c&n�$K��)mPAm:9e�Mi�
j�)�^w�	��(���E���Q�I�#�7�����F�Q��S�3u<�$Tl\���hz0�!��K'���D���PpgWQ��D�J�.i���	��.J%A��v.�R�2�R$T��(�Q��� J=
�D����Q����(@ f���'b�
���@��1
JH-!��$b�
�`�@�B1
D�Y� ��!5ti���j�C���T�_����E�]q~F`�P��p�X�� ����j�cmTH� �p�y���uFEH$K�Xk��Jc���_�Z_o
	�]�7?#�b�%t�%�������zs�	�^�����y5}����%�s"����1�eI���Y���U�uOuH����nKY�|e����d4v}�������@N�:)e��s�������l�����R�8Y*{�)TG��XwU?�3��M����Sj'Ke��9����uP�WF$+��%�G.N��\��>��.� p 8YK��YN��B������H������)/k�!x1K!HXN�2@pr����;��3��r_�!����	>F��D���>�#k��;g}���^y�Y�,y�t0����U��>�+B�����b�r�c��D���2G�`��; �
�[���xV)-\\���_J�����l���,�+F����yiY������#��CDu���kd��*��3����U����9�����e�;^�	��y9�$������J@���s�����Sa9��rN-��S�T9��rN��\���R@��.gwM���s\�G�������D������IM�p���7�S=���,����S5�c����L�|�)���4eP���jv�3�_:��l��2��(T��*��#Q�|,*�Z�F%P�����8�j�7����
l�6u;u;s���`�����J�+}�e`%����2��R��&"����Ez0���x�/���?�
��a&�����`+�J���E���@t�6X)��Dad�K�����&e����SI��BH����(�2e��}���y����:
''a:�ck���O�H�������l[N�����QK	^����v�<#`O�A��H���s��Slb�-��m~����k��CC�1yE��T+Cvbw�DQ��d��H����31(R��4f\5j"EvQE�t�Xa,���1���E.q������G	�f��q$���Y���������������[��X�������/'>%��t�S]lB�������r:����������k,���k������|�b%}�6�`�|�
,X-��8�.����o�Z�L��o��|�����|����=��/���l^����_s�h
f�6����U�]E��(��*� %vVo�pG#�����W� D��~T>��w8"��	�p�U����!E4C�P�D)\���3W*�R|�fR�[���:�Gn	R*Y�
�#l��Q0(r>��1A%f0���4�DfTb��O�l �>N����l�`���c�hR��Q�r1�q4��jr:��Lu��2�X{�)CM�G�2Tq^?��8�h�P���j�=�����'��r)l�] g��3�b`��8CN�!O�!�
�
�����R>�#�0��H%�+��*�	�w�+����.��X��z���~r�����riXs�C�ne��f~<�L�,�#^
�_�Q�Z�(,T�E&*�H6��"
#�Y��"
+U����D�����YK�H-�Fj7R���Z���r�H-g�Za
�:}-D[KM't�@M���
�4 �s9��5�x�����UHZZ����+%q�������?��KRm`����V*���d���:]`�2+V��6

��*"�!~m+h���-�'���(mB�!�OE4:�e5ZM$��kcGH��T����z/����j���&(iO���	7����i���k}"��&lhO#��	��D��iB}�4q={� ��o�Rx�&/*>�u��4�e�a\J6zi��M/q2��~��M�n�#:����9k4Xd[��"S!�Y!%=8�=������:�8�T���@4E����$��u�c�`q�(�!D����N���DS6���5eM�TSv�);������YS�h�F����l�)�j�N5egr�>N�s�a/J�q�����kS
�Q�u!W��������)��"B]��|�:)��Y�a>�R���m��ls
$'�4J�pn��5H��O��V���F+���"?�����
�u�~kH�v���h������|c>f	����|@��|����?r
�|�{�>�9�����jv���nG���jw���~G���jx�*��G���jy�j��G����!�d2}�Z���Q�o5�E���<P�E�W~ �����28��<P'r����D�X�@��S�1j��-�u~����,������v��m#�s��{��_�z������g��C������7)�Y��f�i�(_y�Gx/���p�9�n�!�aI>�lN@Kv��g�f��Y��D+w��}J����}����^�>%ZR����4U�����W��w�}��X(�~���2�r	8�/)��1[����<�L=S��Y��/c���s��m��On���Z��T�`��-���Y��/KT���������j�l�QSa�����fg%+�,���SU���7|���w�W����O�����IJ�����C
'}�O!�����������)>���������)��?=%D�����
��rQ\xqiP+�0�V
B`B����Z)\�	�Rj��&�J��N����*��v��@w\�h�
@$<��c���S�9��oR��0[J3%d�"z�2@�"j���EL#�3MJ*	:X3���5n�xQ��jB��Z�3�p�
�w���H�����j�O����\��/�#��z�����h�<OYE�s��|�
*%|�z��	f�@l3�R�X���D�N �����@H��R�Y��I����f�H��X��6��l&�D��"Hr'�Z���<��r����w��
����PX������[���r��E(��P�8i'�&U�T�j���4x��#N���M���r��&��k��.V3������t~n�3
��5j�������s�k��5Q���OI�C����c�>r��d{�8����|���js8�R������^�Y8��������(U�\�q	���U�
���Sc���,M�������#c�����&S��#����^,S��)�Z!$J4@p���|ZD�Z!$A�C��
��&,3<��!���Z!���;�2@pgRz>g�b,�����F�iS+^
�|�����4G�w���H�/8W �	�!���J�"|v�X�n]���hkR_����t'�+l7�u���w�/�D^�:&�����r�#6�{�������.�qU��_����u���nM-.g9�����If����r����H�720���h���A\�����f�!-�~�����%����N��*���;�������P^�V]�DP%y���)�A5����4@y�[u�A�<T�K����&�����:U\��/���������/���K����������X\aN�`���
l�6q�������9��Ez�D\Ae`%����0���W�f��J�[85����W����IG�F�FUy��]�u ����7$�
�~C�_8����2����xYC��d�a3$�uI��>����j�����
���|e��h��W`�D����m�/��kS��Ln���b�:Q%�,c��Yi
��b@��5�|�C�:hQ��H^@U
Z$��5x���X��h��Ycl����� �)_�Q%��Z���7����	P�����)�$9���:[,����[6r���c�����#j�i��=s���cN����B_!�F���Jj(��0���n���g�i���P�R�%2���_=�K���%�}� ��~�����dW�wnw?C["�L�Q�
��~z�
 
�t�
�N��&�`���	+@>I�
�h�����O�m���kd���-Ti.��a
{h�C������������Do�-���P�����-4l�a�VO�Bw���E�u����eQ��������������v�fv����G��6'�:O	j2��a/
{i�Kk�������C�Dy���]�6����
�������6�f6����C�;�)��F��J6�"E8��Uc
D�n4A���}�^	��O����n���&5

��}�0�/���}!5�0�/��c����@������������yg�^�#�n���tzO
�O{����V�fs
�i����p���
�_E�Wn�~�	%��b����n�����j�8�nT�]��[r�&���Q��[������*�����y�C�>�N�Q��0���NN@Q,1]<
��!�Z{���|�u��$��������h������#���������?��&N��U��0�Hd��&W��RLV�I
������vA5*�'j��pvI5�j��8�I����7��B�Q9��[c�,���{�G49A�u��r@1����+T���s����3��cO]�c�r�j41!����!�L����isH.S�'+g����������1e�r�	"YL����i�HS�'+g#����������e�rQ��W�K�������P_��
����m�#���$��>��e^s����lu���[\UU��{k��5��	W�����4�������o�k���q����F�o�U�r�*h��*���W5�������b�y&��������.��2��XEk)w�����.S�Jm�#�:�=�T2�W�/�
]oD���j<�U���M��c.��[O���D&n
�����V�	�WA�*w�>������R[���N�?Om)����K�C����l��9(
=��:�p+A�km Z�\�n!�7��}���PX�-,j%ew����$��m�}���f�j��Zj�����3d:C�+�Y|	*�V�u�"5m@�X����*���]���d��J�ZQ��n5�
�W=�T�5����6������mT1Y���WKV����5�.�BIk�PYw�����SY�����s����_]���4���%�����%Y}������m-Z�"x/���3j[��;m���}tdb�X��fei�fK�h�s�*�-�ZP��n5���W}^�:�������W�*Z�����jB]�:YSi�*jIu��.�_���F���t+�L�W�$���L��gP�*ou�K����zu���D]�{[���OG���O�TYsM/ja�v��_���u��6���T�FU�-�Jt���eV�I�������4�5�Q���K<ce�i0���}J�I��KZ�/�������a���\�N����>_]"���S���gt8��YEfS�4��i���];���i�������K�4�OPjv������~�.��}�
M�K��2W��A��v ��o���6�<��Fo�_�7��S�x�O�j�G�|�r�D����!���V�^��aj�\|)��g�����,��?�79�L�\��f3�Z�j7�|-s���oW8���������������W�\D�,�/�������zv�������_��x
>I���y���?;z���������������/�>DP�.����<@��J������g������W�/��;�!��A4���+ �K8�>���A��������G�T~������
����?D�! <sBz�k��w�#�A<����tp���0|���������o��C�M��K��d����:��j�I�V��DMm����_�p������f]3�����D���^3����w}�<?~���r�����-���Y_��������B3�4������,'�YNJf�X��Kw�`m����h��w�*t_|{���?`��$w��q0����x	���\�?��6���S����D��P�|lzu�z��R�j27XN[OL��F���c�#�
�c����e"k�k����Q��<^�3Km��&Haf�����c>%%N�������4"v���f���naxt�@�E�@J����7�>|���E �&�,�j$b4U��:@8&*�`��X#��v���\��o�u���e�e��@G�7�	:���
*G��ft������@F�����lJ �8�%�-|(����C�>T���S�>��&�)w�:���v��r�
5a����!��uR�l�d6��^���D��S��0�!"#P�]s"
/���V�3�8#�T8�������3� ������!J$�Xz���V1��&|9���6;7�=
����m�8�����-��$dq2�;��_x�F	��P%x�$w��f9���A��l��S��_�x���Y}�������A�[)G�R��p��x�]l|��	^b���k��CpR�[Y� ��T5X�n�%����r���� �RjjF�	��H�����J��%��:�����$e9}��(���b\�V��&�L�-��Sp�#�HpbG�V�U�){A�t��R��W��s�l��-��$Cce_ydd��I�n��@�T�"�����]��?�&���8�M��g[�Y_��������?���<��o���O^��qJ�Y1V�-��h���������i]"ieM)uk����_.F��>3���?�����d���z�p���#�B�OS=�-c����j:����S��v����U�YV��	Vx_��U8]��c%��8�'D�=��4M���D�V��������������lG_j�����)^�a����I����C���@���J�M����
3�~)E��2ul����o�����I~�O����k���N?����������M��LaNt�LONT�L=.*Y�%3[��3,���
y��^f���t�������o
�����^�*)ts5 ^��&����Iz��@��yi�h9�����?�Q�.�����;��lN@OxK�4�.2��]M��b�H�DR�c�9F�)#Q����}�#M���bT���Z	w��JQ0�C�:�M]v
���j���I�}�,��q70;�;y��N��1(X���w������|���(1>w�����+�A��$w!K��&
��n����69�&'��;� ?�)��a����l�x�+7���q'C�� �:
�>Z����$�+%H��H�^�O���)j����N�����SZn+S4I������./>��l��E�PP9c������l��=�������#��?��}q���>}��nv4�BH�gz�A�����@PT�P�N�i+l/:c����SI|����E��4��O_hR2���Z�.��<I�1N�[@LK�:��$�^��	r*����-�V6�2V.�x���b���m7^�U��2��!h��P��=B:�����X_��,j�D�K_lVN����x(^�h09���'�$��gq:���d��j��ILFg������i2AA��ir��ZF3��������(<�tt��Zf5'����l����I�k^�������t:�����c���Y�\�	�K�_���Z�_�����_��c�t�|T�w�/����7tE
����}z�$<��������4��&�1T���s	�����kW�&�e5$@X���H�,p�o����c�%a��w�����	�y��o/X���T�:T�x�3T��5r�]���J����_.v��?g�_T�����$$��
�
�`�do��v�#�Z��L�`B���z
��^|�
���Mc�r��_8B���$���t�r��"2�CO ������{,��t���'�C�6�p<�-�������K�&	c\RT�u�)�Ie:9e�L�c�(G	�4�h��Q��S���Q�N9���ph�8�l���`�(�?����j�_K�l�7����>K�����'M0I�
�y*����R]�?�?8u�����t����������c��t��8(���` FN�1s��a ��b�,
3W�@�Eb�&(H9	1�U
�;
�42��]����ads_�!�����42��r�T��T:�wXg��������J��[^��
L��x��A��t7��1h��S�h����B��wj�[�ee��j���+k�eR�X��E��p��h�si�1��9e�A����!VvD�O�D�}M���O9E!p'z�1�B<��gG�t�6<��-��� F%�-�Nx��\����=C E����]��%3��\��5�����2{������`4&�t=�T=����9�	�EC����^q�^4pk���bbw��p�R8����E'�,qMPF����
pM�1����T��-���(�T���c��[{�m�Wn��@�
4`g'h���zW�MgN��L7�X�ar�e��u��v���'j@���U��D���\m��`�&���.����@m�.����@m/�bed����l
2	��$S@��2��,�@-
3q���k�7����
l�6u;u;s���`��,��J�+}7e`%��VwX	z)X	z)X	z%�d����*_���j�4�O|����NI��c�p�>�f���ND<��zA
�F6��)�fdR'�A"O��Y�Z�|\-�u��Ws-�G��;�j��*:��NN��Z�k���r"K`D��5��9b�/�v)[����TJ�.��8�N��Ld$y�Q>���.R��4f�
��T�,�2��T#�f�9���1���A�8��A�6tV�U����`�y�������Tj��������x{w!x;_���K�����/�^��u��������^4Pc������z�_N�k��O;���VWs5�Gp��:"����H��@h���F�@�]��78��r�dQ�����f3_-7 ����=��/��.��D/�e������6����U�]E��(��*��������h����iP���)����Ar��r���A0�Du(�0YvU:�j��m�������+v���\�ngAEzB��R8'
�D���$��B��8�6�t}R�S"`�b{�h���r�N�:��5��]e��V�*CTq�X$��gq�L�!G���Q�r��$<6B���M�g�d��R�x�$�|@G�gZ���7�(���7�~��[	�J�V�R�X������Cn���f�u<����?T���02q��x)�Q�ZD��j-"U����Z�b����x�����%i��Q#���e�H-�Fj9m��3c-��W������?�[���O�s0j�$�z������vu��]�l�O��������T�?�r�����7���}|�����M'F-���Cb�aHw�b��Z�7
a�02������4n�=�OtO���x;�4�^zw������qd�i��{���?��J�lML0�+��O�FK��M�/`;9.7�����WXG�%������������v����v�|��'�9�R{�U��*����DS6���5eM�TSv�);�����>[S�h�F����l�)�j�N5egr\6��7_���]p��\��6UW�����U/����Kw�w2�Sp�0"\8�Vu���h?��gbv����+����W���S�I5]Z�Sr+��^;�!S;�����O���b�V��
ZZIK+jie-��������V��
\�zZ.r ��Y���h����m�5�<P��E&E~ �����28�q�<P'�0���DfF�@ekR��p�`�E��o�u���vf�z�����m$�*��{/�={�����Y��}�����u�����Bt�J�������E�/�hX��\z�����	|��fo�C���ij���,����9!k��cC���(Z��t�q��5$�e��)���~8�P|{�7��I���;�+�I����a@��X;�#��)���@��S��?=�'A��S�?=�{A�S����e����i��+�i���i����i��'�a���@�+���2����@Y}o���e����-���E��7��-R�?%#�7e�
D��6������g�x�tumWm5���[�V�����n�#�V��e\�l/,zw������f:&�N��7�e�L�4�UiC�?�p�U��
G�����2����a�����w�a������*�����_�/���r�:o(,T��L�������?e��-�C��h�
yB^�� ��y1�yB^����yB^���1�ET���!/@
EXP������B^���������i�!/y$�� D��]!��Qk�I���I�h�!��S�&�(�luw5_f[h���e����u��f��0�L�v�M��g[,:"���7��]1��NT	���E&�0(��wS��G�oshb	uH/�<���h*W�k���,�5����z�c���~��1e��j�k��D?/2�3�%�.krHO-.������l�-��=��t2j���%���bf.�a+9�5+����R�����i r
�\j/����Z������<��xHY�S���qy�����k�t�(�
���������\���'�{�����<OOY�]��Y�m�V�,�����m��nl#[#����dJsa+[�	2le^���F�����P����n���x����Ln-�da'3A���w�;Y��N����F�I�
KfY6�gaS��.ljaSso5lj]��]��������[Z���+li����V��O{Kk��vb/�w�f73\s���>��]���>��j���Hm;���S6n��w+��5E��U����*����o_�W��^a�3��X�3��Z�R�����[�F��1}aD_ F�cpo��`�Q]���[h�)�����`uHp�3�
]�u�<-�"�: xy�^U�������r&�nM��)�b�,��0��l�n�e�n],o��!��)w?� �q�d��#�������;�0:L�P�O�>Y�!�Q�u���YrbW��>�z'	l�P�xn�����N"��?Y9��I\v�'+g�:����d���'1����\������}�XERx���u|����T~��x)IwY���^�!Z��7�u�}<)�.s��
�W��$n����U�s-�jd���`�m����o���a��s��{������L��9E��
_5�r��'�]��_
N��2��.xj��v��e/xN2{������b��O%��;{�\���^u\@#!�
k4iG�VM�L����c��Y�iF�iM��N�G��(<���;CEf��9��T����y����E�����M��KA��t��	��\��f�vD7[�Us��T}�T}��Uf�sG;^��~+������\p���������������&j��6���R���O�sx��A{����iR{\�x8U��#��a.������r�0O:�!c����
�2a����p����p��q�`��N lW8��5N ��B8��4��p�t�� '���stq��� ��Rr�+e�Z�������;������f�&�D����������|�^�����7t���o6�1X��u~��bt
V|?����,R�3�>e��m�>�~>���5���~���W�����H���p�?�{����_^} y\���(�x����z�z���m-�����u�F���(��HW��V� (�,��kU��.���Kk+BP�J(�P�>k�AK���r'h���Y�>��_\H�\4(�Z7��w)�
���R��us]k�"�@�s�G���*��Fd]���Hm�o|:@��e���$��/�,�K���Q8�6�
e\
�Q:���Sz/8�
��u�thbPwJ.3��e���@�z�h�0Dk���};���M��d\k����z�CS=�K=�uT������W�/�D�2���� ����`�W�3}P����s:�y?��|
>��
�F�M8��gg�(9�������o���W��>Y
�$�2e���&ASO1	��n�b���&����1�M��H�X$*�f�P�B�Z1�� �Bk����KJ�We�t�N���|�.��X���� ���\�����*���^����8�zV��_�����KMdMW�C�aoNMR���D�[]N�
��]� �hs���F�U]u	E��j�LX(���C������BU����2ud)?��4Z��rZ�����h�!�Q���>�N�^������1{U����b�aM���y��~�8�o���v��C��1��{ ��{���;�:�w��/jJ���������_�G4'(����C����p�.��R�!}���+��9������v�\��%f���K1��hS?�\]�G���Etu�ur���r-)Mx�\��j*WD�Z����L
����Ui@���UR|��"���b��D~+qy+��J��xj�3�x�����w��D�q��s-������
zuPF�lP]�����������,��K��I�����BV����������[�Mc�|��������7\YR�\�����_��s~-�'����K�=w�.c�"lC(��
rC�����5�EB�E"���](�#]�_�R��P2#�����@�T��t�e������n�6[-���4*���a����8��<�5�
��!$���+��z��4�
�F��j������	�5�
�� ���P�)���}<+��pC�]����bf�q�������3������#ew<r������h�(4v�8x�*��Y�o���u�n�a3_~��E��n\��[�6~�3|��X�<w����;W��s-����ii�/�2�	�o�)���N9��^��[�Q����|'�|��x���Nv���v�avu�����~�>|'}4��|6}�6����;����I},��[q�K_���W�*9�=�J�/�HS���e=���Wx�,c#8�w���H/��i��.�m�|�����S�;_z����E����|�p"������`�bG�V�9����d�����P�x0\@��a�tLJK��u���(�}�E�*I$k�����?�5W@+K�`��f�X����a��r���N��:1.kCs�/��[���Z�k��������N��:��u*��R���S��N����\��:��uZmY�(vv��+h��[�^���Z
�r���j�_��^��^��V77/���|�]@EE�������qF�����;4�/�Uu!��[O�������xI?<m��`}u��r��)�Wu��*�t8V�Z���c��&>����IK�O��`Ci���v�!�N��R�#mh��}P.�0�T��Jsp�oT���=9R����]PeS���w\��AU��
W�q����������Sv���:�P��v�E��z;����]��>�~�e����<B�|��6�9�q����
���e��6��<�����b�9����\��&�/d����.*D�*������l�����z�^��G�os0��j9�l7Q��p�y�(��s����d���sW_��h$,������q'<NPQ�C
�)*?�`�!FY��%�
�+��y��+�B��TNu)�;��AR�!QI�"=k�ir���������!�q�_j�Q?MM=��y��g�u'�;p�N���Lw�L]��(�ck�����f�,�w�Sc=c�zNi=p�n���|�v�����.��}g�����7��k�,oFX%.$�����Pp�=�\�1�6���#�r��3��^�m�1��I���X�$��pl&}2���]�x�w�h���r�L�C*�Q���m:6<I�ON���KgCS&�w��CSK���Ib|r���Le����z�k��l�-��,�c�P�\�{xd��uB5��`**���T�#�����0�_��YpB(W'~���O��'9xN��J��'r��P��?MN���9��+5�?���:�������>)!b�T#1�q�A���r"��H�������d�a�X�2�K�	�I1L�`�f�`x���b�SC�g��������9~�������/o.J���{��b8;H:	�����eL�K�f�!�1�����pd&&�8�����j)Z.H0(4;�a��e����#X��fH��5����@��_��4��2�U:��}@�E��Ev��[Y�R���f��]y���@�W�V�g*�
1�����T���R��^��D����>�B�3�`��[8���-N�Q9J�&���8�5M�������7B���/��+���Q[��HfCm���r���/��P[��XjC���/*Gm��"��%���Q[�����%��������q�;�_t����E���o�_X���ay����Q��-��S���/*Gm����jK��G���Q9jK��OP[����%����-�_X~���E��-�_X~���E��-��<	G��ay�N��Q��-�����/*?E����4���#���/*Gm����1jK�����a��%����DS�����a�)jK�����a9J)���v���|������FI���S���/,G4J��#%���Q[����(�_T������FI��rD��Q9jK��OqVK�_T������FI��rD��Q9jK���/`[�����-��������FI���	�[��#%���Q[����(��5
��E��-�_X�h��/*Gm���rD��a9�Q����%�����E��-���a�mI��rD��Q��-�����E���o�_X�h��/,G4J�����a9�Q����%��������FI��r���/,G4J�����a9�Q��HVU��Zu����v �3���6T?�����.GYnd�bD�����J�������Uv�sK|���W
j����Nq��/W�z�`u�k�����������
�7�2s��C&[���}n�3����~�BY_���a�(cr�8��A7��-Od����X��|4n��6Fp�P�#��I���Q��{�n��G��P�$��I�&�Q�%'8����h��H(��f`�f�Q��^(e����x,�J�5_��K��d�^b��+���9O��d�*6�~��?����o��|zvbB�P%�������}w}�����^�E_6,���������w�^��^+�%��Ei�����x�����UM�R���,��>eRO!�����5���
�Q���A����~Y���p������'(�SN�}k8qv���%�3����u�R�X�gY�����B�W���q2�8)1�Rq2�81������
N�s���%�������q,:E�>�-FF!,bI��E����8p�=��_������[:;���8!�Z������k-�ESp��`.?LyAQy�[��Cr����W����<�1�<��GAb��<��7p1�.�k������}x-��=9�-#�l"��|������L�����hn�5^�T���6����I����� U#V�o'8qI>XA��������'�/N��T
�X���dDq2���U�����g��^@ky������:|�[�Rj�;�p�<��"����n��l��bc�>��A�����KC�)�%�'8"�f�����c���[�a�xL�@-���&���C��������:G�������vP�hL�����w���1_��M����	���3����w�|�S7������?B|/qn"��wq������i.tb�BX ��L~�|��>7�&����F��1���j��iMmoy�Ur��:3�%s���
Q5����K/�l4�xipO���r���x���`@1 Iq�K�#��xO;3�69t�_�z�����C4E�<6wp�^��>�r��l�8��������TD���H5����E���M��/S��g�
,S���D�V���*\�C�^�Z���U��b��vW+���A�z���jv������mu�u�]���3��������fv![�r�7�Y������kL~M�Vc=k����e"��������Y�0���;�p�[����..j���GtHz4��C0�'g"���8���D�!c���m2�(�����aft��0x��*����,����_~&�����
O����;O\{����2�B�e�����G�P����^=�o%����jo�#�5h��p�����gHx��a\����q�QniTZ\��J��*:�� ��!E%�2��ZP���i���������
������~g���@\����p2��]P;',��+hsH
� id�(�[Iyr�z��U5��Ry�;Z�v4��h7[�K������Gjf����34��-��R���������OEi8�+�!NF��������ZL��~P������T�P�g'�j1��T���������b���\$����SC�#�Uz<��x::I�8�?���Pq�)O�b��D�!�b��"�?��d���4����WN�T��Pq:�&���$NN�&8n�PW^999e��z����0����%�R"�jL�N��J.�-X����8���
{�V}�;�Y�.�$����������`/7E>�����:��i�;T��i��P�%�
n�>:X�����u�a���lh�����!�1�.]w�[>��&��/�i��U��}���������A�
��������?5�"`���_�-F���O�)vq0�b���&����d��|��������������C�k���	�M�S�->�C ��[L\]���O����O�����!�_�-�!�_�m���#�"��5�"����o10�5�"����o10�5�"����o����o10�5�Bv|��0�i��xHn������!ql\�-��!q�]�-N�m���n����5�b`2/�|���d8�����p�b�-&C�����8%C�1����b6�"��Gs6�"����6�b`2W�|��+2�!Nm�����CP�0��]���C�-���j����[�5q�?4�"�+0�zh�E�)�}Jc�����)����p�Gd0�[�{?"�1�"�1����t�`'��2�-�������n��~Jc�����S2�-�=��?%�1�"�3��32�-���?#�1�BXL�a���������[�6����[�`/��8��n1��M�C�-�Mq�S2�-���O�`L�v�{?"�1�"X�C����:���
�v�{?!�1�bX��	���Nq��d0�[{�{Jc������d0�[{�{Fc�����gd0�[�yv�����[� [1���N���<4�"X$� a=4�b�S|wJnl�{����n��~Dc�����#2�-�����`L��~Lc�E���	���Nq��d0�[�{?%�1�"�S��S2�-���?%�1�"�3��32�-���o1��M�C�-���GJ�-�=�w�����b�#%����b�#%������F�x�`1��F�x�aq�	#b�E���H	#b�E���H	#b������1�"X�y��1�bX�{��o,�<R��o,�<R��o1,�=aD��g�#b�������<4�"X�y�#b�E���F�x�a'�nBn,�<F�1�bX�{��o,�<F�1�"XS,����F�x�`1�1"������F�x�`1�1"����b�cD�-���'����b�cD�-���'������
"��A��Hw�1G�v���OW�(
W�*>lq5��������k>ahw�����Q�`������<)�����B=ZB��"u��-/���>"�	����/p67�b��O)<���F��1}aD_ �cp�xD�<]g�q���ea�6�l����
��7����>��{o%[�Q��sU��������Z��P)"AICu=�����g���}Y�����s�����j��Ij��i���������8W�/��#w��6��oS*_���������Xp�km'�v�\�*�9��������-��N����J6�N���.R��<��g'XW�QM��.4F��x\���>�/���G�������gn�br�j�]���������d���x�L:�/�{0//�O�����`�[�v3b@(%lQ��f�����KOQ��iP��}��p�TZ���F����?Y9��'���,���b��77�����6����g_o�N��:���7����u�X�����6�~o]9������(�>������IvvO`^��h����V���n��������I��L#P_�3d��e.Z�����7�ZD����]�w����.�:[~�o��Mt��Ay�y��]=l/W��h���?��r�����Gau����6�'I���r;�~���w�Z^������a�G�|�������K�<B��/?�^����O`@��X�%|��`��'�Z�]�?~���@��4���=�����Z�-��>F���l���C��j�$5�����-];�(����k�e�9����w������x���\��y6�A+�����������|\�j8��4H���`=A2��. ���l�?�7y���*�~�sn~_���
��Q��E��}�.�+0}�=
B�s8w�w����|a�����)��-�l�$X,�-X/����*���T(Mv���J��~����>�bb����v�
�#Xw8��{_�ix���v�W�"z�)�&{Xl��Gy��|�W���]��ps�����h��j�����f��1CQ� Hp�l"��9�5��1l�[�����u�4����Uh���8��v��V�>2`�I������*4��"�9_��_;���p�rb�`���g��v��jpL?������"?�`\or�V��������)2
/��0B���1��3W[<H�`��tia�p����,������>����O��+n���O����r���N���	{���H��W,�>�1S?���"��g|�CC���x���{�X�Mt�^���� �f�x�5���,�
��_Q��]������������H��*<����(�Few:����n~�D���(��^��YG�J)�H�Jx���<�����:����:q%tB��ver{��9?_n��}
�ZB�6wBT3�')�b�WG����H���5���%��G�_��m�&���l��~eb�����q"�?���d��7)�Y��f��8@L
'�9��g&>3�::�6� l6'�
+Ixy��GS������6���t�.w���o��~��������#J��"�G���_w���G� Ev�Muq����nb��>����@W��T��c��62���x�R'�f�w��fW�V����k�]�9r��s���;G����]vq�������v3Lj���iw��=��]vq���7��iw��=��]v�Y�
��&��ew��=��p���7��ew��=������luw5_b;���Z��.}�Xy~�M��RO�j��i�)�kP=�.��:7�Nj�]0���v���E���tv��>��v��v�����g���vR�"�6��p�n�eq�J��T��v���R���-t�R<���R�o����
�`S���:�.y��5�Tu8�\p3z4y���N7}u$t��@�|o*���Z8W�'x�4����<p�N���Hu��	8��s8���J��"w��g��
8�V��9��+x�������N��	8�Co��9��C�c��9�<p������}���b��	8�JyJc��6?x��.}�8�N~��'x�t��>�����8CR��9g<O�C�����5��G�[p�i;���5�c����\z�KOp�	.=���Q-������\z�KOp��#�P}b����'��t�p���"��g�����Hp��7v�KO��\zf�
.=�D������\z�KO��\z�]��\z�����"7��t�_��\z�KO�\��X;��.=���K_�����up�	.=���p����y�|��7`G�O�;�,W[���m��2�*X��u�]E�\����������y��uJf���V�]g�r���g�u�A���92]�����_�zK
��XE���������w$4��y'8������w��Np�	�;��;�y'8�t���y'8�t_��Q���8$8����y�C�
�;�u��}�v��Vp�	�;�y�{�
�;��s}�;Cw���~��w��s]�;�y�-Oi����������/�G���:8����|�8��q��ZA{Tf�����|;��F7��"����h\}V7Q�PW�F��L��or��#;���=��g��oxwT���A����s	|R�xgd��RM=��Jl�Qi��h��5���P{a�
�M����N9*5�.hw�J�|Ajw����'eA�szR���q����#*xD���<��GT}Tv���#*xD�(K��#�`�<��GT��:jn<����}[���}c7xDu�[�#�`����O��{�
Q�#*xDu��#������#�`�n���/r�GT������#�Ez<�:�k��X;��Q�#�K_�����u��
Q���p����Z.���|����2�Z=,��Wr���8������U9���T�z��7�j�-w��3O��<��� ���d(&.?��F���M[^.1���R�\�7
/To����Q�7J�F	�(�Q���o����Q,mo���~�F	�(��9���C�F�3r�m��Q�������no�����7�>Q��}+x�o����=�o�C��8�>o�����e��
�(�w�8�.o������4�Nj��7J�F���}�#��w�Q�7Jw����x���%8�N�=,��{� �����wQ��F8"g�THM�����f�{�Dw�vv�oNp]��	�d
���:��h�m��I�N��?z�������u�UgR��!y��6)��fXM40&��:�������.C^T�&U��NZ���2�����5	�@�fG"��s%��D���exkS9��6^�[@���*6%;�|�.��^O>��j�uK��?��������������������������Ol�?ib������w�;�M>���sZ#���m���.^����v�13����Y�Z�E���~V�U��M:�qC��K]��<<�U@1De�����
r~-s�>�N�_w�u�1�:�Y�.��=��"��ps=���n��G��F��Z7�w�>6������x����N�+NB�G�����+�P���?N��#���F�;��>Y����F$%r��`�2�w��I�x=zgX���F@�@�;}���v�#����d�o"0�����g:����v��;��|��/g��8\
��%�p�Z���TW$x�A�;|CZa���X}���N}���Dd
/?�y������9*����m����������z�����,���c�T���3�B��K����~�z���p�m��;�!ul��s��0w���cS'Y�<��kqH�<�&�1�]{#jM����8F���:j]X�iw[�^J��1�6�nM]�9��
�[�i��k#���u�����Z�8�A�ot�R�&��[]�F�����f��!ul����umD�M�h��������[S����c���uz�����5u]��:6����5'�x��rK��Bv�����8�V�k�R\��[#\���J
�k�����Ja5tj(������������i�p��F�a��j�Ty�.,�'�����*?~���	�M�D��[�X��#� �P<����:��9��g�(���A���m���� ��_����p"!p}5C>4_a�t3J�n����/4O�t�hGh�	?���l�*J�������������;�����N^{I�c�
u[t�Tg�'��
9�����X���d��)km��.v������qe;�TO����������Or]�Sm;����]'��fO��H�j&�m�?5���,jcyn����e�n���a����Q�C���������~}&��F���j������j�����S������i�k���e��B�\��@:1Q�:6�NLS���
��A�c�hz�:���xF��������tc���Iun$���nS]H��5��G3�NLT7�������-�k��Dup���8���|�'j���
Xk�-kW�|��iX�O��7�l���	{����=a����'��z1g�k��,=�'�!q�^g��yC^��q�������hm�����N!8����1wkc���n����up�����Ep���.C����Zp�����up����Op�>�Aw��u:�[�H�1U4��@�1Q]4��H�1U4��@��up�>(��b:���X���i�"���tb�:�tl��������tc�:�Eum ���.nR�I7����T��	��.nS]H'&���T����i�����tb�:�Aul��B�����Z7����H��|���g�_~O�u8�l���	��=iwk�g�1��D@�R=���-���6�j�s,��:��"fQ��5�Y����q��z@.�O,���D��A�!*�T�&��6�G�;BO�����������W��-)C��OZ�3����g�������B]���:�P�9�P��k����.���:�Pw�?��� \�����B}�#��Tu���k��Du���s#��Tu���k	.�����\	�����cG'���\@����� ��qt����T�F��������tc���Iun$���nS]H'<	��Mum ���nRG'���[T��������qju0.4Ok�X���8 ��	����=aw����/�'�p��]��V���y�h��?��g���|�����/w��M�g���Q�?�g�����Q�N�(�5t��"S[����q�F��p�>����p8Hi&��sd��fvkm���0���#��O�>���|��w���&L=���s�����gGr}Fm��������+�te��sNW���q�\���7<���"x��u�!���w-x�wt�s=x����'x�� ��z�:<�u$���ZZvm ���.�Zvn$���Z[vm �s=x���1��~,���4u���@:1Q�:6�n��uq���H�1U���6�nLT7����S��m�k��g���
���M�c���4uq���@:1Q��:6�����s�Ic�K�q8a�u`>a/����/�'��p�����������j�`���
��V��j���F������������:�J�]���Y~|68>��o��:�F��`�l��AL|�:��M���WsS���K��\u^����^�F�Wo�^�����2��������� x�����������D���^��W����S�A+��
��E;����S�AK��
$x�����n; F�c���e���.r]H'&��<@���
�.nP�I7���[T������&���tc�:�Mum �pn��6���tb�:�Iul���.nQ]H'&��T���z!��xu<i�uc	>���,�'��p�����=T�������j��#���tq���|���%�PK�����f��
�����h����`���/�zC�(���B������-�`�����6���z>��8��oV�([,�Y�� o�]��s����>�=��}��noWo�������r3�����o���x��u��9���-x����V�����������|����jy+F�D�|s���c]�`�.�8h���Po���^���6��$zK�����_�u�:�fS]�!z��O�zv��%��7�?\�g��/�B������!�~>���������6���P�W3�?SmrfdX��+>�|��?F�)���%�G��� z��O����#����Q9���9r��>�p'5N�p����D73I��It3���ef��$��I�13�vf���kn������������4�~C���_Q����5�}I��[5��&�p��e������D?oI�yK��f���;��[b��d?���y��sm�{��\� )|Xf�Q�1>�*z��f@��&Y��fP�1�����E���},#�����6�o8�ETb�Ik\��f/�z����iWz���n��
U��8)����:���������-��[�����7U1��!��{e��nQ����R��C�3��E����bv�����sLh�7�����l�E>��zE�K(�]���4��}e�>y%.`�bt5�?hDd�j
�������DE%}��F�N:��S��6��`d�N��v;R]Y�����nG��o(6���
dp	���Ma�c"u;��B�v�$��w�$��f����G�������
�k�g|�X��&�����4�S���
���)��m�Pr�F`X~����\�P�w���t}��|����*���t��	����?,�$���M�<#�_�J�T��B�;��E��0�|�������8N�����G2�0AD��u22~0�2-�|P��p�
2~�t�F���f8���8W���y��h�'Z��/�hx��@�2�����JO�>�{i;Sm��V���b��\�P��l3����f�;��?�`�{�hV`��Z3D.��e.����U��t0���(���bhZ��q.7�m�1�	e���&�Ho�m��,m�=v�%��H�br�����-�]�t��Ek�;h���	���P�0�H�e�p���|�%����xFw��9x���M}\As;X��U��m~�d�U������������ vrt�H'k��NOd��SN
@_������D�zT��y1<5<�H[PP?QL����wU���Q�
�e�R'z�j%������I��|���5���/-���w�������u���\�����f;_�����)���=���������`�4f>�����h��
������[|J���;��Eh��G���W�P+m���?�y���U��&KJ"E��
A1�"�u�@z��L�P��54.LMg�&(w��P�r7(w;������v���a�`!v������K�`���=Z��u*s�*H�j)n�Jk��r�*���rM/��l��V���wND����e|+�u
�-�N���<�2���:��v:��}:
�r�G�����.j&��E�<������o�o��d6�r�]�)l��[���I��w�����7��X�~�������J�^�RcE@"����0��`d���!L@Pc����M4�lm����������e�53mk������SA���*o�	1����
z�]Tz=z�X��R�z*�#^.P�:�Eo��Au$�d��I��m�K�v>����0��`�@>Zj"R,\�`���&Z�$�t.�����B�~���U�� �gf��x�h�����m]�~2>�=>��D��}8�O��1��U���Bt�]�&���+����Y�:�+���!�bGv�!�b����u�+���!�b����n���O&x�a��Q�+��Bt�������_;��ny�]��]1DW������{�������L�_�)�y����hCt�J����!DWl��!�b+����Bt�]1DW���e�05�����}BC
�������v2DW�Ct��.DW�i�Bt�]1DW�Ct���"�u%z`��.��S	���hUG�����]YTO=�9����{��fuGC0.��m�)[<����P��q���E},�����_��+���a����:[~��9z}�	�Q��4���8����������Q������<B��z�]�-�ul*�c�x�n��6j��F�Om?!$0d���naBA�Dlc�B(�
R3�!�C'C(�
2���H�Nt"��� ���B(�
2��� ���
��D<��=�At%p\��@;?���v�;�-�+BA�P��[8x���w���
r�	��!$%;�38��mYi1<5<�P���0��l�!�#\BA�P�>���7���S��Oh�A����C_�N�P�!d����P�;�\BA�P�!dy���Wx���:����z*�z��H����
�+���������Z��$��t�E}��� �������|v����J<j���M�gT��Q?�g�����Y4��<r��:�_�����8��:��=��
�����2���|B?C���q:8�pp�zQ��~��1�*�c�0�%��1~��{�A�?�����C3�|I������BH��1�d!w�~B26���1	!CHF�!$c���uB2���!$c���n���O&��a��Q�+�BH������tz;��ny�]���1�d^��k������������Lpz!)�y�A
��hCH�J����!�dl��!$c+�!��BH��1�d���e��05�����}BC
�������v2�d!CH��.�d�i�BH��1�d!CH��8����u%�`��.��S�9���hUGb����]YTO=�/$�j�`q�(����j���Fg�(f�:_ �r/DX�r�^1�5�h���"�c�������d��(�c������H��B$��GR4|9���X�5=�h���*DTCD�Qq���!�b�=
�Q1DTT;"*����\W!�b��"*���]�f���d���� �-DTly���hO���������CD��t�nm����h@_�������Q����c�i�6DT���BD��z"*���Q��
.DTCDE�
�Q��S���	��'4���
����/h'CD�Q1DT�A�BD��v.DTCD�Q1DT<���A�+JYW"��rY=���=��Vu$d`@O����E�T���"*�����bA��-�!�T�n�O9�C ��Mv���H$F(XfE���1 c��f�R ���A�Iw�/����`���A���q8������i�.|Z�-���X>���]��{!���k��i�v8I���S��t3���ho�t3p�}��^�F#���L)�[!Df�Bd���m?!2��Q����!D���"3�����
!2C��"3���B7C��'�����(���z!Df���D{:!�D�<��Lt�Bd/��Em������B���u&8�����<���Ok�!Df�����Bd���"������lp!Df�Bd�l��2c���NMP�>���nP�v}A;Bd��!D�zBd��s!Df�Bd��!D�F�z\a��2�g�������yL��#1 zB���,�����"s���2�I����@L�P����G�L��^D��6�|-��Zl~��R��������i�)<�`:8�=� �!>f���b|����4�O7�e�ub]�X�!�e�ub]�X�!�e�ub]�X�!����
�.;����$���.�������R<��=�At%2^�u��@;?�����;�-�+b]�X��:�C�������r�	��!�%%;�3���m�uYi1<5<�X���0��l�!��#\�ub]�X�>������S��Oh�A����C_�N�X�!�e�u����X�;�\�ub]�X�!�e�uy���W����r����z*�z��H0�����+��������d��3v4~��q�u�^��_����6�P���w��<���Al�u�j���������l�k	+�b�����$>�.��?���9:���:�gx����:Gw�c���E�V�=&M|G}=tL�c�\v���Z�����K����R<�Q�.[i�Z��-
'�y��EN�1�Z��uT�~��G������Lp=B8>���/X$@� �gg�(9��4������~��ip�*��AoU[O1
��nX,T2s�ix1�3qSe2_�.���p�V,�g.�g�W�)�f��U���hP�,�y6Q?���#�w�O��a����Moj�iso }|���W��	�.3x\@n�
p	���R.�p����O}SRf
b�����,�%<\b�Ky��)����h�F�����������&`�Z|-�v#�x����FM�hC]eL���R��t G��*)�]X��BF��V"������o��[<� ���u�M�D50�!�� �a�
�m��}���������1Q;��IuWy���kf�fjT����F�f&�k����o����~0�[t�Q'���������2 ���WQ��>=�����`j���y)<\���b��g��N���1��:E�����([F�o�u��K��p'��Bj%l�3���J8,w�5	n��,q3���l
�[,V��o��G�����L��H=T>�&�;1)M��g�[���h$��	J}������_i���R9�<�Ec	�H����Yk���E@���[t�)�����G�:4�_�i�|�u'P*�S9���{PL��Z{`4Z����
�7�?5�3v����[�7���jo���/�����qf����__eoBR�
��`z����E6�\��$sHFB�w��Sz�O�D��M�c,Q���;�t ��m�����e���PS����&K�7(�P�&��0��5X�a���Iq=#X�D����7�I��Nk�������##����y���C����<��M�����	�S�������DN����)��a�
���O����`];������7AK���Q��o_]������/*�����A����>[wu���`R�`t�����w���U����!�.$������A�L����/������~i�.M�1���jk�-�=B�]H��p'���yZ��[��A�y�~Mw�������L]+)]�D����P��'}n�M���p5c�31�&������
���uf�K�&=�j��s�kX(du4��pO��^����C<E.�� k.N�)�o���cg��"<�X�z���0�2�h��xl�p����4�������^Q�Ou
�i�y�j��B�lt��TC�
��,S
�6�6}^�P5��PX�t���n�&��V�R�d���i���7��������6Nne���-�z?'X�`q�G�UK�4Z�����pd#������kL~M�V��3�yJ�2��\Kp�u�i��Zlk�N�%���1�J����9�������
8����E���zumQ{�5�%�R��'�O�6n��dXfi�� b�#b����DO7P�X5h���>V�x��P8���J�ik��)�i���f\�I��>��*�]\���^c88���IJ�w��,�w�$�A�i`��yj�>=��P=V�8����Q/>�����{���Gu����?]^�_B����l;��7_C�|,zg����7��q��7������;���� ���d2���t<��0�����i2MG�$p�p���+��m�����mn����	����^?@K��i���<�&Yv3'����f:%��|4:�
�����4�~Z-��=Tf�F��rqV�����(�6� ������?�r����`-]_��Vw�}��?G���Yt��T���?��?����p�{�������������-J���C�9�f��T�=�3��^�����I��*[/��:���ab�hu�yG���E~:�����:��|�9�����*Z�Uy/��e���K���r�?W�����$���o`,�������e�mD��=��u^�����C�/���dPj����-qKp��&�z=(7�[�|�a��4��q�	������c�K==[�?�����=�A���:��
�C\7�'�	��,���C�h��^��Zd�����
���5�Du�D�oP��!��-��0b���$�w�_�8i	�D�s�)���x��a���/��:���9#`�W`P��'=4�wya!�I����5��K����a
;�����W3�	q�qmVw�!����L�z�l��W�mN�e,W�O����f5��g������������������{�0��=em�"�q�,h���c�]{A7{�n�z�I��'��;��@@Ka
�
,��z~�@���qF�\M���/���]����z���Q�K�`����~�+�I�����L{|�x������K�����[2��Wt��o�<�w4u^��������<�WI������
cPD���!%ua��`*MT	�g�\��	���%���v��k�iWb�V"��O�fu��E�&�4�(�)�� ��YW�E�v`����@6�~�h0ws�?�wz?��4�X0��8�s@V�:���|�_���.�"e��k�+hd�!��l3� ��_�[�������lV���Q����s��,jH`6T���@
p�����������l0\%�����?-X����iB�����+@^!����Z`M���c�N�1�0�_���oS������9�.�!�	Fs��������}G���p��b�-��u���{pm|w��f�}�6�c+�3�w��\�����o���+o��
��H�����k� ��;�~G��0w���{� �h������f;��@���o6�B]1u��k��Pg���}g}Ph�������	{	���u�������1����L��t����k$��W.�F/���U��H����������w��5C���f�b��S�9-������G�u��F���S�Dg��+L�"�~�.tM�=��h�|Jx'g[�f��������|��~�`���A�O���5
H�|!��������>����N�G0�d/�u]�7������o��
�W9Fu���ve��O�����i��3�
���`+���lm1�p))#��a�G=q�@^&�
"�q=$�n��x��-�G(_z�%x�-Q��6l�BZ��+��|Da�N!#	�`��	������V�P��)G�X��M��z�1p��(jL�&��4�d��}������Q�>�A"��Q:o����	�pzd8��p�p0��l��q1'��fp�$k���k���G���t�����$�v|y`�4�qM%���s3�{���<�+��MO;!�g�wO�Q��������Y��	�khV�l��B9�E��]��3u$�!;/��\F&���d�>��[��H��������`�5!�>cH��B��E�3��=�HLoP'�>b����%!���WZI� *Z���v�$����
�
��u7�K�����#��y�C���&��j�@`��KD�A!���2�K�,!p����h�:�4_=lz}���<�����1�2�k��G����}�Pt��a!
��ZN���D`�x�����*u6�&��F�����GQ`^��2���!���|�/��1�
������("A���W��
zp�n��Y�K���j�O��{��Sx@�9�c�{��U����Fj�z����wb�J��	��e�����h��x`Oj���B�M-0
�g8V[C�z���B��B��0	�.2	����h��}}��~���_��m�X}����,���Q<�����K�ou�b��kP���k"�n@���:�&��c������E�}���/D�����/W�9�O�����5#�-F�	m����6_���	���m*_g�p%�M4�N�P����v�6����E���z=�����?�����&���������,�������,9����p2��.o����������l4��?�qJ6��n=���������Z^"���/�����[ .�p�q�� _�B�G�<,g�^��\�����.�
S� ��E��|�|X,�^�h{��)yn���2��C��*�oa)j�*��x�@N�v�s���/I�w���<��:6X2P0�w�����jA()���*�bx����i��K	�R���_�19��?�(}��_�u�����_f�T�L����5T�K��>)p4�5��_^��{�5T��}	����p)}��T;�*�'�p���O+0��r����):��B�����I�W�$������tX����Q����Ou-zG���6i��v
�]�R��z�1�ym Mo��m���/�'������gc��;��g�'�����D���DD�+�@;-��DzJ�������^���w��BU�D������B�s#�@��B��pW��,v��c���tT��{W`����f:�Wp�`��_FO�I��*D�aq��=��TF�0�
q\�����_�����?YZ
��|��'-���~��W�!7H��i4�1.�����|��^H8P����:�Wc���M^6Q n��|9��4�����:#�#������_!~:O��V�I<��d�xj����<�`��@M��/���w��������`M�@��W��{���wo�@D�7h����nu��x�D�|�������4wx}�����i1��P����8��^h�@{��5��e5�3V��b}�b�"`d��@�C��`D�����l���J#X��>>����\UJ��������������-{�~�!:KR:4��<��t�Hs
+D�U��D'2_urJ�
�h`�nu#u
<��K���P]	`����TY[Q�2S��C��u�X�9�*�s���Y-�j����&���a�����s����	����g�a	Q���CB"�{���sM�������=�, �2y��W���|��J��:&������'h���>E��_i���)��J����7k���A?�K�R��?E����?����^��Z���������7�X������~��@C����b|����{PT���<��(�&z��?9��@�7��3�y�T�<���>C����������~��K���g�2��
�D`�C�
u��~��>	��P\�����bK_����
�
�\���W<F$!��2,iT(i�2R6�Ev��,����!�7����SEM�=ks������V�jU�`��cZ�����1Q��R"�gLw
�+F����
T��EL��{���?�.�T	2F�B�b�/?no�J� ����X��u�+��?��	Q�Z_�9�_��>��D��xV�-���V���G�@��
�
�,[��o�})�H�&$|�������>���8������7����y�:�0@T�.V	�M�&���|v���{8PF��q�`t�Z�(��gRs�F��G��%�[��`�8����.D3��V�A�����D�a}	6A���*Z~��x�z���W�[����?�+�I=���T�!C�P�F���M�]?�` ���M�$p�fdX��
�wr�S�V��e$UL ��������}�|A�=����`��	-�B"�,��pdM�e��T�,�#��h@��gI>�=|4�(�7j�ph��r�1��?�;��������rE	%T����������J�9�UA��sy�
�(��$�Q���O�Sd�hqip 	�K�e@������-Q���b���`=��5r����od�*C������$�H"i�~����X�������L��7W��z�s��kDy��I����~D�H�cq��"z���������2�F}A�
z��f�}�V7�U4��NE9X]T��e��u�W�l�4X�X��]���W#X�����m�e@O���[���l�CUcm�������U6G���!�"�n�V���8uoP��v��
u�@�{�_�r�4j�H��������6ZiG��f��p�$���~�9�8�8�T>���f�	�$9�XqK���_M+��`��*�u���O�Y~��3dh&-+i��J`��+g�z�/�J�����pm��+lsfz��S��"aeD�d�/��9�9*��=c	Ci���\D�����C�EX�|�c��k�,oyRU�[k���H���8��)fLC���������UR��e��D����� ��gE����}w,�5��:
�PH
�{7��+����M
��[|^�h���3I�5s���:������F�?������p
Z��r����������
���V��Jk�����Q���BH�d��" q/��a9���� ��-��&�	WDVC1��/�po���-r9�X��%����
�+3���B���r0�J.Z%���H��3�����a�2���������gg&=�<�8���XOk�����cd���,�'�+����I���"�N�}��|").uE���,�,d1�VBF�^���/����:j�������r��up57��j`n�N/,g� �����V=e�.�n
zG�0�.S�s�:���2 ���o���\�nr�� ������:wi���L?��}���d�1�?�������� 4O���1S�����L��?���>HrwR(�P���:��%:xuS����������4��0��q��?�r_!�l��Au�4��b���`A��K�����p?7�Z����b�9�9*_����*�q�u+�jRN���g� ��}�Q	�Y�WQ�%��9�]r��:;�`��c�w����9�E�+���E�������&�.*�[-�yqZ�`���|���2!�+�O�o]}�ae�/Rc��UP��Mj���J�Dd����U��s���Cf~&�t�%!���h���7B4
^��2C>���_o�6�� F��C�A8�����|7q@�"��e���0c���Cb�PGP�X�|��
t�Ej�6�3�]��$^�4��Nddr�t*��j�C�?~�$X�����N���T@�7���l��v�(���_��wr132���fK!��%������>�2
�d� bHT���1�=��[S��!4+L�D-�L������ %m�1�O����H
���^`f��q���l��$�7�i����U��p���I���~�3
e@�7L�|���=|��X����6[�Ae���t
lr��>'��QHXE�Z���#@�x��p�>9�W�gY�*ON�W]Z�_/�}�?t�.#B�E�.|��)��|��Fg�����RR�O��.�B�%�X_����,=���H3H&�b�RF/h	����*��X������u,U�n���:5$m����@;P`�AP	P�l�| 1��I%�.�L�B(\�B(�o���vs��S�8G�TQ��|�����P�YT���ra�H ���l������O�� ���#�NCA�3i���#�e�{.[��zN����g�9�)��T- �����?��3��v4������cH.D���`�:����awH�n��a�#��@�Rh�0GB����Fs<�L=\}��"���W%�x4p�i�XjB�Ig�PA�u�u^,E�Y,(�,	�?	���������w\*,]
P�[H��!=��g��*�����}��H�9V�7��dZkt���#�3Q�t9"F?�{
������'���G��B��5� ��sk��F?{F�v��s�|��h8��P�0��o�����#1L���,Nk�F��g����+���3�����#��D�U)a��e	���G&XE���8*}&��'���\F+aq��T��\�D���"��h|�z��g����1��������j���8��|5�4���������>Z�YT��`#���>s>Z���o�.�u�3_��P���dG�W|���+V���J������r���'j�}{Rt���i�!;�����J�,����������*���EQ�+\�L�Ucm�F��������iK��^r(���iX/�����������T�G!�>��(�;����0��B�=��D*��#s������r�o6Jo����8~0>����[3Wb����T#�&�B�1�l���U��3�����+Z�&:0��?���(\4ATuab�0��a�uY�M>�%	HV�W5�d�t��+�	�8���1��@�JCW�8���.�c�&�,�{!v�&�����<WIi"���95
����K��]),gA��d�TI�Jy�b�y=���?"]U�J�7�]����(�=p��1��q���oL��Tl����7�?8��p�h�}���T�����w��d��n�>c�����f�U5�<b�����j��Lk��e�]~��FG�xs��6�	s/�2l6v�����%��.�B�~>�G�L,�)����v�F�����I�9��b�\�`���8Y�w
����`f�?�E
7�(x���(�����3������8u��ALK���IH�3��}����a���8����Y��k��p�\F-������D��x�R?��w%`-�0�amX�J��\h�6���]���F`���g����flm�f����nf����J%�N�Q��	�wie
j���e�$�B�����@��<���;,d_�n+3:me�w�[�D/���V&�/����5�(�v��@F�x������8�La8��_R�f#?2jtm�$�Z��������S����q|CB�_2-{����Zy9uJ����������v.�1��%�?:������]l�5�`��b8�������<}L����j����J�-�hI�%�<��z1Y7J���t����1�}�8�ef��L�d�(8
;�����D������T,�4?�Sj+�'��&��i	Gb\�J�OJ���������j�����/��
��Dg�PN@4
U�(����0��u�~��) C�B�)96�w�J1
N�4�%o� �NI��T}��P�u(��l"hm����1�"������ �C��������>�a�����.�[�"d��X�Q#b2Ug7������g<M�G�w���S���A���g�-�9�!��n���
4���\:�6��:�������
�fq��/V��[��ZKQY�Zy/[��
%![\�
��f�^V���������-��hHJ6#��������r���U��� ���;���F����%7�w�������63�D�EkV�7��05���u�m%����O���dh�mo����
���va�
��2����ul�a	I�=,Q��kQ+I��Q���Y)���vKq���%�&����{�3&_��WFc����|^�4������s�1�\6&'*Z���E�p�K:;p���IQF�/�s���p*NB:$�
�����)F�z�"k�������Z����� ��G��a2f�Cf{
�3�h0X������N�4���h���;�?�(�p�z+k�K��3�F�(�>�����:���%Y���A]�g��{�=l�(D����q���iNS���OD����������q7	����O���	�u7��B�"H!�1�M�-�H�_��*T��7��Nj����QFH ��r0��<�����d�lq
��y�)"%B�CB:���+���f���E���=���Np��Z�XTH=G���|�����@d����$���'N����?d![T?�H��#�����{��#2�����c8�D�,�5j
e�'5�x�U���Y!J{>����G)6p
��5?����<�����5�6g�d����"m���
Hi�%z8������9�[k���5���:�M1yz10�����&�%�)��&T�/E�����=6wI�N���`"	���"p�f�7�*��8�������bs�7�bBl#Sz��i���t@S�no�F�=<[	������k��ta��
�SW��YP�Fi8�r�n��������Q�|�����Gk �����`QK���('L���`��/�o�w��+��D�K�������{���%H��o������=�l;�1���9'S6:�y�t�;7J,�C��u��g�>������l��!r��Zg1�I���w�p(�b	��M�,�JO�gQ����zu��FD���FiP����f���2�Gt��y�B�!`�����
�.�f:�Vs"xn%��%/�_���p�����<�������]�n^#�������lk"�m
��$��VV:C�Gd0$��D�tW�[�o����
��#�!
D�X��m
����^l�P{�c�����f��IF�j#�����4�V��|(}��A���q�u��*L�����4������[x�<�X���xW���>�M�	���p����F�;�"]��o�(H(>���P����$��aK4_�<�����l}���DN�a������e*���|��B���s�3����L�*�`�7�����5��v����[|�����~+,d\�5�d���:/��#KyY'�6d+|a�����<�
b�6Nm�o��|c2�^,7����bVl�r
�J����x#�?h����
��|�(:����}����J���nET\����#�4�"P�� �is�����t,�j�W�[��@���>8�i������
^#���I���7�d7���f�����������K��`.&�I���*;B8'n[�=���>L�XD���7Y�M��
"��#a�)��j�E(�r~�$�b[���
��������TS���g��\��F�KbZ���@|���'��n45�5`9:s#�;���
����5�����I����'!f�h!��_�[�$�����nH��T�G]��.�lC�%�SQG�m����i^B�,Z��p���������8�P(�o�j�3EL	�M85D��_��N�d|5ji���s�����	���4m�+��J���H�pB
�\H����l������KYG
`��k����x������g+
�4�.gF�U^���%���J�rwbY��!��i?����!#���d����k�L���,���`3Ql~r��������v�[h.���v�|�����Pc��x���Z�&��
�[�K'y�/p0)��}NDw��`s����17�i����+�������e"QDhOR�����[.�]���.�K����?�/G\z��l@S�\��D��\I�����A�:�GS�1V�M������bU��_L����u����/�|����M%�-����<E�VC���3fK��
�K����%���Y\u��A�(�?�VS��O���ie�1�gi5�`�_�X���Z�C���3]����`}���I�t\��% ��������w�[��8���il��G��e�4���'R��h*���X�=��J�p�0�?]#%��8fS��,6���_B%�<$D7D9���Z����~u�����+�P&��#O����o����?��%�BL��\;��,TR����# �8]���3x��V����1��tz�����Y���,�<��I��[HR�3�����E�\_���*3Ce^��x����8�|����������������B�X�JYy���i����8N i�W��d��z{P,2826���F�x�>	|��tb�W�&/"JB;i,��}�F���zf���!�P�rP�N��=WS�����DHX^�
�k"�/���V���{�����jA���"�rY|����(�����wc�w��,
�%������2�(C��(�U�� 92��sDA�������J�D~��Wm�c�/<�u��^��;�}"���*\g�,��Y
e�l�:K^ �M���B�/Kx�A�.�	��G�gh0C��kh�r�pw���B$Lv*�o�)��]����wqM��{]!V�rP�a��������t&Y�<�������`����{�K~������_������he�k���`�!��?o�w�y�\��`��
��3��Z�'�b��
�}4�#����A$�fuGz��?7X���8R�Z
Q���S�b��+HA��Q�z���"�l�g1������"d�B�s��9
{����K�\��$�6�$B��{���&]jl�
��V�i�����o+r��m��9^pT�M�`t���\hJ��5�Y�����@Q�|<��}||������J�i�a/Y�<[���b���� *L�L\�M\��7v��pf�VG#�����+FZ�b��*r-��������[0�n��8�2�����N����-�pQN
����
�`��;� ���(���g��
}2����wV.N�1p<������X���D�D���������H|\�	�Jf��rfD�^4%�1/�1B��w9�|����sG���;�e�G��<����sq�c�P)o�%%��`�YY�iW_�����g������$j�����8�h�������X�i��:�z��X��P!'	W�|#�QT�����7���
�����_�
�\����~�:��\��5��|�Dd���0v^���C[�m�R��FyTj��2
zW-�9�	n�h�|�$���}R.�:�,8�?
�� �n�Q,�3D��B��%�3�"M�)���s���;�:S���V�B`!1����GQ/jMB��gf�B�&*��6��������&fzr��e�l�E��h�h��v��\��{�������b�%-'pH������
����2�P+�c�6f�H���%��)�~E�b���T=w�dC����)g0f�	M����[�-�g���z�lP����3@�T���n_���a[>��d��[�k�Q�.�[����Sf���m"��f�����2�)�������	�!bI./�~��o��P;Us�3��aIY���4���Y�.��0X���������������l����s�_/�s[���YRB�g���Y2��s���T�%I@p�.*�+����#��s��_��\���0����b�jR4����-���Wp�YC��<K�9���jXf4�e!(��D��U�U�o�FH���K��75��9g����H�#{=��� �2.��W�����*�:�B5��"Y�8�K-_�$_Es�ql��"z#���\m��(���oA����0���r����j�3�����0���E#(�/��{���������F:����7��f�P����<H��x��|�+V��n-a~���$/�����P�p~Q%�/���E�OV�iYA�Q�/iX,����Os�A����Y����\��u1v���\��u�����.���Eh��J��������T�4�����-��8s���v/��}�=���Jk�����X1��*UBsH�h��l#��C�q�����!'l����R��i@	V��A�I�B���<��<X4�����2���V�)Z�V�b	td���s�n9�,/`L����/b�(JL�%��C���r�^CP�J"��BK����b�w�=f
����Q-P��DW�BIz,e����wo���������>���%"H��iw��i��i��Iwwv�>z(��5�%�$'�n��pp# H����m,�n8����}AD5�\	{cw�����@Y��P!REy��5$|�0@���O7���i:�-[�=]]o�t�?���u��|
�B�r�%bQX�kS|��i����t��P��
����tBkl�Z��Z������Qc�b_F��$�OSM���b��Z�Vd������l��v�)Q�6����l6O ���
�r;��mN~�w��FvK*����Q�o�cDC@�EBn+���D��b���
��0[�F2a�"�h�zn�Y����
Q�)�H#����&]1�"�/7 ���ly�X}@g7�X�f������B9.�D_�.o�"DO��>�Q��^6�d-���kN���IQ
�h�T�#+�Xn���^��H{���'�'��|
�C�����7���VICt�<��$E ���-W���Q��q�
C���	r���K��i@��
��J6+�����w�� �x��3�Lk
�z&:Nd����bs����t���fo��
96�\�jwXeX��av���5�6D�w�����io
)1?&��^�'?����m��s�$>�l��q;�;N<j���q���Mw�&������T���
�n2[2���]��H��q�CT9)}�]����Y�9d4��m� ��(�+���F�r�!�j��l����3���������5L�
����fJ�b�����hT�����(]\�����6]rMa |���Y?���;� ���
cY�L����:���F��0�q�@��:TW!0�s�a�����?L�A������DH>�]�0��l^xz�>]��K*���R�����w�i=�9��rU�r�G�te
A���H �^�{1��/�-��X��7��Wp}!,����h�B-�S�T���G4���D�R+(���@�w���xh	���,\��|��N5/v7��'�Ol��#�T��4��	�:��n\g��;+v,�r<F���
�u����mY�G" %�E"��Ab��#����Ue"��N��Y2q��dtT{����4��������h��8�lTz�F
(��1�A��d_v�(G�>Q�x
B7���D���n>
"�����>������|z�=��������`��$R�PN>���`�c@��dY�d0�WXJ���U����F_Yx����>�,P�N�{i%�O����G �>yR��s���Ro��9}^���9����I/nG�;Q?Skk��o~���|EUBw��364���w������lSM���d-��w�%��T�&r��m�������&��������m�
�|C��!7|�(q��i��|��,�-�u�$�
I��x��@�K�^���g�x��/Q�������'�S&F��tI�Cb�7�{��~\}����B�����|���qm���m���\A�&Jw����"��T���H,��\��R�d+ej�����1P���9����\����mm^��+���Hkp���z��3�L[�$s�R"+����6�@Y�}h�obm��BK��zu}��`��$�	%lM���"P�_j+�kH2��O��AQh
OA�a���tU�V���.P*OP;@[MPn��xg������q]��w��a��f�� :?�U�|�j�(�M��."�	|Fl&�M��c��c/a�N,@ �W�������J����o���k2g��v������=b$�%�1v��G*6�[�h��`!���U���N�LW,�:�A�d���R���_M@V�$�U��T(���[���2<��8�����8�v�����p<���i^����p2C�����b��b;���M���z����m�*�e,
x�����m���\gtK����P�������f ������	����m:_(l�K�,��$�C9^���v$���,<������Op�9����������y�����l����2��$�Zt����L��W�+S�� ����Q4�3P���B[
�
���Q�\X��T�i$��PU6��g;T�fp�.W����)+��=���[�I������\H�_�1Dw��UQ[(�U�-*�
>�w6G�N�$�>��(�i����{Fh�T�u�L	�$��L#����aE��'�u�]�n�Y�|A%��]'m������?�sFt��#X�pd�<
-�J6`������'le�}�P9�P�6��u���,'���,R}����|,���
� ��8Z`�hG��z/X���q��?f��� �����m����@�@[UW��#���p��|S����4tA�P�I.�	���Sn�������m+�9�����8@Tv��f�����Y��+�v����hvo���fE��j��
k�T%u����&���^�*y���D�H���r���������$>O�{����O��7-��>9�+����9�;!����.i�:E��_;!�oX�u��_�����~&�^��������E��7l�i�*z
N�x�S��r����JC�W�0���J�m.2��������d�������)z�'���)�^!E	�N�g�����*YR���Xj�=Y�1����dF��������	�&���[��<g���mo��r���&�����&�Y�CH����QA��m�cF0��{��P�y*�+,4�1_e�h6��a�;���)�����+���8��t�@�Bpp�_O^��)?����H���)
k�ts6_w/�������[J���4`��`k7���QN�6�G�.�Wxm��	46����XL��<�\�Y��
��0A�;��Q��)�oS`�5�B����T����q�@+���r�������i+	4Q)-��Y�
��q��jD�aw��7:\�����g�'g���*��~���p�]J6g*VwV����3s�����`�:���
�
��e�B�� �z�A����S?��^�<�b�$���cR.����O�wpC��?�����js�*W���^/��wS�g��Y6�}�����x���SB��Y:�	�����f��H���3�?�d6�:�c��M	�?�1�N��_@�Y_�	�~6��VW�4�&�=C�n�����F(J�%�g8B�^?�5��L�������|���~:���	�D�
 0r�[�7/,�|~PMm��D&^���s�� �"�9��kT ��E����`Vt�Vf
��|��a�y���-�|��)*�=Km^c�����<���Yv��M��t��W����H��e�UvRF'�F�V$��$��2 r@��x���p����Q	h�c�!�@���X��rmC����	R%
�P����y�i����������#0������H��9�s��v��'�����
����;���	&�����L������U�����hl���h�v9�6������f +�j�m�'�Q���,���x8�
Fd����A����g�A�<����9A<]�Dkl)���0_�r�e
�?��dhZ�+O��i�NX�d�d"�v>���t�g�8eCxn
Zb.k)}}��["�
��~�����y{%L���2:�@C�NS�k�>[St���>�D7�d�/��
��/!�F�J�3����������8��F	j������\�ia�&�o��)u>j��>�u����F�C��%�kM^��2��(�	e0�a����@*u�Q���0���'C��`��d��mi
�����l�������^V���x��������_�����w�7�~<9��t������;bGTmo*�@���O��bxM5�O��X��1�vP��{h���.*�����*A
�����h���� 2��4�~������4���pj}��9�]�),����y
�����|9� |sN�?�h��B�1A�����#P�B�Jp~ss~>��(�@��^;7�8e�?`���j�y�~J?�
�f������Yh�s�n�Y�OW��-`s,c70gB�f��J��=y�N6�B��2
�*p�����l��|�d21^�t<�5k�����i4{z��X ��F��|��G��O�G���n�����js�U�����~�zO$
n��^N`m���)���[�I��&��M��H��;�Q&����1+��O_,���I�W��+���W4���0�����f�IZ��&�(;�[}R^n���}VN�E��d�sm���3�E��%���B[�OE�	
�1F�����6J�p^f����&7�!GPV2����7�!)
��"&�I��C'��
��16��O��V�)�`"�d���6��ka'�}������L��!��_(����8'4�z����}�	[wmr���{��n�����i�0I�����\m��I	u�B@���EU�G�f��%�zo!�k~��T}��g�3�<�K�F� ���>��kp7yg�Zl�6�E,]/��w����"�e���7��2LA��=�`]����v���*r��/�-����]mPH���'f#��m��v��,� ���j�>�9?zv�!����8�hTQ������r��0.M�]�i�OVm�Z���R��OT�g�J�l�����i��&����M���F�����J��v��,n E|���_�� �_G����\�m�,Y���eo�L�+e������������CI8�`�Jv��H��������Z�a~�����a�o�CH	����j4��
�G%O�m�%�J�1�X}4_�E�t�A0�l���;�i�(���
��J�D�I�b��������@��y4�%�x���@��]]O�E
j2����������+9�����_m2m�<�^�4��f��{�z���R�j{M�K��dVI����T�@rz��87�
���F�Ad'�8Y%/ �!X�b��v}��:�H3E���bB�����>z�n�Y���d�u���s��s@�|| �S��LI �(���JI%�%���6��=��|+���n�T����6B��0����ZT�.����f����@�4c���&��	j���e�LLP��<�|NO������aN����Z*/Y}Gl2�
����1�W�?R�CbN��*!P-W\[���,����$�@�gYmO��p����4��P
��-&��@�r@D�[_aM��(H&�\fT;c�3�o�Z����mP��G��*6���� �4���V6�����lC�VRDE��^jYB�2�2�]�L��
����H�
1K
�
g���B��YC,2�B����-f�=�B
2��s�����"���
���qCA����b'`�ho��W�6D��2(�}#����%=�&v����,�cx�1(�.��G����P�aHHu_�=��/�V$����u��A��y��J�-��(�N�dT��8����*����Pv
I�j:�B��|�����e"��Y����:!��_���a���txI[�zs��Y/�r��Po^�����>n�����G�	�2�lE���d��=d��L���""e�(������8���4p6��(mV�b��qB5��-y�,��N��r�`a#�w�7��gdSl}�����7t*�#+���j����w���{�!���u"�a���^'W���������?�~/&�5�<&�`�s������Y<����t����������gg��x��i�����S���������e����?��t���Em=c���g=��N�q�����+2������{��_��z������T�t�]���T����.�.k~D�8���o� �o���	��B���P<�	<���`�7������<�3�J���>9����i��{���+*������W����z}"���>5�{�eK�.�2�����'c����[�6!����i��NO����w���:A�n���AvM���'d�<��:�o��h:�f
K����~��~������_N���o~"������z�5�S,6gr���1����@��8�A����fP8}
�������w��kr�������w����E�Vj�&������z|y��1�v�M������P��#���xrz� �2�'!
���h�[{mp�G
K�����'���}&)������\������Z���j�X�n�\��K�������_l��k�� ^��D�w ��������$�6}3�}�Ne�=�cdb��O��I�+�&��p����m��#��L�m�Z��-�-6c��������E���`���������r�
�����Z�X��k��L�|e��f��������Z���j�XCo�\��K<e�b�r�NFs��Y����6�������=�`h1���4� f&�$�,�\$����Ev�m���pJE���,m���fN]n�L#O�|�E��N���W�/_��wo��{KP��9l�n���3�w����M<���vP���&^��4�4������c��~r
�	J
��,������_i�k�������j�l�Bj�Zm8�������������3:�V��X��F^)���
���a6�>�n�<{��t�����
c
`�fGo��[��[\{�5���O�
����c.�/���� �b� R�dG5Go���TA@�I�;����Rk1mh�q��+�0������apB���{��6��Hb�J���4qD�)5��D�#r����:�������f�6�/�����E���6ZC���A���:�����#�	�@�D L�q�u�Eka�U&4�u�(Dq`�1����V�T2C�c*p��R?��z� �l�9A� O�>��>��IL��G��<IX1��� H��8��*-�?Z+�FSJ�a-��v�?�,��P�y��C�G�P�y�t����n��J����3�p�����Ox8dE����'��a�&��&��bb�������qe�f!�g	l$�1@�/�qU�1�"����|�� ���?��qril�$�vJ�*4�������w
�����+���2v�����t'f8��Y=�k��������c^�:[�f}�8�B�5��� �^`�ug��g�mw3��W������e+}/0�������b�5)�&zD����v������;�.�el(�~�����v���S���YT�=5P$�]e��i�-L����>���}���A����^���u%����g}������z�V��V(��;�}z~�S����<f�q_��Y��' �1oC���y���Or�	+��6�I_��>�����|_�lF���)T���*� P�z������{=~����x�|�-v���oE�[�%95-}"��l�S)�P'S�S����'T��N���N*2���O�xG�XyO�ZdNnK�\��:�����(����=��@}g�������Pi�5���L��bE��4�T4n�}r9h�cv�*����?�X���)�WU������.#���}�y8z��"�+Gyg:��MW�}�6w{/?�{1*�]����k��3��5����Z�8m�������;�8;Z���Y��-��j�:�V�s�VT`�y+b��F�D tR�����e������l���)��l�?N%22��R]*�g�7�`L���I�������;}x�9:(�Me5R�:��CG��u��)�N��2u����*X|��a����9�7�Q�q�`QT���������qb����M^������6$&���U��u���+L����x	8,�\�Y-�+gr�S:f�^^������
/`bJ�	%pZ.�)��l�M���uD�Cr{�?� 8��\��HbweF���2����1��1����>}L#��FtL�	��E�z�"�����sb_�W��k�����2�'��X���$��I*������<�<TR����0�l���=����b���W9�u��2M���K���Df�<�7�w�2k:���j�pp4�i<nG������QeV�vc|5���� �	�9�#�q35ES�4X|�ETk�3h��y��:��W���<����!1a�H��T��y{����"y���j���P��nX^�<��$�<8z�iF�\4qRkk������)�Q]y��}H��D1�����FL�����*�bV�e1+���nI5f�c�|��$���'S~�I�b4Z���
]j�p������������#�<���=�<*H��~>�}e*���P+�+����'D�������L���2-�}M�k����l�n��D���q��B�9��.zycQ�sx���[��4��������DA������0�	!�#L���Rr���P��_���I�cBD<���2���
����a��J���u��
��9���5�nM���W�^���a��dY&��L?�������i��L����4�{��t�o��y'���+��M��P��G��I��!�@n_���c��#LY�������aOe��P��Of�o4�`LQy�Klz*Y��������������i�����j��*X���~�
�2�>������
{�n���������d�0��u`�d9�IjyyB��/.������?�W7�|��5ul���*��w�
�2U8j���~��CwsVnVd1(�d���eRv2U�w
b2-��i(��I^��[:�R�*
sl�8�%9�-����
�l4&��Q�a^Fw�l5Z6��e�'������T�Oe���<�T"�Z��������z(��+���TD��vX��!����������K8�\Y����&���+�
H�����!	�����`1(��k�|'��}6�!�bg%MxV2���NX�	��X�S7O�_�?��J|���Ol�B�s�>���3AI��T�hk1���2��;��+�����M�D�h�"z��x�e� f��
��c�'�6�V?�GX9���\Q��Q��/�W~w�������o��*_���!����$j�����T�L)Q}���_(�}Dm�}�j��[�"�h������]=�U�w�������=��;	��)j.��kk��.����~IEYRA~�����}O�~�!3�|OI�~H]�g\�6f�1.<���j���OK�F�z��>��x[��N�]<�CFi�����1
��L2��S`;���H�d)/�@���q��
9 ��|����&?o[��[�������Ee�F�F��J�H�!�}(Pt�$��;C��pAv���C��a6� �v��2���~gk���zb��g�L�{g�k���������3��*�C�����Aij���>~�S�����G��c^�q{�Eh�;C���5�:~wrz���l#�N��5���0Qd�����W
���G�=G����4N�wM���CbWT$E����F�����v#R�!�������9H;���m�����U������H3��xTC��y��'\���
�W^W�������#�':���^1�����3��B_��egw�H3^_�����%��������g"b����.^v����lKbTnF�!��3X'����f0D�(^vIc��s��Vn�v��L�n���8��������O�������Xd�fK��m(����n�qsb���U�P�m6���`����������<��_�0�UL���b)�D���|����\�g�waX�(�� �GT�lsp	;!�v�I������:��@X��!%9���&�lG�����B?��wB�����6Y�r%�&VX��l�"�3���`4B:7��r�qT~H�����a;�/r����CD����H��8�t$w�3��]NGu$��Udo!����w�3p�Sm�[�D{'<���t�G��\E������g,g9�2�>h���r���o��������GC���1$��	�(���h��xD���Q{\�fS
 ���927;z�l���%	�����2�� %]-����W'o�;i2������'���*�T:N���e���W]Q��z��]'�����z��������2RKH��^f�:��Kx%�%5���|/{TruGY���8��6�+���Ye���hh�|�{�N�G\`�����U���5g�8�4�\��i�������j���Ua~,����lU��������^eD��'���������;�\��x�p����
���t��?�l������g\`l�l�.��,k��9�����N�����;��|������V�|����/K���8�}�D_{�"�r���eGUAT@��r���������X)*�
p���5��f)����JL8\�����f��	��@�s��,54�pF�m�29�3t�O8;C�[��~��6���(&x��y���5�7�,k���S|d��M`��C���"�'8W�n��N!s�[!�e�����������+�����%�FF�nxQ�����Y�n_F�6�Gh^��]	K���,Hx�%�Bn�f�t�%�D�l	�,��f�j����p�[��V����WR�B%�8yl�V_iAYL]�4����}A[��4�/h+�����
�lAc����
Y��������(���+_�������.I�IKD*]���h�F}��n��Z��;��i���:�~o��1j�zC�"�x��}$|oj}COa���)�d]�n`
��JB�l�)����oY���3�����I�G�����<#=�1���Q��V��]�^Y�pa����8����
�$��	*h1M�r�����Z����DW�l�9�H5����TV��j&#�����u)���������b��|�C���Q�����g��O�+�G"�#1�#��T����xT1��Q'��|�����=e�
�-�:��L(�7�����Q��D4���h�*���Q=����V;��������e�f�#��W|�g�W����u�Cz��QR>,�����0b���G���;������u�m�N�m�U�0�wl�f���$8o�k�z�g�����gl��8,K3������������+���+��N\B���^Le����hyE�(\VUPp�@���S�=2���� �A�����A;�B��|
�:V�w;���������������]��������(��|'�~XE�]�E�b�X�o�Dt�#:����
	�&����������y��� O��>���.;���h��)Ct��6.���[TnOo	l%q^��X���l-Gv�9�U�:W������x���
Y�-X���RB�W7�����J4>6���!I�9X�f����^��/�.$e.VN�2��E��jj�q1@�McOs%C��X���V���������D�6��#��������0�PD�\I��!
Lm�iv"
�E�����e��es$�9�y�:qA#RL�����Z���YV����,
��b�gYx�v��#\���A��}Q�9�%��s!6�&f����Gz	�V�t���<
H[�����A^aI�k�k(^�u`sh�vM������A��.����Q�{:��t��dW�������L��>�i�_����k��	���^�QA*s��������cWh2���k�J�j�<y~���K����+h���	���^���'d����T������}*�y�_�Z�*p�H����2gw��d��"��]��1�]WHZ������u�1�z�b��Y�~T=�fN��������^�>�1�41A�"��>q��0���;�����}Q�`U�H�=6�*���r��wU=0�Sj�,�I��H*�,K���OY���I^0O�i+���5^�d���0W��ofX�1��C�5���L//���!JDA93%�U���+f`����rR�dK�jN��U~��&dEV��j.W��*��,�hR�Q� kV:XIIY�5kBN����s����������<�s���GJ�*�Ue%�!��(}-�R
�����N)�M����S�"+3XE���������:�6v�VRW���G'�����b�>����P+�a�c���*�+��ZU��%M����V,�:�AJ���Z���!��HfW�#�YUh@���2V�G_�
!Y���
�7�,$k��)�j��d�Kw�bj��^�c�(��-*��/J<���Zb�X+P������5 ��@phev���Ef������vW�'���������T�azUv��I�����[�
��k\�K����3�Ut���jn�5���{���Y!�W����X�q��tWP�G+y��%��W:(yJ������E�@������M�a����egy�X��E�D����mn�����hM�z��������<Zby�m2t�B�(��lF�3���6�A8���Mx}�^����A��MN�E�J�GZ�+������*�V�����n��\7�5-0�\���R�AG4�5���	��C��z�����/a<.�*+.�G�qw����<����D���J����a��-�N�p���~��~��+_���Y��s��Q]�;�O���(���o����#��6��Ldy-�G�o�]]Y���V+��S�ZK�Ri6J_)�b�V�
H4]������p�@�%]���W�R��2�6���g^��&������Bd�Y����
kXX��DzlEz�DzGu5��H�~{H����q��`u�J����i&�G?^y�
LmvJu�&2$VdH����.d����-[WK�{������:6�

r�����4c��x�u,}����2��#�-����+�8����8�mz�*V��T}dfi�$SZ2���(�*����.�%%���a:�-��\��5�P�#�o���>n��B�
A#��m<���H$�4^:�s��(�(h�[�������o��Q.\����@�^�/.��E��|������
��X*z9R�^��5Y��?9�}3�%�r�)�\}��n��|l�]�o��b���{�e6[d[�	����[��vO}�c�f;q6'���,��U�D�J+!w�o'��xp�X��a�59����s�1����)SZX��~0��m��w�K��Ck]�������5r[��e���v?pa��<(m^��\t��"�=�"j:0����S*X���P2�U�L����x�Z��0����
]�����c_��D���,:o&�x��Hp�p�����s}�����s��5�,�X�*��W������*[Z��9��#��-�Vj��V�TF�D��x(6��X���XX!b�A��n�N���}$�$-�����}��}^cK;��7����M���T���0����w��q��b!=������s	���y��ROQoH�du�!(���]�b�%Z�BfSZ�lK��i����n-Cfn�Yo6��J93���'#�i�Jg���d��G?G9�{n�~}y+����D���\��6��WL&A�G<Q�@$����	��M��M�z2�V�}��:��xL��_����z�7��j�?{�����*��.%���R%���'���Z�I�2�bawY��-�	eq����]g�~l?������8�:l��U�x��������_��m}Y���5,���W��
��t����q.aw�Cz�YUU8�c�Uk4��6��r�����K�j��r�U����I��+�v����@�QQ�QK�lm�HY*2�cT��	l�����e4z`��`���|�t����z�rG�?�j,�\���2{��y���+_��?�
Po���ce��R��N�a���8]CC�������+�-ZM�P��#Xh�Q����-Y��*[_d4o�t��P����%AL����A������%ys����Q���]g�_�a��}h%	��"��lt�#Y9����q��:G)C�"�c��F����,�*8�K�q��[�;
���i�+��J���G��$���/S�
2�t���q�q��.n-�Y]�.ZY��Y�e�4Ot��X��������m�6Q��E	Z�`����0w[�I��U�j��C�Zq��+�ki^��RyM���ev�,7��`q��u��yV��l�V%�U�����}�m��+.X�bL���F��egWP�.jc� k���pY�9;lpFi �{UR�#
wg�]��jS�������/�$V��
z�2%��b���/��`���^��Qc�l��v?��\=��[Q�
�_}�����4�ez>�K���be��pv��������>��	��;N��(A�(��/���xi��-?l� ����v]�NR-����A��@*��gWY�G�����"��s��b����;EgB��:�n��le��gw����^�x�m|��60��6`�o����� w�*'X(T�K>?��Y
�u���U�������52���,s�v�*���� �� p�1�c�h�>�
��P�I|�A��AT�hQ�|�~0�H���7���[��d4�*K�o���!�gC�esjsF��v�
�{l5i���k���MW�qX����k�n:�s��<$�U%'�R}��Lq��jJ��8�=~W�F�s��_P�XP�hs��M�Y�����y�g���5{7�
�yAUL����^PeS\���N���r�J9�$�i��C>�C>�����F�|�s�s��S>��1[c �2�A��[���zn�Q+��v4.j|�8�'����)�3���N�ey�8`��S���L+����P���E���G-�"gJ��#p	����u/�R�)��������Y�t3�v�Yf�w��L6�<]lT����e��r=�����3���8&����*����*���� �� /`�eH�f����[�-�Pd�U
�p�S�3���F��~�]2,�If��r+�%+Vv?��7��o��.g�������$R�p�����$��C{G���z���g��O������hvzbqzn��O�a���"����]��:]o��9A����A�,�u>�7��
�7�z�����A64
��|�/��m�gJ���b����5Z�����;�����[Ik�;=9~w��F?��{��%A�7/X�^�9E�~������	�pn\M�����C���h���oON�����M�
_'s��d��hN�|5�]��&�������������H��m6�d�y��V��x<>"���`bX<��������_��lb�����9��K�l��t����_���������d�y����EZ ��a!�9���*w�@�}�G��� ���z�a=fA���2�����qR����P�V?�aQ��S1���2������=����i����,�s��g$�<���c�J���Y�����OK����r���c���}���J+��m
��%��"��E���^T���E
��^T:��E=��^��w���EI�D��P�2]�gM(O�����"����L�fx?'���="*�C
�8g��DJw���RR�����F�Ox>*��-�|T��	�GE�`���Q�GE���|T�6jn��|T����GE[a���A�GE���X��N.�<�`��v��Q����y^����i��|�<��&[d�m��S���|���x��2[C"��f�AM�J$��BDsv��.�r\^���5����2&(8~�~G�Y��F����G��*�x��Pmg,�E�;hx#�u��T�I7,Q���V�a�rw��/d�$���)O�f���7�>xN��z{���XiN�I���<�=�q�{�jH0��6�<Ik*3��4�@��mp�+�.c�k,!�+�6.�"���a��n7�P�n�$[��=w5����������U������]5#�J{��]�F�v����Ns�����pRy?�{�C��B#RIi�"��[������fq'���\�wr�T�����b;]n�%Y.W��t&]����eh�t���D�t����.����!]���*abE��}��A��R�-�
�����F���
�F�����F�����F�����F�x���F�t��gE��U��J����1W+
c�46����;�[f+G�i�f�}��E��|�V����u�H��[>Yn@G��(���mv&�g"�{�8�(�a�I.~"�Q`��V����^��J������
{P0�������Gky�hU8O/^{��\��+gHvsJf�����x`�!G������+rG%�ht�����8��A��������6�+D��e��~_�T��E>�,;cl6 PQTf�\���`p��}���y
Ma��#��_ei����z����[pH���UaO[p@���UX�d�2�yH�+�z��?�\�H	�G�A�=.������|CWu���>������	�}�,&X���NcV��uD�$�y�
���,������Jz�J������x�YA��Fek������v���>`��|����INJ�*lR/~y�*`����E��eM����r�������v`M���7�E��.wf���������������ud��>��������S���j�h5v�;�7iF��Yy�wbmto<N���y�b,����ww+�;�����;�P�K�Zrggr���~�7sn����2�%��_i?SLh4���k��
����,%��j���n1���J���lC2����v~���xqE�Y���p�������3�%8C�Gh�8�1�#�0�F������A�������b�h�����;�O������K��NN������������{z��n�)W�������������2������u�l?�����xU`NS��5���8�[�8bY���,
T�*������[��-������[V[�w��-Jw����"��F����H�{p���:��&�P��J�Js�g���4�(�s4�{^�����������p;w�s�*{��8lCw�
�"�������6���]}��s;+�?��������<R����G���#x��}������O���'U��t����	i����mX9�"(�\����6���|)fB�������I����B4��������3M�����Q��,���������A�p(�>����PjU��P����jo�(j����FPL�p�����nU��fT�)�)m��N�?Q�>QV����e�<Q��DYl��DU8���'3��a�Y�
`?X��#����>L}
+�a=+0��7�b���k�"-J�i�`kz1�6zUA������e����c&������e�a��25�`��J%d��_��fN�����v��������EP���;��;���:���������a��0c�0cm��e�q�a.��;���7{���X��9��6��e��J�\�M�w���D���y���r�CG*��s?p��;��V�Y��{4�$���R��r���-��^�I��?�-����<���-��x��_)�+�a\�[��E�f4(�t��5Z�7[M��B"�M����=�\��P4�wfdM�;������@%�����A�/�>]������z��]sV�LL��B��p���_�z���6|`���f��h	�g4�gu@10c����G�D�/�.������}&&K����bVK�7G�LL��`S�@������Q�����������#�K�e]e~�F���<z�f�H��D�QzvF�2]�
]�#�+�����^8��=����fy�)8�-�
-�m����m�h�����VhC���o$4��lF����Sb'`f�3�G��A��f�E����6����g�EzM��bY:���a����"�f�[��9������zB��'�M
���2����(��c�����E���j98c��Y�r�b���p7��}h ����0G��m����?�����pZ09���1�qL�3��#5�
A���G����OD�ip����/���i<�l�����O�<��;��8`
t>�9q�!�����c�Zy�&�������}��2����P$!M���^O�x8�q�~����rP� �?@��c�c���O��SC� �������A0�j���=Z��%E�x���Q���c���1�et/t)0
ge�d!L�PL���qoT*)��*�R��r��
�Jt�B����k��C�_w��>Z�F�G$j��az7��w�����=7E�M�����Y���i�'t�8�sPA���j@��Qm+�Iv�C�y�����4S��w����1Pe��w"��������|=+5��3b~7�Zc��1��_������1+�M�9f%$����F�}�����,}��Z���4sZ��|������3��p�6d-��:n��J�(�RW��) �)��
j�h3��_
���M]�I�e!Q�H: �%Z;~�m0��o���q����SS��S��I�������;�����C#��=�<X�<X�|b6��c5���Jr�t6,��~Ile�u������:�H����a���fc�vcw�
�R�5���G�C{��O��
�D�^����\M�$4���k1,3������D��Z�����U�X�f�a�u1'������c�2y�AZ�@H�� �(����A+���0|D�`��\��U�~kl��G![k��^�����x+�7+��w��C������;�V����-�������A8m���������h����.>�������>�0�pt��T����#�Ke�P)-�0��U�����.g$���������q�[�j��6���o���C�=o�]�����+��l�������������5k_a;���-`;�'��F=Q}h�������u������;06Y���N���p�]����6���KK�
��K*�; "�Glf����J8�>�t��fV�X;���r�����vXv�XR��z���E��D��K_m������W�[���!�
�tI�_P���6�
�e�+����y~:���3f�-����m\�d7e��CU��p 2tF_�"?����g~��W��jO*����������P����T���HH�I���%�l�W,m�G���������$�d�e�W�b[�2C�T��L�����7'�����`��:�\����n��!+���������9�|-��9J����hv^��j�(3��\�5H-q�Ivu��V����X ���'F�G���gz�vD�Ux��\��9�l��#GlE��$$�
�k�L�rbb
`jSw (�X��^�HR���9b@\0��LT��|\>Zo��*�����%����?@a����~!8�M�5���Qw\���Q>(��p����wG���Q������#`��A��Y�v��p�P����C��.��q_����0��%����=f�|p��Y@�d���h��(���'&�r��2�?(��p�$#<���7$�-��n��I��Q�?=>��9����T��+�W�����P?z�e���E��T�(]����Cvf
�d&a��O�b7)y0q�����(��>����}<t<t<t<t<tO��x�x�0 ��Cu��x�x���s�T�^a����2]�-�����e�V7���&bQ�{�uV�:o%5�b��B���D�R0�J�k����v�N�+���8#�qrRCo�&"�\_Q���(�����6�b�����?�1S�FIA�(��<������	������-��-��-d����rY��w����x�n~�qI�;����O�X&�=�`&����L���$�o�`&�����������D)zf�j��������<(o�����!��!��c�3�L�L�Zx��������j����+������;�?���c�h�T��#�h�{����{/%���������� ?4(D�5~��Vb
�9�,%I�a��d���p��4�C�aq�n���"3�R��2�4�^������+�=�w��J�g�~��*"����U�O�����Q�����~��������G����xl;�6��nI���w�_	6�?��?��|�IM��S~X�������F$����#�=.�����5�{?��a�����?M��wc���i�nPf��CV���_�nb��{7����������������������N���px�G�v���`�����z@HL���s�Go�T������7�2K�Z�K����7�)�A���P�`5S���l�f�O5�d�5�(�L�g(E�Ak�ZJ��
1L�����~����9$�����l��9�K���K�|M�����m/Iij���4��v��@��
J��X���OW7��M�a�R���MU�9`��*�/��4j��+��69����(����^� �J�Zr�f��kGQ;�����^;�]H�7g�������b����ow1�X�nB�B�V5�?�`=�`}��\�z��(<�h8A�g�{!L����b���C�C�;�����$��s�/dP��*�3�(�J�>�j�Iv�0�4�YY�]�\���W�q�FI=c5QG�#����T�Vs�<�o����./���V�����k�Y	E�1k�A�H�}�����,}���{�4�T�!}���9_��|����
�]��l��c9$��l�����=�o���4< u�Y;�}�m����l�T�q����SS��S��I�=�W������{���}TI���wQ%���+����C��G��+�+�@��W���[�!Vr����-�X��X�{�-��9=y{�e�t��&�d�A��qI0bA���'���V�QH�d���jqs���r�=����j�h���I���>�#["������IF�I
����)>:zn�T7�e�(&�T4�W"L�{���O��?N�8��C��{�����A�p(��0��A1{[1)fV���4������GhN@I�^
��b\e��M8�Oa�Hi�`�D����e�>Q�>QV����LT\�(���!_��+�������|��V��V�~����M�f_��kXy����mJ9���K������+���" �E�5%�`%�W����j�����z����#�3��X~�9�/��V�<��[D��("B�j�����F\����N<����d#>������R:0e�,��Q��G@p,�E��j�GU���-�a_ue��fl3��;�k��-��+
3f7�}��p����5��5�F�{C��@�XC����������J�WFw/�C�C�C��bV��jd'�X���|^�v��P���;iQ��&�HU��I*>�J�>I���dQN*�����TgTg��q5BNx�/�����{��5�!�����,���{�R[�����,��D�Kd�}K�����}�_y����R�/�5#��@4X�3��h�] (+����`�f����-H�4}�t���M��.(n�j�5��^�}��u�|�5c^D��>c����������u��8�>���s��kjlp���x{�/Z[���le����L.�!�h��1U���%
��������	^.��8&�Cw��j�v�_�f�9��c	�j��W���%��ex��X��Q{L���X,�e�W=��zW��
��V�+�S�U�m�`�[i��=�z�����za/'�#��z�m`�:v�I>������w���K��)�f��I%{��3�.o�]u�������^O`h�N�}�����N?�\�te����|y�6����-�b�#-y�����=j�����_��o�;�����a��*���m�:f��^��*f����@d(����o~����z�*��=	>��z�wW'w����+���(v]f�s*��+vi*�,�T��`���������q>3�J�����~M�V������|��a��Ou"8�{���YC�
��6����������z�S��u`Pm1���n(�ui!�N�D��It��>�|���O��S�e���L�yo���p�(@j����F�#�O$�	���[����� f��{C����w�1 ��������Nj���8��n�:A�������|,�]�?�>���Os�v.��e���n�t{��;n�dd��S�h��@�#'�(���[F8�v�&�q������
��w�F�v�l �[;���'�$��	�Z;����N���`��,����h��=�?�`��q
��������!���	{����#��6;a���=��;B8nG=�H�8���!�#�����`Gp�#M3g��O'Q$�
���<�����m����8��G���%q.O�^q��Q��r{���s�:�82�a}h�c�b��:?y:�C~�� �����5��.�C������f���������G��~t���
�?���s�e�?�\������"���Cu�W�.���qD���2��\��"�;�q<���rM�"T��/Q�Y�S��u�H���^��gK�]]oo����D�6�	Z�8�r��_ �	��/���
�c 4���-��a�4�'v��I_w@���[�����"3�
�J�,����UZ=�x��^`�vh��^����z,Z_a���>Z�K2D�����^O���	+��
+��������
'w����U������uu�j��)��_R�
U�+}x�/c���|k�{��:��W�����b�������6���m�J����]�
[�����W���$c��**�Y!@�u�����c�����
�w���,�r�:0���c����;�H��i����Q.w7��Vws���a{��V�D��������v��m2Yj��a���J��+�K�(x���g�Z&O`�VQ�u������JS�(%�v��8ZV^eAvq�������Q�a��L%�Cc���*.���Q$�7�s������E�����	
������N��Dt�Z��Dv9i���ACSf�����`O�������cUy1+���$���^�Iy��u�-�����e�;x^+��M�M�S�|c��c���l����Dj���+���IH��]�����%�C�J��
}��oV���7���}�mK�&����0���T���j�Evj^�Ji�X'�wyz`Jz��:/:���W����8�t���-��f(��^�	k;���5�AOP���y�&X���Pp�l�!��J���������]������4;;p�>�L���jFC�g��C����H<����}�<�~g��l ��������]e{S�����f�`x��w��TA�V�pKuU����������Z��Z�������\��V��C����=4\�H��F���Z�:�������l��0f�!~~

������G��I0hG�
�	��|+fJ
�����y�����������f���
�i���OptO�5�BGXO���'����+������L�6�Kw��V�^�����VI��S�:/
j�X�7�a
�
�){*X'�T)�ty������/�Y��h�����X�jY3�sK]c�����a�Q'R����TQ�"#*�w���iH�V�+9��R@����{9�c����`�������z���O��D�{�������	`w��B����9e�R|6�F�nw��~<����$�1��km�Z�
5���k�>j��x��L���j����v}�=/���-Y�� �to�������}���z�|oG���������~z���f�� 
W�9�\q�d2���%��5=����c�3gM^&�����qLq_�Z�P�?��(�>IH���#���^�����%m~���������$UvE�e�V.R�E�rX-���b�\�a�V.Q�%�����d!�A?@�_�7T<�laQ�/�a��1
���I(�:a���$�(�������i	'�[�S���������(n](���V0�ek�Ck�b,7K~���_�	(W_�]�m�X���l-
��V �e���j���]k�G�<�(��ZA��.��J��t����Z�D-�0xj�F�v��L���Y���2Qx����a�W���"��`h�,�]����{�m��-��s��J�,�l���i����o3�TxM�\l-J��B�V=`��-�����e�9*�'�2 @�Z��Q���A_��U�V%�^��PZ�HzG��=���J���R�Gj8���X�S�i�d"��6��`��(�R��U�t��\i���W~w��������w�K�*�Y�.�������R���y�V��SL�Y�j
�{~�Q�X��5]�uF�$�2?64?myE����Wt��m�#bQ�o�E��������yz#�Yd%%��]L��ww�ZV3b[���*0�A�+�L�X�peb�%2�Rf���e�a���56X6	�rgu�r$�*�s��,��z�4�"�dT��b)��v�n�����Y5���[8���|+��r�+)i��<�#6)�������b�uI��K���\�He]�VH�b�uI[a�-il_�VX!K�Y��uV%��`�K�����N��A��8���QQ/�[����!A��N2e�:=QT�`|�s��2�l������@v���\n�����&62�.EA�'(bB���!2a�����}��G.T����	8Q��t�Mp��3v�����2���Q�.f��MdTARc��6�������M�w�j�������P����hD�*�&O�4@���=����;�~q�nB�-���X";Z��l}K��|
������6bV_��M
-x+c;Kl�,jHW��Q�O�y�4���)h~{]mf5)��?�v�� �	�O���X���-�����o���=������i��/�>��D�u��.���s.�dm7���w�����{�w����Y���v�/*�>�*��}N��umen9'mxT��C�Z�����p7N3$-a�_�;�[m�x�Zq/i���;L�x��j��q{Wugftt���2�w��������t��������Q"��������R
�i����l[�����X�q�>�/��8a`��c��8�Z
t�ks��D�
��K���N���,=:�"��1������G�m{+���^�Le�g�%]��NmY.[��RQ�S�3�p�{���;�!��s#fx�v/F�'f���t�C���q�����S���J�����&���JI��+��p��FZ���i
��J������
�/�u���?A��!?�'��q$��t�G�U)�o.������'�A4�!'�x�x\�LCzx����<fbA�Tj���[��xe�,�g
��x,�|�|��Z�b���]U5e��s����2������w�SY4'U��a�ka�sb�KATexG�T�aY�D.����)�B����.]R5���k�a�����������hf��(iG�~����B���?�z�X\=�A�i�]����e5|��p��\�f;@d��8C?����g���������g��2#�E5#�0�R���<�+�������K�.4���/]�u�H4�4�*s�
���m�f"�nVT��C�����%9hIZ����akI�]��n�Z��������cb���8������am2W���C���������5,��R��
��*�q�X���9@y��f���N��Y������!�{�FJSz���b�*��R��n�QZ�J�� ���E���6Y��ZY9#����Q]���-H�RJ��������Q���c�����W�{xXJ�G��<�=~�A4�!3u����|0
y�,��i�������NJ��"�C��m�
�Cw��j��=��P�>@�]���a�;������s����=6x��!����a~�#���Y����<tf���Q�.^��M�(�m�fk�zOv���z{{���w��������;����F�O��z]_L�����k��er�X��`D�9>;����lzs���6��"�lW�b���A�;:��G�B��>$�
�=�.���b�$���cR.��(������k/BJ7�7����U���#�^�������<Mz8�'���|0M�G��(9�3|>���l8M��'�q���Q4D��3�?�d6�:�c��M	�?�1�N��_;���	[��M������=����6��"�h�E�g��Y�Z�~�k�����ld���|���Q�t�+��D9GN0�d��8r���
��F���������j�D~����3�=o��`g+��6��%a�H��K��/;Nh�R�x���{v���������k�z�.����w�����6��4�i��G�2[\o`D~[�>��H���=�PK#�5�o�vf����hX�����&��z�M4��������!58:�4h�WHT5Mg�~���z;���O�~
��A��!��#���Q�E�ns4r	�l���dX�l@;�l5�����O���D]�8F��nw@�L������!�i����8*`A�v���Q�0!G��	�����|��Z�V�6:�o���N��
wp���<9��06y��o�8MK�'���'�bu�����_~�����D~9F���2�J"������,����6��M�6�Vo����hB3;������Q�|���#��<y����B��Kpc�8c��y�R�V���0;F���2Dg���[��T]���������2������������{�.F�'��&Y��v��y:��`lD_���?��{���a?��}\��gQe�A?Kg#2x�p4K{q6�������y/�����������@��(Q�c�����,��r	�D*r6��)(���mu�����~��q�a�I#D�B �&���P��H�7�T�F���-"��}S��-�����j�E:�6]tJ�%�&7��/9��/���MX&�vB��7[��}��y>�a�S�����z Pq�o�n����mJv�<$���>�W�e&j7i� ��0��	a���Qe[{�e%8%>��iu�g�h��EnJl�Q���R���1v��%[�vNs7<���c'���
�6M�$�Q�K� .(��P��������� �z��-��?y�.n�������vK8�W�����k@dJ�������B_��I��j�@_�7����+�����y�a���%�����v7oBpO9�����R�����M�p0l'j
������r�������}{K��R���������O���c��q���:��O��;=~w��_'?������F����'O��C_#��N#�L��H��_����t'��67��z�g"��m�lhs��B��V��3�T���>��&-4��\m�GG0���?���6�����f�D��b������XJ�����9;�H73R������������	���V��l��xA�e�Z�����&�%�k^���l�A�b��w ��m5����Mzq�6[��v"�x�9 ���b�}�E�6�������l�r$!
����B�������Qs$M��8�L���v��#���f���������Vk��K9�g�-�Z(e�'n0Gl�	�S.�������z��u�F*}�j]�����e����q�`�m�iO���|�����D^r�E$BRR.n�����O�z��,�������s������[8�|���w�����~M�.�0Y���~#=�Ygl�a���J�*�`'f���[�l���������-�D�:������{��r���W�]`�����;�a�@(#���f����8k�������Bk
��_i���m��	�O��n�@F�!�&��)�?8�j�1=��,l�|Nf����:%L���[�M��M!�fs�w�l������������K�[��";!���Y�z��L�N�W�[v�*b���Rx�����I��v�W#��3�J����0��~B��z��CZIP�*e�
��a��b92�m*ym���UF��)�5��:��b����o`o1xt��Sp�TC�En��2@
��>�/�Tw�S�]j���K��%5Z�K�o
FC�4��Lb}r8�BA)��ja�X-�P����B��^���^M���1������<����
�d�OQ^��`�z
^�����:�;sK,���l�1K�Mb`��J
�����
<V�Q�Y$c�B�$���\��c��R.���2rG�P��U���zx#�w��t����M!��������3E�*<Q��D0�|b�����=)W�')A��[Xd/-����73R8�gO�'`���c#�v�<h3[��,����U��=���k���:\��p��u���:\��p��u���:\��p��u���:\��p��u����ZW��0%
#24Dmitry Dolgov
9erthalion6@gmail.com
In reply to: Ashutosh Bapat (#23)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On 26 February 2018 at 11:03, Ashutosh Bapat <ashutosh.bapat@enterprisedb.com> wrote:
On Fri, Feb 23, 2018 at 7:35 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Feb 16, 2018 at 12:14 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Appreciate you taking time for review.

PFA updated version.

Committed 0001.

Thanks.

Here's patchset rebased on the latest head. I have fixed all the
crashes and bugs reported till now.

Hi,

I've stumbled upon this patch and noticed, that it's not compiled anymore after
2af28e6033, where `parttypcoll` was renamed to `partcollation`.

#25Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Dmitry Dolgov (#24)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Mon, Mar 12, 2018 at 1:04 AM, Dmitry Dolgov <9erthalion6@gmail.com> wrote:

On 26 February 2018 at 11:03, Ashutosh Bapat <ashutosh.bapat@enterprisedb.com> wrote:
On Fri, Feb 23, 2018 at 7:35 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Feb 16, 2018 at 12:14 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Appreciate you taking time for review.

PFA updated version.

Committed 0001.

Thanks.

Here's patchset rebased on the latest head. I have fixed all the
crashes and bugs reported till now.

Hi,

I've stumbled upon this patch and noticed, that it's not compiled anymore after
2af28e6033, where `parttypcoll` was renamed to `partcollation`.

Thanks for the note. Here are rebased patches.

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

Attachments:

pg_adv_dp_join_patches_v7.tar.gzapplication/x-gzip; name=pg_adv_dp_join_patches_v7.tar.gzDownload
#26Dmitry Dolgov
9erthalion6@gmail.com
In reply to: Ashutosh Bapat (#25)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On 12 March 2018 at 06:00, Ashutosh Bapat <ashutosh.bapat@enterprisedb.com> wrote:
Thanks for the note. Here are rebased patches.

Since I started to look at this patch, I can share few random notes (although
it's not a complete review, I'm in the progress now), maybe it can be helpful.

In `partition_range_bounds_merge`

+ if (!merged)
+ break;

is a bit redundant I think, because every time `merged` set to false it
followed by break.

At the end same function

    + if (merged)
    +     merged = merge_default_partitions(outer_bi, outer_pmap, outer_mmap,
    +         inner_bi, inner_pmap, inner_mmap,
    +         jointype, &next_index,
    +         &default_index);

Looks like I misunderstand something in the algorithm, but aren't default
partitions were already processed before e.g. here:

    + /*
    +  * Default partition on the outer side forms the default
    +  * partition of the join result.
    +  */

Also in here

    + /* Free any memory we used in this function. */
    + list_free(merged_datums);
    + list_free(merged_indexes);
    + pfree(outer_pmap);
    + pfree(inner_pmap);
    + pfree(outer_mmap);
    + pfree(inner_mmap);

I think `merged_kinds` is missing.

I've noticed that in some places `IS_PARTITIONED_REL` was replaced

    - if (!IS_PARTITIONED_REL(joinrel))
    + if (joinrel->part_scheme == NULL)

but I'm not quite follow why? Is it because `boundinfo` is not available
anymore at this point? If so, maybe it makes sense to update the commentary for
this macro and mention to not use for joinrel.

Also, as for me it would be helpful to have some commentary about this new
`partsupfunc`, what is exactly the purpose of it (but it's maybe just me
missing some obvious things from partitioning context)

+ FmgrInfo *partsupfunc;

#27Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Dmitry Dolgov (#26)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Thu, Mar 22, 2018 at 4:32 AM, Dmitry Dolgov <9erthalion6@gmail.com> wrote:

On 12 March 2018 at 06:00, Ashutosh Bapat <ashutosh.bapat@enterprisedb.com> wrote:
Thanks for the note. Here are rebased patches.

Since I started to look at this patch, I can share few random notes (although
it's not a complete review, I'm in the progress now), maybe it can be helpful.

In `partition_range_bounds_merge`

+ if (!merged)
+ break;

is a bit redundant I think, because every time `merged` set to false it
followed by break.

Yes, right now. May be I should turn it into Assert(merged); What do you think?

At the end same function

+ if (merged)
+     merged = merge_default_partitions(outer_bi, outer_pmap, outer_mmap,
+         inner_bi, inner_pmap, inner_mmap,
+         jointype, &next_index,
+         &default_index);

Looks like I misunderstand something in the algorithm, but aren't default
partitions were already processed before e.g. here:

+ /*
+  * Default partition on the outer side forms the default
+  * partition of the join result.
+  */

The default partition handling in the loop handles the cases of
missing partitions as explained in a comment
/*
* If a range appears in one of the joining relations but not the other
* (as a whole or part), the rows in the corresponding partition will
* not have join partners in the other relation, unless the other
* relation has a default partition.

But merge_default_partitions() tries to map the default partitions
from both the relations.

Also in here

+ /* Free any memory we used in this function. */
+ list_free(merged_datums);
+ list_free(merged_indexes);
+ pfree(outer_pmap);
+ pfree(inner_pmap);
+ pfree(outer_mmap);
+ pfree(inner_mmap);

I think `merged_kinds` is missing.

Done.

I've noticed that in some places `IS_PARTITIONED_REL` was replaced

- if (!IS_PARTITIONED_REL(joinrel))
+ if (joinrel->part_scheme == NULL)

but I'm not quite follow why? Is it because `boundinfo` is not available
anymore at this point? If so, maybe it makes sense to update the commentary for
this macro and mention to not use for joinrel.

This is done in try_partitionwise_join(). As explained in the comment
/*
* Get the list of matching partitions to be joined along with the
* partition bounds of the join relation. Because of the restrictions
* imposed by partition matching algorithm, not every pair of joining
* relations for this join will be able to use partition-wise join. But all
* those pairs which can use partition-wise join will produce the same
* partition bounds for the join relation.
*/
boundinfo for the join relation is built in this function. So, we
don't have join relation's partitioning information fully set up yet.
So we can't use IS_PARTITIONED_REL() there. joinrel->part_scheme if
set tells that the joining relations have matching partition schemes
and thus the join relation can possibly use partition-wise join
technique. If it's not set, then we can't use partition-wise join.

But IS_PARTITIONED_REL() is still useful at a number of other places,
where it's known to encounter a RelOptInfo whose partitioning
properties are fully setup. So, I don't think we should change the
macro or the comments above it.

Also, as for me it would be helpful to have some commentary about this new
`partsupfunc`, what is exactly the purpose of it (but it's maybe just me
missing some obvious things from partitioning context)

+ FmgrInfo *partsupfunc;

It's just copied from Relation::PartitionKey as is. It stores the
functions required to compare two partition key datums. Since we use
those functions frequently their FmgrInfo is cached.

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

#28Dmitry Dolgov
9erthalion6@gmail.com
In reply to: Ashutosh Bapat (#27)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On 22 March 2018 at 14:18, Ashutosh Bapat <ashutosh.bapat@enterprisedb.com> wrote:
On Thu, Mar 22, 2018 at 4:32 AM, Dmitry Dolgov <9erthalion6@gmail.com> wrote:

On 12 March 2018 at 06:00, Ashutosh Bapat <ashutosh.bapat@enterprisedb.com> wrote:
Thanks for the note. Here are rebased patches.

Since I started to look at this patch, I can share few random notes (although
it's not a complete review, I'm in the progress now), maybe it can be helpful.

In `partition_range_bounds_merge`

+ if (!merged)
+ break;

is a bit redundant I think, because every time `merged` set to false it
followed by break.

Yes, right now. May be I should turn it into Assert(merged); What do you think?

Thank you for reply. Yes, that sounds good. But actually you also mentioned
another topic that bothers me about this patch. Different parts of the
algorithm implementation (at least for functions that build maps of matching
partitions) are quite dependent on each other in terms of shared state. At
first glance in `partition_range_bounds_merge` we have about a dozen of
variables of different mutability level, that affect the control flow:

outer_lb_index
inner_lb_index
merged
merged_index
overlap
merged_lb
merged_ub
finished_outer
finished_inner
ub_cmpval
lb_cmpval
inner_has_default
outer_has_default
jointype

It looks a bit too much for me, and would require commentaries like "if you
changed the logic here, also take a look there". But I'm not saying that I have
any specific suggestions how to change it (although I'll definitely try to do
so, at least to get some better understanding of the underlying algorithm).

I've noticed that in some places `IS_PARTITIONED_REL` was replaced

- if (!IS_PARTITIONED_REL(joinrel))
+ if (joinrel->part_scheme == NULL)

but I'm not quite follow why? Is it because `boundinfo` is not available
anymore at this point? If so, maybe it makes sense to update the commentary for
this macro and mention to not use for joinrel.

This is done in try_partitionwise_join(). As explained in the comment

* Get the list of matching partitions to be joined along with the
* partition bounds of the join relation. Because of the restrictions
* imposed by partition matching algorithm, not every pair of joining
* relations for this join will be able to use partition-wise join. But all
* those pairs which can use partition-wise join will produce the same
* partition bounds for the join relation.

boundinfo for the join relation is built in this function. So, we
don't have join relation's partitioning information fully set up yet.
So we can't use IS_PARTITIONED_REL() there. joinrel->part_scheme if
set tells that the joining relations have matching partition schemes
and thus the join relation can possibly use partition-wise join
technique. If it's not set, then we can't use partition-wise join.

But IS_PARTITIONED_REL() is still useful at a number of other places,
where it's known to encounter a RelOptInfo whose partitioning
properties are fully setup. So, I don't think we should change the
macro or the comments above it.

Just to make myself clear, I wanted to suggest not to change the commentary for
`IS_PARTITIONED_REL` significantly, but just add a sentence that you need to
check if given relation is fully set up.

Also, few more random notes (mostly related to readability, since I found some
parts of the patch hard to read, but of course it's arguable).

```
PartitionRangeBound outer_lb;
PartitionRangeBound outer_ub;
PartitionRangeBound inner_lb;
PartitionRangeBound inner_ub;
PartitionRangeBound *merged_lb = NULL;
PartitionRangeBound *merged_ub = NULL;
```

Maybe it would be better to not repeat the type here? Let's say:

```
PartitionRangeBound outer_lb,
outer_ub,
...
```

It's just too long and distracting.

```
partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
Oid *partcollations, PartitionBoundInfo outer_bi,
int outer_nparts, PartitionBoundInfo inner_bi,
int inner_nparts, JoinType jointype,
List **outer_parts, List **inner_parts)
```

From what I see in `partition.c` there are a lot functions that accept
`partnatts`, `partcollations` only to pass it down to, e.g.
`partition_rbound_cmp`.
What do you think about introducing a data structure to keep these arguments,
and pass only an instance of this structure instead?

#29Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Dmitry Dolgov (#28)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Thanks for your review comments. They are helpful.

On Thu, Mar 29, 2018 at 2:53 AM, Dmitry Dolgov <9erthalion6@gmail.com> wrote:

But actually you also mentioned
another topic that bothers me about this patch. Different parts of the
algorithm implementation (at least for functions that build maps of matching
partitions) are quite dependent on each other in terms of shared state. At
first glance in `partition_range_bounds_merge` we have about a dozen of
variables of different mutability level, that affect the control flow:

outer_lb_index
inner_lb_index
merged
merged_index
overlap
merged_lb
merged_ub
finished_outer
finished_inner
ub_cmpval
lb_cmpval
inner_has_default
outer_has_default
jointype

It looks a bit too much for me, and would require commentaries like "if you
changed the logic here, also take a look there". But I'm not saying that I have
any specific suggestions how to change it (although I'll definitely try to do
so, at least to get some better understanding of the underlying algorithm).

I am working on commenting portions of the code to make it more clear
and readable. Will update the patches with the comments soon, mostly
this Monday.

Just to make myself clear, I wanted to suggest not to change the commentary for
`IS_PARTITIONED_REL` significantly, but just add a sentence that you need to
check if given relation is fully set up.

For that we have REL_HAS_ALL_PART_PROPS. IS_PARTITIONED_REL is really
checking whether a relation is partitioned and following comment makes
it clear why we have to check all those things. So, I still don't see
why we need to change the comment there.

* It's not enough to test whether rel->part_scheme is set, because it might
* be that the basic partitioning properties of the input relations matched
* but the partition bounds did not.

Also, few more random notes (mostly related to readability, since I found some
parts of the patch hard to read, but of course it's arguable).

```
PartitionRangeBound outer_lb;
PartitionRangeBound outer_ub;
PartitionRangeBound inner_lb;
PartitionRangeBound inner_ub;
PartitionRangeBound *merged_lb = NULL;
PartitionRangeBound *merged_ub = NULL;
```

Maybe it would be better to not repeat the type here? Let's say:

```
PartitionRangeBound outer_lb,
outer_ub,

PostgreSQL code uses both the styles, but I like declarations with the
type is right there along side the variable. It also makes it easy to
delete the variable or modify its type, without causing unnecessary
diff on adjacent lines.

```
partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
Oid *partcollations, PartitionBoundInfo outer_bi,
int outer_nparts, PartitionBoundInfo inner_bi,
int inner_nparts, JoinType jointype,
List **outer_parts, List **inner_parts)
```

From what I see in `partition.c` there are a lot functions that accept
`partnatts`, `partcollations` only to pass it down to, e.g.
`partition_rbound_cmp`.
What do you think about introducing a data structure to keep these arguments,
and pass only an instance of this structure instead?

I have discussed this in an adjacent thread, but let me repeat it
here. Those members come from PartitionKey or PartitionScheme. If we
pack those in a structure and not include it in those two
structures,we will just create a structure before calling the function
and unpack those inside the comparison function e.g.
partition_rbound_cmp(). We could argue that that will benefit
intermediate functions like partition_range_bound_cmp(), which just
pass those values down, but there is all the possibility that its
future caller may not have that packing structure available readily.
So, I am inclined not to add a new structure just for this.

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

#30Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#29)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Fri, Mar 30, 2018 at 7:36 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I am working on commenting portions of the code to make it more clear
and readable. Will update the patches with the comments soon, mostly
this Monday.

Here's set of patches rebased on the latest head. I have added a lot
of comments and revised a lot of code to avoid duplication, causing a
net reduction in the number of lines.

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

Attachments:

pg_adv_dp_join_patches_v8.tar.gzapplication/x-gzip; name=pg_adv_dp_join_patches_v8.tar.gzDownload
�S��Z��kw�6�0��_��_zEJ���K����W?5���q9��9�,/��med�#�U������ .@R�����L��r����x�����}9[������d���O�i���iD~��h�������d|:�<��N����AQ��P���&^!�������r��*W�~O��|����_�s����n��z�99�$����a�:�U��]r��6��l�8~&5g�w)�
��''�l�=�f�����w���oc����_-�Q���w'�����w�?�O��,��so�p�{����i�������)yA�)�����|��G��$�#��o��|��M���I�d�������������~�7�����
<���C��0�?z'�}��8
�G�^��'w�?���/���/:����J7'*6'J7'�����e�������S^6����f��\�
��@�*A����9��;�{�	.�Y��%��z������<%�<�y�S�0�'(^�����.^��U�cX�����;���X~��N�D�6��:���SE�of�9������7�G��l�~M�d����'������Wo�������.~B7�_.��������O��u���>�/�:~|z��^?���w���{��<����gr���-�������589=?G�I�Z�������U�����w2��<���1���C����}G;��[��\���V���(m;H�n�����#�n����-����^���UBf�yy� ���P�����A3������?����s������"�B73����>�'�(��ww�}����A�~�|����|~���Q�������/�7�������������\(�T�/���0(�����K�0_8�R�)�	����,HF�>�M��7���{R�����0
�����3C��7.s�L_����<��/$���7]�$o��������_����x�Ws|!��}KI�����7���������L����5����#�{w����"/�����"s�h��<��#���!~�o���2����a
0]�����zG�|�#o��\�gx�P���]�N��x}1��`V
b��6#4�?w�s���m5D�L�o>�|�x�����\��b����}3�?�����F�\�z�	�������W������X�35�0�H�7�B��p��
N`�TO��<��X(�������7����O�)��H_}������Fh����h�1�-Y������U~3�}xL	!��������m����!����{q}A���X������.���F�D���jr���:3����f���A�i��=(����lJ���A_�%p���/���z���v���=������M�` �A?�Ox���)m?����\N>\���?�|��~H(z����.������i���C��l���6L��vIi4���<OI?����.��V����3�������������-�m��|Z�6is���&��c��4���M^^��=�!������3��J/�7?.�D�t�?���BA�S�?���0
�?��/-J��gE)%?OQ����9�q7�e:�����xk1J�?p�v���moc��j�?�GJ>���V�O���H�|���H2����i��i��44iHF�3�0m�n�� @���5�s�9FTcpG��c�{s\y����h����_����<R�i%�G���Mk���#y��� �Y>��!-f5�G<�Q9������
�qF�FT�����'��Z%�*��pa$��;��x���#��/�3��/�C>�^������+%�����\�MU7bz����V�X
�O�z�����4���!��{��L�0Mjl�l.j��n>4�������DO�us��������w<'z����o<'<@�s��
ts��������AJx��*�M_�?�C$JB�B����x�j
�"�7���e����h���d�Z"��1��JcR)�#9�f�.3���>E�5�AS�E-�EY��f���C�#�������y-�K8iV��'�u�NH��,������Ya��+}�u"Z'ba���|r����x���J����j�r@��������0����`����'UN����<��O�����|q�� e��� ,��|h9�3����2'~>'�{D��4�-���9	�9	l�D��4�-���9	�9	m�D��4�-���9'�-�tX::�9����Q�#�+�E�����T��o/)`����5#Z3���?����$`^��>����nz��w��r~��?���;G4����~y�<B�2���3�'e��d{��������*��JU���m�a��9f��p�L�w3y�e��nn��S-�&w>��#�����%y7-�1���=Y�Z]t�O]��:o�C��!����4r����b�M�k;]���[ CNs�1s*��s�I����So�������S��S)���S_:�����x�N���������JyG���9
,�T�v2��f�v�4d�T�{*�4��ih1�*���954 ,lhg@�[f"N�0�+O��)�����d�2���e�,�ps�9�9������[�.X��5z ��GVW�<N���b�?�w�H��������$.�����Z]���j��1������)G���A���tE3�R��W�z����r�"@Mwm�V���k����*���u~*_��w���D�/��GB��%1��WN�xd�����"F/�3
��������d�Gt��z&"�%q�����Z�����AD�IAL����9)�������xvS�&����h���9�2'5��-S�n��f�����wd��Q,b����>`mQ!�U��.���S]o�lSn���i������L~V��������/�5w�_��\>�����y�x�O��v�����jy^y���Z'��	i��oc���H�)y�Y���m��l�t[�;�d�
�M���G?�0Bg�6�w�`�3��YL'�Id�������[����i��(�����\���h��F�2�Fr������"�x>���j���m�����L������.1��������L��r�F.3���p�.��c7{������L+3�3������X���3������c�Sv�����JL2'���L�x��c�y���)�!�-L��O���=r�����c��8��iN�m���c��H)������|�v��Hl������~��z�!A��i����l,8��/���Z6���U��M*�R�\���J1Xcy[�>�2�p������Y����e�	u��j>���qa��m��I>�L����6$]���)��#���EY�)�~-
�n^)U�c?���F��I��Pn���E��r_+o��G�j�N�����	����������q����8���3/N=?:
�9�*9N��������t������%Oh'��/���C��I����3J��m�����y@�e��?��B�5��_���6��|�����mS[���Y'3K��[j<�
\���&������"��UF�&�-���lb��tO*��k�(L����H�5#��@P3��kg|h�&�5���k��
�i��J��P>(�A��3>�F�~~���(hO�����5#�L`P3���g|�&�h�����"R�wZ?�Z�f�jF�x�����_X�� <X��������k�O@TBi�����#�Ru)<�tT��T�E�r�N��*����p%����2���{�X�71�*��oZ�41�'"���;��g�bl��fF�r�Xw^����X�7�1gA�vy���k�/,������6��i�nv�4*�E:xQ	/���>��Nt��rjx��$�7	i]f6��-��}��'����%���<{�)�^������%���y�U���)C,��X�*`�d�b����}��a�!����aJ��\B�G����'�����Qv�������+<����AF���0�S����f5MfAZ������Np�(Jc�a��T��8����+�t}�������v���1e�qPF���
���L����(
k�0l�}�����Z���������k�j��Xx�P�5��	j�����eS��w���^�&woO�f6��ZZx���l�`��|��"�z5*�m,�I���a���ki���It>�LC�����������	�gmY����g��&R����,C�����1��p������T��)�o�D*��:�l��Q�f�,��\��]�F��A��l�s��v��!������c�A��;�VB��@�[��0N�[��0N�[��0N�[��0@��'
r���Q�E F��
�)�U�H����+�Ui�;t�
lU��@���
2V�2�Y�g����Y��!+��:�$��H�j,RHSm�V��F}��������!d���98�b���pu6y���������b���.��>)��=����r��s��JX�l�O�s���r����C�v�}������d��hf����z�g�����X
C ��QM�{�t�,�!@�=�=vV�1� aJ�{,�li�9�/[�[�������?/��=�������T��r���~���X�}���Jd�\'!8��;
6!����`wX� p�3�������lg;�a�i��vi������`��R(��f���9��vW�u������������%�m?��l������]�M�����t�k�	@�\��h~&g	w�0Wr�o�����e�����|��H�\1�A�� v�����
��%������1��]<#^:����%����.���������]F#^D��
(��B�2�lh+A�����p#
7��F���+�g��i��'c���I*���lY���$i�)H�`8��� ����l'����l'����l �C`0�B\0��$�Z�����E,0���� n���&���5	��r�`8E`8������)���#�`9UwD|���`9=,�,�f�88Q,�f�88�,�f�x�"��9i0�2��M�&����6���j
��"��D��t��R�j�a�HA��I�m!�&�L���� cu��p�����V������`�*�a�H�*�a�0�*�a���@��s~`���4S�}��VuX"�����`�*ms�.^��J�Y�b�A�����-,�S��� ]!��S�a�U`�2��IT`�2���R`�2������H��oR�[:T)�	���2ZA@J�t��+Hi���)��a��  �Q�����  %#9B@JH��e��0 ���3~i�)H�`8��� ����l'����l'����l ��?0��	?0��$�Z�����E,0���� n���&���5	��r�`8E`8������pz|�;F��n�<�-��l�X���r���Fg"�I�6u����K�'�e2�'���h���D�h�d�q`S;�{�D3}�)��v������F������A�V��j�J[��z���.���']���R:���U�;7-�*�nI��9
D;t0�w0�Z=g=$Y���ZyW��T�Z+�z�VMM�{
��:�"'��*u�^��D;tM1��1�Z�k=$Y����Z��
��8�V���4��j�h�u`;D/N�wU��4r��v�e��e4�I�zH�
-�[�C�R�Chm�#zi25��5����v�^�l:��zi�f�����C�hjc���dZ@/���j7�f�����G��dj��k`�����8YUI�"�7�"����B[���BMv��)^�M$���?��������?`�1O`�1�~���0���?`����2~��~����Je0�����1��Oev�^�}�
��������}�:��4�}j�f������>`�1O`�1��}��f0����>`����2�}���}��f�Je0�����0�����>����d:4�D��`�i��co6*�l?���U��~Z���l?��$7��N7���X���`j�;��"?G2.7e�������`j�;��"�\Tu���A&��UkP[���5���9�q��lB&�&�U�P[����^���&�+����n{�z����MN����X��� 8	v �U*�����I ���R,@m�N��l?��`�i�p�>`��T�O[����^j����������_���/��j|_���'S��.��NO��%���b��6��a_�fwOh�F��C�:���h�/���/��z6�'�x���"�W�Y���>A��J�1y�����/&7�,�A�f������}�WwO�j�~�\�|���w4������^���D[���~���mr���'�y~�p�>��G4+@y&������&PH�N4N�|��tq}��j�����l�f�������%�3�Y����|�)��d���Mr�NV�d=8��G(<?��	�@�mM�&���W�S�C���e*�_�.�����u�����/[	���H�8?]���zy#.�_,d��3�����I^/�����h���

V�������O.�`�����k`���z����ea��a��Lj��f�5�v+-��8
4�-�T���R�9��@�&2��g�����jx�&P��i�<-_�e�����a,9O6�BP�}�m�0K��'��fZ�U��K��(�0�P��7�����q��Q#R���n�\t�k��������������=����`F�m�������@�`N�
+�K0|�����k��g�(D��P(g:�S��+�y�,o\�yY9����I��%�ZS+�4&��<�C��0�w�A�F�)��i
��-�hA/��0�4+�
A^���4�h�`\�I���?��uBZ'da��\��'�
����r�k�~'%�j��+V�mE��}��+DJ�T8D�h��	�!��B��8E��4]V6`�gW �X��u�
<m��7��7o`i��3^��o���v:�9��]��(�����S�v|*{������n@k�����������|.J/��.���=�����(|���S���y��ev�0�O�
�>+�
yU<�JU��������Q0�~���mI^fd�Z08����~dH��0C�UJ��������,�)P�`@�e�C2k��i4��6�G�96a�%$�Uh*�]��i��uG��#v��9K'VG�)z��tZ�M�K;�+0�\��e�=*��W�
|�P�����"�/�ik��Z>��c�x,�i
���~0FJ�q�H�������X���w�_�_3�.�W�r�Z�l�E���+�����w�pn����Q�yf��"��+�vRF���&��D�V�+����3�o��u�;��fjj��]M|����W�h
�\���@�����>.����sR4���I�)���������h���9!H7�Z2th�����bi��W��J�R|��(J|Nt�T�3Y��"zg�?��y��������8+?��#A����<�O����|^MR��~����iy�Z����
���	�uBZ*��x!��~J�g�F��oyC����n�r��A�������L��(�4������f�m���'#?�����:X;��Fl�I��I��%Z�c'���wWP���Jn�RM������EE���U�R�c��s32��H��3#SnFDcALu&R�1R�R�j��W�AJ4W>K/���E|KG>����� �&��'������9)6Huj�T;D!���(�%���`=&��`'W�����bC�V�G�W�c�Ij4�����7�P��1�3�������T]}�A��@���II��mO�Y0@�d������Zw�M��~�~-������w* �}L���$�H#� �V��/�*�����m�\>M������D��O(�������xn�s���)�rrsq=��Q�����q��!��$^o���B�MPY��E)N'��t�t8D�5��Y���6��m\@�@�M#����~�W�z������E��U~K�xT���4�D�S3�5}�
b�Y�vd��T�T�'�^�5ktX�
�e�j����F����e�-f�n;^?�#8J�N��HjF���j�~�Q�����������^U-�W���' �P�+�jIf�f�U�F��O�Qa��8�r�U)yQa+}:*�U�/3�T
"b9^I�8�������XR1������I9N�����f
�r�X��\��"��~�Kk�.o��cDxa	/���W+2��y�T.��X���19����>�(4�}��89�h�zj�'aJ�������(*sl�V=Jj�X��X��?0o2
�X2�?�o�:,4������'��T�%G�C�9������R_T'��fWe��$Q��_P`x)A;t���
�W�_)��.��]�yGW�����.����f�����Za������a1���\yw���v�5
�Xx�P��+C�i�5��h�
�sU�t����\��d�T9D(	4f�0�V=f�,���@7
`�������]~�c�n�f���wlV{��������.�f��
,�]���X+�]�
����U�������zLU�S���5u������:��+ms�f�����5�eR��C*(O	{�)a���C��-4���W[a]�m�=	�R�{�H��kO���������0�?c��nt�������q_���S�Sdu7q�q��o'�](�����q_�(_�3�#gH����q������=�����>�"��#����7X��N���,5X�KIV0�a�i��v�*�gze3X������{Hwn	������d!�e��=����%�m?�bM60_�3_���-^�������#�]
����-W'�uA����x�����_�l�p�{$e={>T��A�N��YY{�V�'B7�p��#$eCk����.�ie$���������2�l��G>(#�!��5	�,PF��H`���|���>(#A�����H����Twe�����<,���f�x����*�#{��4Y�����4��Z)�a� ���I��}6�&����/H��:Y|��+�c�]��+}s��Z���l�T�+pU�K���Nk��tW��`��J��Y�"��e������@u��4�W�e�AuuX�������y���U�����zKNWpT���:,%5J����C�FiW��~�b�!P�_�C����?�4�2�dPF��[��Ai6�������E���L(#��e$0_�c����������2���w�����y:[���r�F���*A�'�D^��mr��#����'�e22���h�����^�M�|;�����y���K#wu��r���(�:^�����*��M� C+q����:�����D�������l��<Y5�s�U�u}l�L�PofC0�Z����Bs�b-W�!�Ck���h���T��@k��!rq�2�R����'�����Re4���zH�
-�[�J�REKh���#zi25��5�G��v�^��W�$�����z�=�z�:�+���.2��4�D��T'�:�'P��'P�t
T'�:��y��q�@u�P���T'�:�����^iNn:5�CB�7�9��]G���Im@qR�5P���'�	'���	(N@q�P����d/e^���Jo����dU����d7��D�����u0�
�=�Ik����������)��PL*���:����3�-��s$�54)&@��UMJ[��M�^J��I��&���@�=T
�(����J�@�b�g�P�'�@�	hO@{�k�	x���&���^@o�o��^���7i�7����/�f����t5��������a��t�n���!~�o�K���H\/��.z�?'(��O��$���b���	���w4[���z=����)YH��"�W�i���>A��J�i��w���}��������.4��l�����������)^
�/���7�����zr���A<�^�H6e���)�����&��^|�����W���|D���gjLa��
�o�TO��YW�	L���)�
�������&p^Z#�h���? ?�`yK&y���y�3���Rh��"Y���v��f�z@�{�Pt~>��)VI�����\M.��b�W�����T6
�l}}���e�+�___��(u�����G?]���zy#.�_,d���2�b,�^@�E9��QA��E4���E7�W��h\
����T����,�I5|���G-p���F�l��i ��4R=7n�c;���ni�A>��|���wi��m7��j�����4�t
��g����%���Z���r1=�'���#Da���6O�\��c\�z�*Y�%u��L���>;�MLx�;w���dl�2�z�����X��lk��M�L��hm0#�
����D���j0'�Z���DO�ts��q�����)�d���Xc[�����&�

�K���~�!�B��Y�����r-7��,�AKd�<�VViL*ey$��a��2����SD_�0�[���^��a(iV�::��*i��r����fYZ'`���N��i���O�F����['�u���;���*��J��XT�I%���=��g�C\�R�-A�e��=+n�{���T�3bl��lN�.�&�
�����������o7'Z��`N�L�vN��~����d]:�������=���[t|J��OeoQ�V��V�
h�@Z3�5#Rs�y�~x��E��e�%X@��������z�2x�^�>��������I�!�.7����U��e�l�
�f����3oK�2� ��������
�]9���d����2L�Z���,u�������."���	�����L�`p
9F*@A1������L=(��9��9�2�9��s�Y�����dN
���������aQ��/�S�bNUI'sj��ogNfN��bN��s�bh:�SC�@�7�����/Ny��M��)5c��
U^\�2��1703\�����#3�c��H�K��5zX-�S�d�x,�Ap"����)���G�����kG�3IC#vA�b0��0A�j�����ym�s��O�/z���0]���/�wd�k����Q�y	��"i�+'�R������D�V�+�+��3�o���v;������$VQ+n\B�Yh� �^�
*K1}\$�7���h�!����S��MI-��)�h�*sB�W��5��[F�
��:�a}@W����N�U�sb��Z����\�w��1����������G�Y�1_	
�<����|
7�����"��;�>�O����������NP��R!���9V�S�<;6�T}�+F��l�t[�;��
�]M���s�L����8Y���Q�Y��k���199����������������������d��2��r����T&.�Z��N&��s�������I20LIT+���o�R�<���D����$��D��hL�zKS���ieFbv���r3"��z�3o":S�)�<6���h���l�
�S�+������������9%6���I��I�s�7�<mN�
��N�3����������hfGJ��4E�f�j����X�� �1H|����X�R�	��?B��~��P��j����U��I{�}���r���??�f��2w:�����7���|b=)�fd�mR�I>��L����6����)��#m��E	���~-��n���|_��;�����	�F�A|��e_U���;�[�nxz'�w�W��o�w�g�����ytz�D�y�|"2T<7=������rrsq=�������iX��!��$^o���B�MPY���+�H�4��t8D�5��\���6��M�����TS7)��_5�-�m�o����*p��0��j�����AtB#�Ue&b�~4��
F���5:\��j5Z�|�@p���2�+Mt�~�N6W�;��^���oi���Y�kb���v�\�k+�;��^������4^?c��D=2� E���>����k�O@���WR���y���V-���0MG���j��Ut���O�������V-���Z[����x5s$"����J�4c����c-��r��9m���)���c-�"�r=X��X�W�39�vyKq�Z,�Kx���J�9#����Jx�^T������\���1	�u���>���,�'}r��'���Q����	���)�.��^����kZu�*�������MF�K���
W����v�T8����
!�������R�{'��f��aJ%�N�_�axG;Wpt���
c]k�RS����sk�}�,S���E	�G,7�U@��Q+����������,�T��k��Q�z��io���C�bH5���`�s�n���QYmc�M�h��&FKok�M��Jhz��7�}G}�/��;��vh\~J��S����\�f�����l�s��v��F������k��o�l�����n��&{��l>:"p����t���88�t�f�88�t�f�nz{��{�Ns��7
:k��Ag][�-p����6w��5������+>����l�phM����x�u:������n�S��mE��n�������z$dt,g�A0#t������=3F���F��:�OTN�����>w�#��t��i�;Y�B�|��h���}RIc��J�������?'��D3[��Y��=U.xNK��R���n���{�Y�Ly����i��#���p��h��� �	��k���Z�+����d xY���0rYj��8,]0xY�
�����ea6�7���.?��^9Y,�[:H;����3O���*��5r_�~�[4X|�������E�M\������h�	�����C�����O���=�^I*-<�����Ke�=���7��
�����SpW�UR��F�kS%p�K�R���r��V%p
W��UR6�����b0]nD�fAu���\�T(c�nY*���ia`���� x���l'l���l `�9R0q�1R0q�$�����6�8��p�[�����)�8��&��R/o�1�q�;
RrYj�q��6�a��6�a�i	k�9ze�0Y��t��M�7��V���lg]����uM:lkp�5ig\�~��&������;�5�����7w��5����qp<5�����48�;
������4���}�Ag}X�5����:
:kms�e���&�WL6���f
*ku���F.K
*��b�Aem6��c�Aem6�7�Lo��fz�����d
�d�	���R^C(�t�*l%c�[�
�d���8�����P2���d �Lw���&����OoA���0N���0@��|0q�O>�8jp�`�NL�u��8�-r����&0q�U����y����t��7��b���U�6O���&E��x�Cw�DNH�d�)���h������d(���^p��&r7�D;���[���g`306q��d�j�rq��j���.��
]��:���U�;7E�*�����������,�F�PkP��$���X�W����ZBk�G�J���v��JJ�C����R��K##r�C#����hj����dZ@/���j7�Z��Z��G��dj��k������8)GU����U!��������0�Z_�!�*��^lu��nH�����������T��@���!zqR�����?=i���S�x�����d\1�D�{hfA3+O��5O��5�hfA3�Y��Y���f4����,hfA3�Y��J*�f����f�:5P��J1�m�=$D��Y���uT�i���v��5]�,�eA/k�@/k�=���^�����,�eA/zYIe����������J
���R����7�D�`*������Q�&��u��(gA9�Z�v���D%��3t��*Z�
���*������LEk�d�����E�IP�r�AQ�V�v���E2�����&@]�Uum[��u�^��@]�+u���n{���h���/����Em�@Q^���m�2�h��x��r������m��EjYP�V*�Z�����Zv/Uj��m������/�f����t5��������a��t�n���!~�o�K����-g(^����s���5�-�*^<&�V��;���b�+,���t��x�?���H�{�Y���/��*�M�J�;�������3[h�67#����;�9^�=��!�er}�����x�^O��z1���K`�IY��{
���k�������g: ���G4+�x&`2��o��K4N�|��tq}��l������l�f����7��%3�Y����|�)��d���Mr�NV�d��t����!�LA
�`���jr����:�nM��i�ek�[��/[	��?V�E��(8��O�'�^������-��~O����|r����������_\\e����C��$��=��?�Mx�q���be���� ~��_�#����b�10��8-u���P!m�`����*��d�4�S6`����t�
[M��"?�^-���G����oz�M��iY2-�e�x.c�y��A��M��F������'��Z%�*Y�%uH�L��	8v��-��D^�#<
�z�"Df^��:��0<����18���E��P(�4�w�1�7���e����h���d�Z"��1��JcR)�#9�~��mD�"�����)�"���,CI����!��u�P�H�����%�4+���:['�uBvH��L��0�{�������1/�P�H�5�^f�F�xY�����!6i���j�*���f��&�
�5M�d���;saI���7���(�����_�-:>�o����(}�xI+��f ������	��X2tB��H���i8MYt/�h���|.��/��.�|�=������a���]xy�<B�2���C�'e2�d�����*Z��_V�6�P��/x\?c�����2��`��j����9UN�2�@z�a�*�zOVRC��jx�4�h����Rd���(��p������Yyh���AS9x���.u'u'�X�=�oE-�/�)J�4yZ�MQK��+�	����e�*�W��
|�8�ga!zY�.���4�F��sJ2f�G�J�q��
ut���R������(�]��$u��{����L:�M�3j��t����t=�l
�]�~���~Zh�F�j�;%w�O�EhF�%��1R�WNxd����x� ��b^9�X>��|����&i7�J`A�u$E=j���Ff}�c�Cb��_�E]}��E=�W��DrD+NU9�F�7���c>���0�Yy�/����c�<�3y~�Jr�n
���9E>�w�}>�����y���	h��Z'��B�����S�<;6�}�k�R�l�t[�;�$��8�&�=�{�������4���[�fn ���$����b�=U���N��T�S�2���2���r���[�*`�";#������132�fD4;���7�F~S��{a���4C~v�P#�g�����b���y��S7�c|:5Ex~����(Z%���`#���`
WM����bCy?���W�c�Ij��������P��1�3��������7��| P����d�a���'�,����8?��sf�����^ |�;�����:�H#� �V��/�*�����m�1�fp���+�d��Px���S+�6V���y�[�������zr����'��#k�C��I����3"�F���$>��Q�P��	��p��kFu�^��mh�j�c���]�h �^%�Mr�.��tI��+	DO���V"7�X��:����Cu�%:T�_P�C�������s�l�����X��+�91��q���q�Yk?�����,��t-�	���J*���[�����-4�i*U�Uu�X��E(yl�V�O����.�e����X,��*g��_����+�e^�?V�,)�)D?���LMR����������r���qQ�������/,��������1o�J@���|�C�w�����x�����E��N
vR������l���1�$���s�%f��&#K��=�����BCl;��*|MJ��X�>T��W����z�9y�5)-D�/i��$ro�P�r!��vG�f���"�*"���F3j�<l��
�e�,�*E������2�d�3f�Q)��Eel<c���$v����*�"	��S�e���t������Ak6*�
�{".-a���Y��]�i����p��J��$dD�I��n���F;����6'�a_��:y�t�?1��e�AP�'������"	`������M�AP�� (���}T	t�k:iv�����)�x�����C���4��6��y�������
9�M3��)/o�9��.�]V�9m���Us�NuY��bzr�����t7��l7���	m�:e�=��yd+�F�\�@\���`����s
�v)��R,��;J��gpxZ`����`��.x�X�K�)�P������4����xlK�%!e�=J4�����c{*/p�p������=� ���S,	)+k���A#%p
W$))Zs;����wmh;�h;@����>h;���
k;��
���m�}�����7�v���-i;����s�;*�A�Q�@���6v���2	}G��&�����%�]EPrl��>(9��*�N����AP�K (�����^�����C�Ae�� (o��}�������se��AN9�v� 's}�k9y������C7'��<����l������5���<��;�p�+����K��m�K�����7��8t��v�������c��@����k;���4	�-���E��-k4x^��y��!@�5)��KMl�������P
'�eB
'j��b�	������=Gf��	[���W�(9��F��cBs}8s��f��`�r�������G��6%��q��������u�Ao��0)F=2��U���n�=-�����Ed!Q�d�:NA�KN*[�N�����J;���*�j�R94��C��Y����%'m�`'��RU�J�
!����=0@}�+���.2��1�D�
�TR*H��� ��6R*)��3 �Z��	��=�� ���j�v&��O�!�WBj;7I
1��
2*�� ���� �:$�Q-�����'�QAF��oY��W"jKz�MT="*2Q���k��YQ��ZTMd g���G0�7m���vD���h[V1����PhY��(�/v@�����0�|K e��h�K�c z��i7�g�k ����#y�������A��<��Me�s�`2lS�!�-*���D� ��'0ZV���O`2��[Q�sb��Q��?]��L�ry������{2u?�2���r��^��fF�=������l�V��1A_�fwO�a6�'�h�/t�<���MY-P�2V�J����;�2['i�$4[��W[������M�G��ML	�Y�eS_f�ke�������~����\0��1��f��j�s�>����x5D�L�o>�|��������^����A+|O��x��6������?�J����#��<Pc
��TP|(x�R8�N`'����UpB	��.�O~���>\}�����I_e8s6B3��G�,o��p�7�&��7C��c�HV�&�]'�Y���q#tr~>��)�J���kr����������_\\�!�H6�x����a�2�#�\M.��������MS�d����;�&_��$��|;����m'	?�)���I��H��|�v����/�S��f-l�@����"�%0���e/��eU7E�x?�~e8�^������(�z�S�}p�"�u�.B���r��F�V��T���w�H�Nw�!��7����x��	����{\H�����H�xB#���z�_��kYz-�e���Z�$�Y>�)>a���y��A��M��G�'����'��Z%�*Y�%u�����/)�4���m"q������b����=\����GO��hQ����PVz<J�u1�7���e����h���d�Z"��1��JcR)�#9�&�~��mD�"�����)�"���,CI����!��u�P�H�����%�4+���:['�uBvH��L��0������:���Oh��?i�I��Z���3��92�J1vI��&
��d�R=�a�7F��L�T��
3[�Y+�i�[%o&��-1���B��M�+������xGy�=�������������[t|J��OeoQ�V��V�
h�@Z3�5#i�Z���xG�^�sQ{��w	��q��x�v�Z�������#�-��V#R�j��[Y������~Y5��C�����p�L*"zE�2� ����������dft��0��U������v��,�*�� �w�P�9$�6�FC�n3���:AAA�[��"%n�T��
��P����)t�&t�!t�t���gV,��b�8%j���S�������*���l�����<�9�H�iFW������?���)��-�*��[��a{���
�k����F����5US��<��j�W:Mg�'��	�����)>��R�/E�)B[h�F1�Bs���T����:2U5A+,R�2/�x���r�� ���$���d��WN�+��g*���4$���+�Z��'#4��m#��qq���p�����0�v�n�������S���Yr�l�:K�����Z��n��,�x	l��W�y�p�+��q��y]a�/����w��|^�Xw�*�#��J�C!'_|J�g�Fv�oyE.C����n�r�������6!�����1�&Ge���u�H��TG���u�M��[TP�)k�LY�2��|Gi�Tj�
V2+�h��Q�J�<�&�O��&��M+s�s�����#���?��lb@��&4��E���C�6h�H3�`��&X�4�	��X�����7�=c����gL����i���&t��b�?��� �\�
c�Ga,8&�o�q���;=Yl��@M���*y�7I��a���2���??�f��2w:��^r���|�=)�e��m��I><� s.e}s)s���eX���/����
�N>�4R�k�-�����UM�i��vO�g)����]�Q���v:>9��g{g)�������Ag�Yt6CN�O��u�M�y|������\\O.y�?��c9 }��<���tyF������g���@1
0�z�h���w�
�N���U��+�
�+W�z������]�c8��}���c�T"B�����C���<J<������=�$��$����l�D0��X���D�q�SZl�@li�5)����uk�@������B�J���D�J{%�����S��TXQ�tT����(�\EyW�mTTK��
3U�T%���VU��X��2�������4kUL["����KR��C2n�y��v��?�&�+��5���xm'�������*f3^X�u��u��s�n�i�Jx�^T��t��5���������I	�D
�zrl�:���{����g��c�:�O�gy��~�B���[�����'�x�V]\J��b����y�|�R���}��a�!�����%aD,�CH��:����������Y�!�$�b�P�&����T7P��:���+�hOLU�
�Y)�Df��$���O`�9�8�
�Y�EFY�*@{�p�n��Vn�Y��m5ks��f
b�h�����!S���������[�<����g�C���������j,��fK�
���fj�8O������g�:��6�~f[d
��fh��D6?Zea�jL+WS���qi��-�*`j�����$����c�L�����C��L���L�Q\�����)$5'&�l�7�tkx��x2��W����qe���W����qe��y�2��;b�v��y;b���f^#�L<Z�?������8]��X�V&���>�����F�oX�]*��>�c��(��	�v�"X��bz�F:�l���ol��f��������1}{$�`@Zx<�e�$W� {���'���d�E"��x����g���� H��)�\� +k�B��'H��x���lh��fQa��0,����"��Qa���,o!F���=���(1[�����}��3��p}v��Y`���\�[���p��K9��@�W_
��N��f����rP�1,��@��3u/%�{��@���6?��M�m��[&S��c���R����Cw�]p5��Wc�,p56�n����>`jv���Whi��1��L}~��Y�g�-�v���e����j���k��f�����305�25�&WP�;��6;��M�m�3��R��2;s��4pt���W�����l�Y�o�����1,���s%����s�|�@��>����'`�]�v\���%��7��{d���}�������7�h���w�u�����}-�������[>Og�x3[.�h��\%h��Y��kR�MF��jxNH�d�f(��2!�������Jq&5�]���7WHF�)$�{��<Id��G6l{d��G)$�J�*H7�]%"�WS�K��S}E�U_Q��U���5c�~a��Um�X6(�H1s.R)����R@-�w��Z��J�-4���������`��p������Vw����v
�;�J;�Ze�����Vn�J 
��d/'�u�RI��nZ*5v�MK%���i�41�� <o�L�S��N��`��N&;��]�d�	fN������ +�VYY�@VYdeC� +�+����n���f7�]V��������w��7!�X�ARI$e��ARf����rAR�&��AR~S��^
. (�����f�d�U�����f���dT�%�� .���2�����;@���A\�}7A\n��}��R|qyo���n!�b����ve�AP�&�AP6M (�AY^��w�6��gAy/��*(�t��t3���E�r�Ws���4���D:�?����}���7h�&!���G8]�.��hF��<�U2O���U^��fF�����	�t�Xn��r��M�	���U�xL����C�JX���brs�L����o���}�WwO�j�~�\�|���7=������^�������?�����F�\�z��~���?\��Ct�
���|	��.�O~���>\}����'������l�f����g��%��o��L��7C���d���Mr�NV�d=8�����C\�~�)�-��&W������C���vS7�
��M�/N�"���'��S(�%N���:���l�fVS������O] N�����i�=EGF�/�o�����'=�^k��#jC�#�_��?*�C#x����-c�����6.���y�V�`L������t�1��b'T�+C�i�,�%�'�|�d�}�a���i9O�q1LERp��}~���|������������'��O��}���DC�b`$�[�����<�xt<N��W��av5���e^V.d�Y��)d�������r~V�g�yY9�)�e�<�\VlL�ey$� �a6eY����V�wRb��$\!�I�Os�0�jC���������6��>�D*2�h�����1O�}�W�J\�2��i���F|>/�������?|>/�3�~P��
��S5<���"��2���{��l,��� +�{���y��e�����b�!�����AU��e�l�RERAPp��
��M^fd?XZ�
��l�rl����������d%5h���^�`?�����lsHfm08��\�f�O1�u��AZCk�v�>m1�����/����T��w<j���+�~y��|6�[�.��RO�F��s�g��B��#�K�1�S���G�����
G	I]#��kM*�L6���+im�-�0]�au'����N��`;��:��>���m77���������X�|>�|_��k!i7�o=�Q+2�8������o{{�v��>}	���A��Y��r���F�$�jK����F\nV6��Y��+de�l��,�r���M�En���4W����V��$�����<����<���3���%M��\��l��@���)27;��T�U;�FG��(��OK��Vvv,��T�A2k��g�s��V:��d:?�:/*�z�w�Z�S�Y6	j�!��UnY��R���a�Y�Y��?���g���v�,1��,6�vR]@���J�MR#��_�S�1���c�g�;-s�C^��x�.���@�����%�8m������d��JRS!�x�X� v�kp���'�L��5+�U���B�����=�xO�S?�n�����8��� 8?�����I�~J��w����� <=�O�g������xn����"}9����\�Xh�	����5}��<���t�F����W�g�O��jL]W��!Z�Qh�~���
Cj�@�>�]`�R_%�Mr�.��tI�W*�`��f���q4gR;~e�}�4%��>�97������~1��f�9�����[�n�{$@u!SFB�}%}�����t#<2������� �_Fy�+Pd�h�Z�@��Z���3�|54�����y4O
�c�yJh��W�H�!�'������WT�,^���8��-���f=���'��<��4�.-i4����9Fg�v����`X�2�����c�����` Q�|u���o���������
O��h`�T7���A\�k�O�^��S�������A�l���qG��O�:;
h��������_�\�9��#���]��: �Y������	�sO�gkH����pm�acO�:MV�s������l8���%=l�	\�$`�����gkH�_��������O���U�d��b���
�9Y=&�=�Wq��n�n���$A����3��������F���4��T�9�
�u6P��8=�iR��f�l��f��p�M
|��y[��+���������H��!1�[E�b$wp%�^cM�x={!�K��&���<�%� �eQ�2��9�Hzh7M��4�Hz��+EcK���(,��/W*���C�|)
K<���),�P0_��	s�h�(��a�
%"�J�e1��k^����H@����Q0��������@<����-��@����`QH�:	�.>�cY���o����^ �G������:�5�	h��:t4p��={�
)�1��^^�N`:MV�s@����mn6�<tE'x=�v�(�?����x=lYP?�$L�����<��$���N�g�:���]qw���lA��G���*A�'�I4r�����8!-���P
'�eB
'j��������)u�����J���"#R"��#YE.xUA6	�����f��{�j/+��6q��2����yYE���D��� ��6�DY�	e�%W����DW�;p�v���4���Z�j��Ri�e��=�i�IK�������J ��Q�=V�/�d�u
pP8(Y�&U�����c'����A�c��Hs��0h��*�O���a���������?�#=�c��%v�d��fn`����LT�%0Q��&J^�(`�P�D9v��L�>R�=f�Z
qTu�-:(��
�'y`��}b@����>�O�H���>�t��t3���E����9����v�H���%^mf�%>�G��9y~�of/��-�������{K?�g/�)^zF���	=��t�<����h��~���.��b���s��{J��(�/����,�����a�B�|���uZ����������A�f���������/���7�����zr���A<���� �����}���k��������6���U�<�����h0���=3�D
���#���o/������_	30���Ia�<�G6�J���O�'�^��J�>]\�`���f/38���? ?B���"Y���v��f�z@���Pt~>��(�I��c;h�F��]��G�D}r5�����Y���o����X�>�|�����/��R�&{���})<�����������{���}���������7��m���{�O�>&
������A~������@Q�)c;J�	wy����c^��Z�2L�]�B�|�@Ga��hS/����b�?�I�z(T4	����p4M55'��3:H�6��u�Az��T�����?�#+'�8f�6�pX�`f�6��$�~��V�S���3����|'�?���wym=S�P���J�G����
��
���5r���Ld�T�F3�]��L�
9�3Sa>S��L�d�T��F3�]r��A
*����,���0O���9������� ��C��h���w��y����-�?�]v�_�!�B?}~�w��/�{h���
Z�n2~hD��/O�yr���U�����]b�)�5�,o����3�bu�\la�G�r�n���9}���?��N�pD��,O����>V�e�l���3P�����(��z����|K�u�a��E�!��$�N��b(T��3b(4�#Y^��<{������������v���R<@�%��!y9���>�u�sN��g8���#��������/Gch��,�d�l��;�u������a�Y�6�2{>���l�����m�jK9xg8������(([e�uo��5�M��m����#��|�����|�(d&�|��
�������L������G���.kt2.�P��Y���c�W)F-9�65�U��
���:��/V��X�jOjh/f���_'���Fy�[�9���'����r����l��E�pq/�T�|��r��\H%�b.sq!�����oaF�B���3�~>����E�T�p�SF��9�
*b����afSQ��� ���yN
���T+��i;s�s�s�s:�������j1I������KF�r�/]�"����G����G����F����wH���DR��/H�xQ�#yG�d�Ef$�y�E�%�Bx����� c���u>��/��.yZ��q��x�n��d����<}�o�h����yY��JY?-+8oV�,�������"/��C$�����6�����-Y��a:*9��M�`����l$S���L��a:��WS�����v�C����i4�4f�W��ZV��r�����������T��+��S�������bNE�����������T��+��W���}���bNE���4��4�4��T��+�4P���
���bN��\��i��i(�i�����W�i��SK;�����
�7z�����Z�E9��?�d�g����)�S>�p��c�6���+9SQ�HX����9e
g�G�J��&��{���O�,�>�rF~������k���W����_��d}�Y��p������M�MU�M�W�z������E�B������[���mG�l�T�6X.��;��{]��J������/��kPG�_�O��y�������l��U��b
����[#��u��4���K��r��"zW-hu����6���r����D`����RWX��q������C�|��<��tb=���
k�vb�����*�s���{���H��H���0%���������m�m1��$��A�]����1���`d�k��[�2iR0/�o��y�Xx����7!}�o(4O���O��t#��
���U�y������:%�[6q<��&�0q�*65�
#�����3�
g�c�S�]�����C�o���cjy=���yvl���m��J�z�W��(b>��C)�"����2K�P���Ag�Y s��������#��)��)���r:���^mab�}�:���X
\��0��.Z�Mm��X[�ZcO�����[V��-L�����UK��{��Y��8j�_��D�+��Jt���D�*���rn�����;Qc?��H��Z���
�~���>[v��,��X��9�%����c/�c�4����e����c���c��`c�-;���c/�c�4����e����c���c��`@�-;�m��&������1�,6TWFs���U�o�����??��Jg���
���R9�yO1�:�����?��x��z���_r��Z��S������/�|����Y���Z��9��j�j[jZS�Y�z�8
[8�CD��\�yhYgf��5��|N�UvYsWM�D�?5��v�T�lQ�9Q0>���]��5?�dNr�e����J3�$O���C�Q��)����q%��a6���<��uylM��<�&}]P�������k
�D
��
��U<7
�v?[of��Ma���I��������3������%����'��[���)�v }����	�VJ���`pG�M��>!Eo|�@iq#A��|?27�z�8���pg�]��L���bpy1�t3�;q�GTedF���fK�O����;�u���\e�g�m4�4	�s������������X������������(��
��5�G)#�U���o]i��2k`�cU�-��Nc��iRi�@qU��r�[���/�L��i~�c�����(v�]�$�]\��\��5��.�Gf�h��r).�4j����&��W��t��P�I�`+P9��?e��1gJ���e�\A�N�E���v�����5���PG����z�g��������u������h;Jh��p�G�>��p�
�'M����N���'M�H?�
ix��)���}�N����o>�~X������m������0N�vJ�\��Syp*o'N�����<��7��tP��i��.���a�<xl��}�8`����\����Y���xV�Me8S���r��*\f�XVyI�},9F������O~K��4�H�j��� �s�S��r
4'���Z.�����M���)������������#�P�X1�(6*MPmdr��AL��V+�5*P��U�FS+�4�������N �^�ES��j�ls��B��l���'6�)�~:�r�{���ht���6�;�M���7KU���d`k3n|���9�
���v�����f���*6ms��nU�F�s�����
Xl����6��������_�C���G����q�{�GT����D��K�|�GT�����+����
�';��]� ���I�)���qqJS	�����8���� N�8�M N�R?��.�V��@�r����t N�-
�'������DO	�'�	�'Y����)�
�HO�;����'�E�8%��T��{�Hq� �8��"�8%K���dZA�:Li����<��]��V��a�+��d}��S�m��:��������$��Wv��t��;fc�G<n.r`p]�+R�Nm��0���G��-!�6��'!p���hr"	�Lr��"�&�������#'�vmAf���`�z�v�Z��t+�o;�f�-��$�6������l��b�r��������������om��^{����$��I�>\��d�����#����-h0�=��A�]���k@�I�+�n�*��{���p�$w��!�L��������b��������=�]Ft^���/���/	���y���%�}�|�#�+���#�&i���2�+���U[k����
�Vy[�A[+�Z	l������,*�Z�����	AT\E'�����X�,�v��m�\���+s-k� ����r�M"��6p���L���v�)M�m�������f��-m�I�n��nP�oP6S�mi��@KRw[Gv�t�|��N���t[$s�D�<
w.��a@*R��tq?v"<��Ar���X�����\�&�����UK�U����%�����	%����0���3�������O�	m�~�-�,������=��rBIs�MvC�x��3�>D��N,��:���N��Nh=��N��N,I[/�K����K�$_G�tR�z^�����O]��rV����F/M�G`������G������[[�?��~��� ���~v�P��#�&�d	�@?�#pi����$��S?�#��?�#��<p?^����^&i�a�H���K8$p$8h����wG��VW��H�C<	�D��$p������wi�8�&����Aq
�)EqJ�@�2Ho@���c� M�i
�6���O <%�@xR$�d	�'���'8�
�S?�'8��@x[�Slq*O N$�@�R$�d	�)����=<���~IS�����I�ki
LQ ;����Q�	d'E�I�@v2H{%;��E�	d'��	��\E'��j7��@�������,A|U�!�����$A|U��{_�w�W���u%!�*�W���U�D
�H��2���4zi?�~�?���.�G�������V�#���Z]E'�#��z@ u��6��$K�Gr ���K��G~&�����p4����$����d_����$�-���E��-k4x^��y�$'@�5)j�4���7Ls"��+��?����0I[!����a�V�d���"Z+�'������ZV���VQ����v�d��SD�3���?��+?��^]��AD.~��D��af$k��Rm���B?Z�����V�f@��1��Mh�����|bk,�����x��eW��G��Bom+[it���fQ��
�uM}Gh�}�H���<j-����o�n���e����
�*:�so�?W+�*�xo������e�u��qD���w2��R�w@K�]��8t�;�N6U��h�$��v#S�p�;����J}�-Yt�ndq��t8Y(TI�\�x���B�ZHx��11�X:������@�m�s6P6�;v�s����m\��-�OA}j\���>�)�@}��OA}
�S�n����S���������>��a�OE	�����r[��}+�3n�Z���<�������<��iEP�����<�)(OAy
�SP������<�����A��tou�[T��m\1��N;���="��jB6I��L���\M��j*���:�QEA��zT��t�����A�
zT������G=���Q��>
��{�G�l�����{���t�zT�|P�*���v��;���"�N��S����t��;�)�NAw
�S������M��@w�-��O�A7��\^d��}5�O����e"�w�S�9A�?7����������<����l�����u�a^��_~'o_���l:Op�d!)��,_�����-�*���o�;�������7Yh��b3�[4��#?C��������5��}=���� M��Ka�����}���k������(j���*}���G4��J��d"��G��_	�7���IaF���O`~��tq}�Q���f(r6J�C@~�(��d���Mr�NV�d= �:�Px~>��(I�D�aO�&�������m�k:�e�����o�\�|���M���/��j��0)��.�O~����wX�������9V�+�A����
A���w�1�������RQ�����bh9i�H��R�:x+Ga4X6�Km���@����Q��V#w��X�Q��@���,8i,u	��` ����/r��B"���+��;�f�,YZ��e�X�weW�]eQ��dK��$�d�^Y����&��J�/��f��������C�JwI�&�7<����Vl���6��j�M;^���d����g��F���[����H�o�����8�Q���V��lk����zj't��2�K(�/�$��p���f�~�,���3�@y��,)|E4kL��l�GJyy)���'U<�
)A+�,?�G�������t����	�`���x%H���%��
��fa�7
�R�s��
!~�`1��L?�g�+<�t������-�3J%����Hf�t%���P��x�S?G;���kH�1�D��8H��\�����fF�V�f
!�5``�)�w�&Kt��W����5����#1��r�4W!�����������|.J6/��.���=�����H{���s���y���v]R�@�uHvGi����AU��lu��n`N*�3���v"/�R�����u�ga9e)P9O��t�=�:2���s����N�~���IP�L���������`p
+
4�a��3�d�ex*�^��j�PvI%�$�]�F[�L0zC82�)������ho����f��s6�9bE�RF�V�����p���j�������0o6 \���1�SJ�3��)�3��GG��UiFJ�����9��K��[��qP����A��������-���l��e���d����d|�w�2����L��	s����%��7��t�Y�><���1Vpj�}_H�G���r�Pb��cS}o�5��`o��Er|3{N����:ia�<�ty��U���t�L#�4 -(v.(J�>��[�����������"X^#W�����!3&�(��LOV���D����S���R������H6"������s���)y�i���j��
�r�(���;�}�.4�KJ����5r8�Ed�Mxbc�}	��Dlj���>�~�r[sj�.&*�K��h"��w4]��������0���-&Lk"�h�\�
J�4����4��j~MH0;�&�� �_2��~uz�#*����R��M��������Pd��K�Z���o�D�=����3�������7�&��v��|�`y�[���������6�X�<��'�
����<~|\%��&����_�S�4���c�g�;-s��:!��x�.�������SO���oTa��� �kn�>�~H[�������
N�!�<�N����7Z�f��.��U����_u���Hq�bK���+)���_s�r#�U<7u���R�v9����\r�M>�_��>�a���M&��$+V��������A�!Z���z�����ipq=n��
t.W�z������E�eo���&g����g�r���6?	O>]i�\]/-B���[�M���4n��(o� 0��XES����+Y���#���q�Y{?h��c����������E~���Di���Q1T~JSH�"")�(�1"�L*v^fC��R$�x1������%
���C���������nA���_f�Au�Q��?��<��TL<x��S����q�c�g�����n,^�����[�_��nu���F	�8���8�������8�������,b��2�u�@Q�X����ig],G�YJ��[��J\'���>�-�1v�+�)�o�J	I�2��hmQH+��>e4���^�(�nj�`i�\#c-��t	0� ��S��Iwe������8�`���
�MZ�K���]�f���
�QZ�M�(����i�=�D��_�\����s�R��\�8�+���|��`�O+�����S��7�\e�o 7��,��b���a�Kr�N8>
����B�0�^s�'0��D�]���o����b��C�k`1t�c��8���+�C�v�sX���Mb\��������N�<���/�/jF�aya�	{B�����	Pw�gK���I�E.Iu(9H���O��K]�Lz�.uq3�iF��f����fK]����$��S��K�#:c�:�G����p��E�ar-�K�2�����vu�;R��	��n�vCS�u�|s���"�n������'�?|���K5���+��1�����F(�~�5`"��3c%,Z��=[ASW7P��LA]L&��jCrlg�4����4��,�\��ESw[G�*��w�am`�-m����r��L���kZtai���4��^�mp�����i@Id�@�)���A�Q��r���)<��>���u� �����%�Px�((�5���/��A�����
P��z
�=�8A��sB��]�;��F���(��	�{�{��S��{����>)�����=P������Q/<&jh��W���h�jh�T��H��CJ���p�b�a���^��'�;'G�a����$�_0���������\��	���)p;�. v#�/���kXY�����X���#�0r�����U�������[�{
��n�o���!�b�DN�c3�p���UlO��.p�*Mp��a���*�::\�;W�p��W9}���=� (��m�rO�kP���	���"P������Iul(�@��=�����j�{��3I�����(���r���w�d���y:[���r�F���*A�'���&E����n�vs����Z��me<f�h+���{a����F��e��H�B���lde$�(�6U��-�N%�SQF�*�:�������VZ;W��P[������N��!�rX4�K
]h'��]Q�u�+�e���-n�����\y��r�SU�'W�XLf��jwof�����e�Rgh!�e[�%����H<���h�;��b�������\\vI!��jo�xKJyW�� ;��N�1���������@����5�m�H�� u��
R7H� u	�n��]���� uu�nI5���[��Ej�{�y:�I,�]�l��� s��
2��xv������	��L�dn�.��m�U��������"�������x D��"�A���X����>o���[�@�� x+���aWA�6���j x;���������\n/b����*��
�n�A��"��� r�vDnI5����	Dn�����������7(r�t��t3���E6��W��4����]&��|��'��|�^��fF���xq�����s�����OG���*F�x�� 
���h�F/��z6�'��S����E����B��}��h���xt��1���8�������7oh��b3�;'��#?C��������C�v��\��b�����0�WV��>���5������O�s���*}���G4��J��d"��G��_	�7���Ia�u���03�9�&^��	�������
F8?�����(����P�E��7��:Y���������|��Q|��'��\M.�/�#�����Q,�_���[��o)d�-��������[���������q��6�8�8Gzx1�4���z�B]GM-H������z�:u������o��S����%3�*��Ae8J�]�]1N���?E�f
0��HzE��E{��\t����U��r�S�m�YL9_)g&y#�y���8&�)G��l�����<9��0������]R�C��
�ou������(��D�cwp�����A�j�~o��*z#���r��l+����z 'MO��2��t7_�=���Wn�#���K�NZ�/���/�MZ�����@�8M��>cT�H���WD��$k�fy�������{R��������}�������Y�;Z��Y��W�d��[�a���IrlfFAY�yN���_!�/C,f+F!�����%��
v��r�*��J���M�5�v>0"�+$XNq����\m�V�qW����@�5��J�.4�-c�cw�%�v�k|���]h4_�����+�/e9���/�K��VE�F��eq��e~��IDut��?������T����r��l��xG�^�sQ!�2������g�Wkw�`���s���y���v]R�@�u������xP��>[�g�a���v����,��r��s��3T��d����������dG��<W������{��d0 ���
�j����@S�S���)�o��h��S!u5�Vu��n��F��������E�m5�����s���OYz^P����7�}%x5�������?��2��z]$��c�=���)I�-������#FJ���5h����Y�l�DFT�iR���0������]'.���l��"���c���gC�\j}o���d��,;�p�%3�}�%#]����Yu<��#C,��S��5J��o�m���d}����F������$������5��L�v�U��~�w/h���q��������x�NZ�.O:]��t�2�=�.���:_���
��E)���{+��u����8r�}E��>H����M���-�Y�'+�IJVn�.SY�����u$�'��HFi�7Xg%C��9&�S�<;62}[��L��Q��!fw��,]hv��\��Kj.��������i#��L�	_lir0��*�mUuf�z���$�����?M��]��	s29�:�0��~����&�G����f�R�BIf]�AN� ���9�Ef�|����X�D�U�������H�B�U��hB�-H��6 �<k�M�tDU�����G�&�bz�e���J���T�
��T�����*��vA*���J���O��I��R�E�T2�JA�MR���T2D�!�5����RA�DPB��z������?,7N*7RL���J1kRccY�>�Xz8B�����,wZ��c�2���
���7�^O�Uh�Yu���Z�C,q_&��G2���#�������-y�ax��������
A�qx9Y����{��p���>N��rh_��d�@Z5�q��y��`��i�n\I8[�����tD�����q��F_Nn.�'��E�O�7e�8�C��I��djJ�a�|�����Q�����Ql���p;ZU����S�&�@Y�ry2���V3Y��\'
�u;�^y����r��RG)�;u|B
	�G&T ��
�7��h��������fB�Y{������Pg���9�}'A�+������=��wbN��3�g�Z�9*�����/D�f?��Qa2�XNEx��bL���E����'i)<T�v�����?�D��iQ��ev[T����3�!�i9
/��G
����'���?����(��^D�E��UN/.^���D��[�_���.Z���-%p�0p���;v��Z2N<wT�@��s�� ��qlI��`���X�iqD���<��h���������SN'��aD�� �ic
iF���J��Ho�7��S�����9P&�k���<��������|3��[
|���w�O�[���82�[x������e��60�hx���f�N��^�^�)���7����@nz��L�XV���w�tI<��	?���z���VX�-6���@�tx������o��0�C�����!��x?�0�K~����c����Q�����cZ������=���H05����6��s��������!�NmU�{�u�z{�h����"�yF���(6
�:�Gt�.u����y��1G]/u��y��.�}:���$�}���`�O�[���Z���U#��{��p��b����V����Tr���n5fPw�}Z�6\��Xm�L�u�����h/���A����/�'$D�j(��g��
)��P���v���dP��v��Z&�s�]�T3;�����Um�����g$9�B= '6�fp6lZH?����$M]]�HC|�/��G+��vir�z�0�nmm�X�94u�ud�������c�li�4�K�f�v����u,9U���:��tA�+_e����na�J���:��R��h�v����u,e
�D�#4�(e���6I����4��$�
_�Nd�E� �B�j�Ip Bmj���y���
^��?<I��J��&���C���apa}z ��������=p_���w�	��\B�@�&������A����A\I�=����~�	���f!�N������R'P�"��E���=��	�	/�k!D
0�CA���;��It�
+������������`�!��~�!?�-N/�?��?~T�	'���'�?���.8�#V�0��18?����v�]��X��������c$��1����H���X��S8?V��c{~���1�����=p_���w�	@�:����=0�S�[�z`��$�++�����������f!������h
�-���E��-k4x^��y��q��kR�]1�w���L�VHkeH5���Po�h+��BZ*��
)��e��^,�N0����H�fGmj�#mvd������QW�lc�@�7mv�������U����j���
���^�`�L/�(�ac xC�Q#��B���
�cam��M-+�������
�V��5� �%]s��P��V�Vn�~%����b��,�v�I����:i:v�Q'�����0hA�~�>w"!:���m�D����"�  ���$�,A@<h�P|�wqa��������v�,�x��8� �V����� �xX�@<��]��M�� ����m��>� �C?*��"#����dD�AFd��2"��u	dD�����?.d����\<v�k
�Y�O,� �t�!H� 6	�!�C��D:|������������"����y~�w|��.�@�M����u�A/�j3#�wP��GO��=�/k4[�U�xL���w4[��WX����<A��	`�$�h�L���/��*��p%�}��������34�q���5����_&�7n>����������x4~/�A(BY��������o��_/>�Ou@"W��?Ct�
����g��Ha��
�o/���������p�����8?�����(����P�E��7��:Y��56
������������'W���������6�4��I�����/�/_&)��Ld�PxR����PR������0�x?���F\]�auO����:e�g���
�����/.�����r��!����l3�"Gdx�q������P�C'2��������N���BjE=d,5�������$c)����U	D��J�R����o��.2��iX��?=	�`�������surVN���L�a��a������(���i9O�q9�J�U��K�P���nvh����*��M}��I �3l@������h���b�4	�Y�C(�Z"3|E4kL��l�GJyy)���'U<�
)A+�,?�G�����IY��-L�e�,\�+A2��m�C�2~���Xy�����-.�B�2��b>
�~��dW�x}�������*u:�?d|^?���N��U��L��W���Umfcp����f
��5 I�������_���"
�}������Oe�H�+�H�F��'cI����i��{!��^�s�q��w	f��q��x�vG���n����}��%�)��C�E����AU��lu��n`(�3c�����2H-w�;��.�m��� d<j�R��dG�S�4#[���{�%:�����`pe����T�X?�;N�t�ex*
�
O������l��`��LeIL}���^�s�=e�vA�3���j������N�>G�������^�}i�Z����9%[��#Z%sz�����1�S�-d��)�sz������4�����j����y���:n_v���A���9�u���V�QL���>��G��#���K�K���9���<ORd���[N�'�(�G���U��i�gSU�9�Z�����D��?��<��7��2��9x��\'$�H�������LOV����\�W���LYI)LY�H�O$���T/��J�"���S�<;6R�[U�L��Q��!fw��,]hv������5y��Z���d�z����AYP���2�[RC���'�����n�%R��q�b Z�`)������+O3������OO<J�'7�������}�V~NV�I5�<+���+xrX����a���������pi[}�6�`P=�i}������l�e��D���z�x�QS�bS��n�@�$f;'�
e;�~7~|\%��&����_�S�1���c�g�;-s��"����@����)h�FY��t�����y�+z�%N��W��
<�0]�>�h���n��W���kP|5�Se*un������R�X��S�/f�i��������������������h�	�����!~��<����5N�| �����g���C�^3����nG�\idtj�x5����&�G����$~����J\�`IA~x�|���P]�U����Pu0
����9�ox����eS����+Y���#&���q8Y{?h��c�
[�������E~����m1'�������������[R�g��?�����W��"�L%^��K���C�g��	��G;�j���'������4��n�2%<�[��T�����RU_��K�Q�}E�/+��Z��������yLN��-���f����w�Q}s3���P��l�����7����<�.i1b	.b�*bM����"�R:��q4�5#���$��#���x���>�-u�tr�l?��$p�A���������
]�d�t�]Z��v�];t��l���x�v��e�7�u(�Y�Nq�^��������y�@0�m��*G�o��[��Ng����kV���TN�R@h�jx�	m�H��N�,�
Z�_A~����j�8��N��x���$� �"j	�kmU�_���)�mx�6?/bw�$�Uik�t�L�.5m���U�3��m � ypnW������M�������S���m.��.�z��H����������(M����NzA�oO�������\I2��_�sB�]qPE��@�jX���e���P0���o�G��f�7�i���@�jY��@r�������C����!R�iJ�@���?� ���j����)���|C�<����j9�R���h�@
����y�+P	�JT����?���S	�]�+P	�JT�	H�.In�=�@%*�6�����&:����0(���A)���v@��M@mU���R:�:6m�_A~�����*����]�6�� �W���Y �=C�@g�Ynqz��W���Y_A|��j#��?TuT��_�%\�A��O@rwIr�n��lp���mP	�JT����?������=T��@}��K��w�6���J���wL���y:[���r�F���*A�'����&E�p��1K�VHke\@���������
i�(o+�P�Vx*6�����6�N1��,��]�m*6��54�Ch���aW�
`�j�����-��%��ST�{�6o��U\��� x#�O��EF���*���Z���e����NzF[��G[�Z�����W+7��HcaK��e!�nZ
g����0g��Ap����=%�N�'���#��t��Yq�n@����n@�aH7-�
w
���\-(9G�k��,�l��8� ��V�d��
�6�m��l���Sm��n��}DGRm���Yq��v�g�b�V���r���X�3wz.�g��
oAJ1��u�7�R���-�LoF!���t: ���c]�v����?v�����I<4�kQ�,��v	���Gl�8 ��hcYDm��
��1�oP�����/�f����l>����i����L�h��<��x���x	O��=�/k4[�U�xL�����z����=���'��|S�B�������R��������Ix��h������l:'p�'h��E��o�i����Z��(�����/&7��A����]�����!�er}�����xk^O��z1�G���R1�>��A��x��6������?�J�G�Ct�
����g��Ha��x��W�����<��S�x��B
��f�aF9����W��0�x?���W�p����� ,{�!��(����M?&�do��u��%�	�2B'��C\�"��|��R~k��>���\�����*�hB��w�G������jr��1�pux�����b���M�7�t�|�I!Gd=�m����/�/�vR���Ma��ES�v�^�nF����o;_���6�]��v�����C����Q�2��2�x���e(�^�VaTF{RPw��?����Q�j'���C�:
�
$��^����M�"���@�i�������4���5���oI:�P@��H$
�x!�)���\z��r9����eO���6t�'�|�d�|D��/O�yr���U�����]R����������a��1t*`{������c����MY�H�&�T�1����`Y�+�Yc�5f�<R��Kq��=��qUH	ZeY��>��f��M*s��ha,Xd�2^	�y&o3a���	^lV�(`���[\8�+��e���4����%��
���fe"[�|N���'l��w���U��K�������S���b9��.2���!��c�+�%Z�D��(�U�hfwF���f
���5�`e�k@`~p���D@�&�����}?s���,Is�i�HZ�$/;��=������2��,������j�N�Y<_/O�G�[n����t�=���_����}���9�`�0�Lf"K�2� ��=������L���j��X�JR�����<W�������{���g0 z���
�j����@Sr�X�:,���-�SWx*��
OEJ��:�*�d)��&�d$��!l��3+����C�s*S�O)���Ga���Jh@�����>G�	F����u������sJ�g�G�J����]Y�b�������sa�vt�6 ����Z��I�����Z�*��l���y)~��0k��Nl����fv�m�l�(��RS�J�6��V-4����E3[d�B'#s�"g��Kq��� O��QF�����b��
�\��2>�� ��[d�V�\��������}b�>�	�&��&��&��.��;����r]��?��W.��2=YIOR�r!_!=�2e%�0e�#Y?�lD2X�0/+�0^�I:������U���
��`*��R�1�#�g�B���d&�\DT3=������Y��`49��t���~M�/;+�t<���M���Uu��
���w*#�����^J
w�U��h�C��N4��F:�@'�U4&0~u��#���',��7��#w��0dB`JjN`
�����EZ�wA`�.�����B`�n	��5���&0^�����l_���`yf��Py��o���U�o����9E���??�f��2����Qur����|�=%���vA�&��$��c�dp����8'n��C�������z�e�F�q�p�����_��C�y�[?��H���w���NO�������x���y�H��!������� A�)u�������%Gd����GY�	���y�7���R�@X1�wg,�VA*�h�f����;��V������Db�J�J���].�/����`[P
��jH���y<�dZ%�8X{IC"���4$"p!�d!���p����K4� CL��4i�@zj�=a���d�V�3���
�P`E��VD���w�O�/���L&��9��,n�[���0�Ulq�r��X�����G���b��e���OR�N�?��L���P<�1T�?^.��ofDu���Q�zT������V��qQ�+�FY9N�/���/)QxQ��V����+
�?������N��*��y��n9}6;^X�a�n�O�w~GV�������;�\W�G<Tr:�eg���������,�2G��!������)���\�}�[zr���B�w0H��D*mL��V��@T��8����\���h�K�n@#���.0��Ch��C���N.65����q�m��*l�D���J7
[F�n�a��D�4l�H�� 0u��3��F"k��E0Z����N�d����A���0E��ST[�"�f�)�Z���uI�'�'O��9{�L���v(��P
;d��.��NI#�C�������&;:"`��)�+S�E���6�K��K�<�5Od#?�lw�9�Uik�\�I������2A�����H8�K� Q�}<q���d��M��Cv�&���2B6M��%S�[re���W���=ue���3W���=�[���1�f��i�R����~��	�����P������9:G���Eq�q�$���8�	(N�k'�
��8�e�I/����o�x�)X�Zb��s�&`�,�{��,�G[kx�����K"��s$�c�u��,�Ae�/P��I���j�P�����,v_��K�D��!H��T
�����8<ID�@
�*���B�b����?q���fX�J%`]�hXW��w���������]/q�����f�#��G]4��m�;d���0�=����Y���"�pMP�>p�������J��v�0���
�����4�nu��s[����j=`��)�h�
Sd��.�����O��������v�R	��.v���]�C�������:"`��)�h�
Sd��.���{}O�[sn�za+U���D�"n���pC]4��m��pC����
��z�NDpBPWN{�V�����=�mx��Q����� �d!��	A���@��=`]+��u��a`]m�!�
�_oO��v]�@��+�=`��J�u�0�G�
��=��h�@��B��f���wL����y:[���r�F���*A�'�D^���2�n����0H�
i��)��2�0T��Z!-�m�J�
��&������
��,��~2��Q��I�6i
��������QW�Ic�p�7\-�TTG&U��IU��dRU������e��� ^���U�yc������9D�Z��q,���5�Yh�ok����{6�1�
���M-��nY��ju'�����o�@`�@A`R�8������
�D	��B���e�nZ*,v�QK��:j��]G-(���-(V��8F.����m�99���/��}������>��.@@�q�}��@@�q��>���"�������tG�������2�,����� �����~}a�}k` �7�&�mw��V��7-���@�}��%��Y��[�b�+���Z������*H� ���+ A�WW�$�v;
�����'������������V(��������>��y%�������>�� ������>�����oZY_����������"����y~�w|��.�����B��C�:����D!|�_�t���_��k��V�<
L�Vy�W�yB_f�$�N��
zY����<yw������n� ��6#�M��;�3D�L�o>�|�������_/�h:�^
�������9�����&��^|�����W��(�c�n>������\	����������\�������9aP��g�F������c�HV�&�]'�Y�G��y�!.F?��8��0�������b�K����NG�l��|}�z��+���'���2�C	[�@_7����.��������!��.��
�1��@�*��?�����V�:�V�O�R���I-�h����R��e���b3�G�#Z60�Cn���g�~ZRy�\"��9yLcOg�n�<y����&�<G��`"��i9O�q1Ln�U��K�>\�#CgK��(����:��/����n��h"4��$[�;vR�%����(��|�(���"�B�T@J\���
�R>)�s�|R��Jy�����H)�+�=%(�����9{&�����*��J����
��?dTT?GE�`���ee���CEfZWgT$G�
8�n��� J���
���,��iT�a�4���e�?��*��)�0^�s��y��w	�c�q��x�vG#��n����}�-�U�@gv���9��x<��>[�g�(�
B�af�^Q�2� ����v6�.��r\W 2����b��dG��'C���)��G���<Dx�~`{7$�1�F�JM��b�����Z��3w��m1��F>8�P����!�!��)�w�����"�/��k��Z>�Hz�x,T�
Ps3�:FJ�J����?���8����f#\�����������g�t`w)�����1�|��oU
�h����T�*k0n���#l�.&����%n��\��->�7kV/�2U�,U�����5g��lH����M��]\������3HM�k���R��J>Zt����,�T �
�R�X�Kyb)O(����e��iVU��)y�����*:�D���(eP���vH��]��\�9��&?=������Y��w���V���2[7������|
��c�~��g�E��1LGT���R����b
6J�9;|������cR�Pi���by��_�����@���;�;�^.k�{���g�@}���2P�T�G\�g�i'�����I�d�����p���U�o�%���9����??�f��2�xN�P���t��n>��i4_� ��|�x�M��g�����:2:��������E���$�����u�������~��������;G^p����<<;!���S���xn��#p)��u9����\rx
M>��x�9V��?��x����������p�.���h�f���nG+�5R�6Q�HW�z������]7�
69�^~��E��=��ZEH���$;�Ys���T��+�Dg[_2U&��.���x�c�Q��0�)H���[���_�J���)Qb�R��){�/����?�RV��E�C��HV��
�����%�JQX�U5q'Y)
����R�^JR
��V���9���u>����-���f~h��w�d};��P���7_0�MS��T�!%�`������pQ3��E�^�����'���N�}���165��m`OS��V)af�M�0�%#p�-��q�������	��Y�C����zm�2��0�+�5{|fT��\9���w�H���F ���6�����+��WT�!�9TL-���m�j��I[u�t�0��bi1���D��O��K�����S[���}v9�J[C'��2�[������ec'w�V��#yo��]9[l���m��oC���6G��a�(�.����X���=�o�Z�"%�5Ni��6�h��#���Svx��!��k#�w��A�h��l�����M��i������sW��g�g�j���V���k��Z������)kUc�����yM�������	�.M1e7�v�����v�� p�	�6 �C�@ p�=8���H@�4�����?���c��� �>piP�2��=���mcj��?
HZSL�
@���8������i� U[uPu��N�157�>(�4���i��V��;�s�8��*��!FL��i���1N[	�� p��SvhW���
oW{��i�>$���qLo%�[>Og�x3[.�h��\%h��7k@��#���jR�VHkeT!���hB�h+��BZ*��
)���&���H����z�l!6E���)6E6t8�Jl2�
��4��P��v"d�/�L���n���OV(�+a=�Rm��(�(���tz xBC#�n4��
���xD]�,�Z����h|
g��"U+7��Hc�U��yY�nZ�������
����q[��
�D`�MT6�D��eu`�@�M��x lH������x��.�D`�I&��z`�$6�f��������G�"��<�G��-����i�p�a+�_�)ZWN8E����Sl��~s�o��?���Hq��-h��A�<�6�XxD �#v�#�e.���������_./��~_������v�H���%^mf�<Nr���B��z���}�Or������<�k��'���|4�e��s��9�����	=��t�<���
��/3�.��]���^q��xs����Q8_~G3<�%�t����
��9���y���7Dy���]��E�/�p������	����$P�����>Y����4���VI��-��)�`��+�G\x�!U����7K4M����)Y�cR����N��A�~�����.&s<bL����v�#�^��y�c�uZ��i�N��U�;���-H��I���x���w?^_Ln.�}K&w����^��;�9^�=��!�er}����.�Gv�����n�����������}���mr���'D��o���������'�F�7�R0�9QA$�~�x?���W�p�������H_e8w6B3���E\����g��o����3�X�1Y$�x�����,Y�#��C\��+Z�������7&=�\M.�D;)���|]|��������E
�d]|����u�w�.�tC%V+3��f�L���|3���D���v���D�
�!�}E��;��_~��o�6�n��W������j�|���1����U�w�j~�j���TJv�s����`��+�R�A(���i������P��V�J�^�@��Lse���0�4���x*���&�������(1h��2`�l(��)c$�]�0*���6N���mu�L���o���?�����&@������N����������L��q�� ����Z��c���P�S>d�?z�x ��S"ib�@���yHN�����B�>������9;�$�|�o� �N�@������]>�7��	} y�~�} p���tLH��^�s6f�'��p<��0X��e<O�w��jQ��C���dV�E���������������� f&H��\l����9�Pv>\�F��B7HH�-�X�n2a`D�{_�����C��!Y%��������bg�#�]A=����|cn��A"49����L�MCR�����,#;�����seM�t��5g5�1�q�y����d�l��n�$���������-"4<M_�G��Vq�w������!��<���!}:�/�G��+�x>$/�����}�������t�>��[�1}{���S��|$o1���~��yL���1J�2�t�c�xF���][�����>��W����g�A.���'e�vIY������sn�fTD��-����l��J�<+R���-;[��n���
k-�Y�YuwY�2v�|v���XNr����D�I��Q�b����|��b|�lW��o��0d�����Z8�tM��{O�+��`�]�g<!��>9e�����8���$����[CW@�l���|��w��r~��?����X������+�G���d%��zy�<B���b����/���d�v[l��X�&��8wp%�1$��~�7
)�e�O��F���8wS$�8E��$��h�����kRP����TR���������;K�P�1i��f��:��k���5=��m���B������Q�n���d� Z����;m�h�@�����%����K�K��o�D����K���R�wX�@�D���%
/Q`�D�����5�"�'���|��pK��q3�=����#IS�49^c�<�t��:�'w����f�qV����������sF��*^Q&e,��-t��le��+��L�%4���&��)E�
�9V
�W�z�������a<�R�!:K���������"Q�f'�:X��(4]�h%q:���X���Y�z���B�T"
�� 
�� �	�� �	�� �	r�T|sB���{�D��D�����+$���D��!Q�}$�9"�-}��D1_/6m����x�Z�:!~�������7��a���1��-pqp��1�1�eL�p��jG�61U?���r#LE �x�����A�V���w�y�7�$���]�rrsq=�,T���t�)��E�!�4�}��?�M0�m�_�(
	D
�Ki�>k�.B���P>w�!Z��z��������h3K��v��J�Z"S�Z�<>��G���X��Z�����xK�������E'��Ak���^�0h��T�������U�Z�P�����~�a��(�>)i��6_S�`b���a������|_���2���=�z}�d���������rD�qjF�x}�N�b��F�H������7������l�bK��'��]�3��F�R��y�1?43,=�2Y��������-S�e�f��<+����,�sc~�����|�ppw���s����`��s�@���0N"�p�F��U����s�������K{�����VP'�������[�K|K<�o���������k���U�R�E��|�>*�?L�3��2��~o�;^lD�
��L<�4�#��������y��1
�:�?�y��O��;�t��BG'F�VmD�/![[�5���6�W@����/����/�d
If���sur���S-��A��nE��3N�����]�0������6J7+�����e������>���t.7���^�f2V�-p+�Lv;��8&vk�s1��<��fh$�5��|3�-o��O����L����@`��u�.g��4[���a���V�<[_X�����w��O�,����<:�|�48��q�zd���;O��J�~x���
O���T��O�w��-���h���	R��;O��J�~����:����:
�yb�T2�#_8�X�;���l�tOo,JrO����LC@O8��t��
I�X�e ����)����LSAO�?��i�+$��mH=��z@�Q�RO]����z�u���z�oJ���1`�{�2�������;OS{V�f��?����7�:��f=rc������)��L��zh]<-<�
�)p���LS�U�zU�KZ�K�����p��.~�#��Q��]�������O?UM�+��u��"��K!� !(��k�:�&{�6���|]�Q��f�`��ns�S����_�|�K��\~BB@S����j���r���.v��~3vxli��lR��AZ]\|(�o������?�:���]�D��R�a��)b�(G�L�
9�(�2:�JfY(���Uz�9�4<y��Rl�|&B�X4�,�X��}=�=g������������Yb�������F�x�dE��\� ��t	���r	���%N����)#�EC�������%}F�
��J� ��/�; ]���9���W(�0�=<��L����P�H�3!��x��J7�03 �9
���~�g��������s ��������3�3m�@<;���Y�g�hV��ee1�HV��cE1�(V��ae1�V��_e1��U��]e1��U��[e1��U��YE1�����x����@;t-_��@��i����-w�"�8��u�����	`��C�DS�|p��G��-�1K��|���i*�����:�o-���6����d���|����[/=*��O�k��W�/���]�M�������K�
Hnu����W���vY� �^�mT�i�,�'��^Jn��bVwn��4���}�X�Qk������Tr�d���i����D��[���Z+��������^�^v����f!��\�Nl����������/��l5%�M[����%Y���$#�iKr���$+��K����$'���dD������/�
j��[������W���S��H������s��j!<T��6�{e�[bo��p�N�e���]��T�IM7S�'s�����z�_�����&+���8b��"N���]�dW0��Y�bPW�t��51�e}_�kzj~v�7�[��d��&�]<5���M�{g�KE�����v����W����.oy;��v������t��l���b;���z;�8WM�s�m�����v�T�5������v��L�e;�h�v���5y���M�jJ����D��`��0�"�5���lU1��X`V,4+�;1+vjV�����������b5�[������[���+V3oy��y�yX���������Y`��-yO�!K�sI��Xl���r�bF��R���=�~�T�c���FG#���a����W-�o��	Z�IcxO���`a<�	�@kn4�.��m|LVa���*k�h�7qh�lf����p�t�,��*
�~'�q�d�d����z����m�"������C��X�2j�T��u�y�����"O!��
��d����q$�J��JH.D"8-�T**x�D 2�!�C���2=�����#��{���!e�Mdn�@Q.�^!%�"7[�����96F�o��x������b�Cb�l�^�
y��}��pw��}c�l�^�Ky�g}��p���%�������9�v|�~����^��X.��������z�\�������&��U�9��>���w�_��J?%�=.�f��	���=;�~}������Q@��O�������OP��������;!������j���o�MD�E��D���`\i=��Z4I���Wf�9X�����uZ�W�B���T���.P�S�_��@5>�U}?������S��{+D�3�wU��T��������\����wr*3XJ
V]'�HvJ�%�e�c	�Hj�<��B%S�I]��2s��`��AY0��D%���'��g\�20+�W�����d9�),:&h
��>�)|fS����^�#��,c�������G\��x����@h�@i�>�4�H;�������!ORi����L���$����j���w_�� 4��'~T'���5����k�
m�x����x�� ^6j����e����^M�Q�H����G��|@Z����m��t�������\�X.�<��4����-M�C����j�!�!8�$8Z!4J}��"����{9�N�!�����������n#'���Ea��5��0O�"���$;�d%����~�i4>a��FE�&�i4����[�=�3�=A��c��C�U^[�_3������<��<�<�t<�i<�^�h�Wu������������5��S�`��^m�0V�N�6	��Fl����*�Up;�����ON9�0�8��1C"&C
a�!B�x�SN���H��m��Tk��S�&g�n�k!��kc>��wdIvA1t���Qr#��[��|!��*�#o��H���[�b���k!���8�����������<�"������"?���5�jl����Na����){^v��%�R�Q�v� �I�.,CO0a��	S�)P�\Q���M	g�N�{Fa��T������
)gy�������j	������T�,f6�MT
�h�N��6�����T�����2����l�uu;�z����2
���4
�H��"!�G��t-U��VXa%/�d�����V��J�X�+Yb%O�d��\��-V��r��8ic���*yHV��.������f������R�|9����\}�o��o�<-�:�|��wYdB���y�7� �N���W���h@Up���B#y�^��-�U����-8�v9�����$�����������T0��4���41.�*�4���	�j���1���A�����@��w�I��x����^�d���������g���I�B�����a_�������������Kl�3�I���en`�T�����K��������l���I��� u�L�+�����?A���SUA�#�� �)D�"���F����#!T���H�� �9�:H��H��#!����H�	!�9E���=B5��������PMo%��jz+)T�[	S���J�
���.�E�]�����"}b"zE@�:��J������1��,I�j�f��r_w]Y�BI�#4����yF�"�����n�Ji��T\+��%d���E,�Tz���?�������8���G��n�FY��Ia�n�����~�������>o�"�|\��N7�h��	h���.|p�������z1�vg�jlA���=nl�����_;�JnK<"�����Q�PIjd�%�3-�=���*s�+*�����u�����B�KQ�O������}�\n��h_��Ee���ws9j���������I����|�9b'n�Bk�+��p�n|�{������W��}�����B�l-�v7�@�mP�6<�%q�9���$���y�=����~����f�x_3
������f`�S��M��/	���w,�V�����1G!�qq����A�4�������MMLji��g��X/��/M/���������F��/u%��/kJ2��%�{/kJ2!��%�[/�%�K/kJ*?%�+/�%�/�%�/�%��.kJ2�����L_���R[2r����u��1�pe��PL�M���S&���rP�;��
��C+��G'�
{M�W�&����E �J_}�����	BQv{Mz�s��T�ng���eL��|P����S�S���;���-�U3%����v����?q��v��.U�T��B��n�s�����N��vb���xc9�F�N�!�����3ulQN�O��E�����6�(+�������fc��[�o63�����</����
w����~l�@�$M��H/����
�������S�|Q��U��X/�8���r�x�-�
�����<[U�7+���Ef�N����;3+v�/��;��X��J�nQ����7SW�f����f��y���p#�<���u�t�n����Fh�n�Fp#t��nn�N?����t��7����-�
��{4'����xw��IIg�N����*AR�G��xX-��]Fa�@r���9j�5Z7��C���SB�G���f�;�aI7�ru��O�E
�|�1���{��x�G�~NV�4h�����NVk��<�����_pn�K�*���~�W�|��5�������qu1`Vb�;~�1�A��gv��hK{JCR�!HGM}�H5j���5o�`n��g�%��b�����	po2O����x(��cnC�{R����p=�� ����a��2�PW/��������dN� *3i���2�4��'eFzV���(���2��9KgK��J]%�FS����Q�
D�
����P�R����5�Zm
h*�T��@S��M�M�
sIe$���U��DV �@^�!y������{�}�D�=��x��&��q�
�� �[&��#����,��.Y����E�j(�
���P�����!��W���y����'H'�u
,4����puuq-DcW,���3�|Or�NyLZ��
��e��
Q^!�+d��~V2;��e��y�3(���9q�;{b��Qc�3N�M5\v����}alj�����������i}��j�b4*�m������?��Kg�|��I�	w�k�e��*�Wuf���6]S���4�Mw1]��edpKc#lWm0���!��67^v�1�s���'�IL��}�Wo�i���Y8�-�?���\o�1��������8�Li���*��3�)��8���j:�[�����b�����M����B�o:<�Z�<�;48����5!|���O&_���~4k����e~�m���-�,�K
\vyn�g�_j���s�?��RS�]�[�Y���������2���e�������/�2����/5�}���e>�
���������]��t|U;�������t}u	t}e@���u}n���T�9!9P����h�|mvw�j>G���~�
���4|}@��k��[��9��=S����4{oJ��m�6�p9i�����5]���������T��.�a�Zf��l���sVgtOo�xS�Z�F�]��Qij]3Yv��$M��j���������&��[�nE�I��p
lC[��3��
�B�s6Um��i	{���,����������O���E��4���6�����
��Ze_0iC-i���0i+��[-�Mi�&m�I��O��I�J��pP|;�Pb�)%f��y��]��J������T5&S���.�S���6nA��v�A�)�
�I�@��M}B�[�`�e�IP^
5@yi�-P^jS���V���{�k�h-���R�wmoln���eO���z�I��9V��&\�����g+=jU���9�jg�d��`i����l�K�7��e�eK(��+�]���mW��2"m����O��0�	�0����v��m��*O��2s�����i�6P���'t
��������4��������"]�����D���������L�t�?���)�*F/�j3#���/O��'4[���!~�o�W��������z=��r��B,J�������-�*�����;�������3wh��b3��?��C����S��_&�7n>��6�>����K@A�,��=����5������O����!�����O@�o@}3�sBN��Y
������\��#��a�����~�x`yK; ��7B���W3���1Y$+��o��'��x��1�.���j���M����|��I�����o��
[��|�}���
���M��f�%@	��.�O~������:	��K7ab�N��j�<�d���7C��N�n����H?9`�O��G�bkq'�� �|A����D��cJ���������'�Q\�P���T+%e����,c�q~��z��3�X��������d'X��><�>���{ys�R7ch���A��8V�����y����?����4:M@��xQ�����
FZiH)��^�a�����X�b�Z6Q$�r�^^I��n�I�:*��n���]2���xxH(M��s�kJ����������������7f&H��\�	�u���#�����2�<�/O�yr���U�����]R�GD��D�>\m�m�D���u����Qg�M+�W�_����<��j�^u���]7����H�?N<M_�G��V����Oj�%�����!��<���!}:�/�G�cX��
�K����X�>��[�1}{���S��|$o1���~��yL{��1J�2�t<c�xF���.��o��
<������f���+��� Zb���DN��G^~��\x
��d�.:�h%��tY�@=bWc��8�|-�[��J�/]�!��F������|����-V����t��&#~�1;��G����o�����,����mm�^��k���2��,������j�Gz?}�t��
�Q��l�Y	jHB/O�G��r[�}��������x��������9t?��)O�}�&���R�m��|��Q4�����CR$��b�?�I�m�LS��3�@0Cm��dj;�����iM�Z(r ����-��<��/�_fmI;@�-�W�q��4��v	L;e�Ui�����R;ek�z$�:�\�y`$#=�aMF.2B�=�����)��^'�3+�N����������a�|.����dE|�f��
�*^Q&������k:[����������zj���3���k����L�����&����1��=q��Z���9b��\5���A���9e��H}�0n=��c���a��Z`�.Z`��Y`�Y`���'����H�K����v�4��D��#���'��H���b���Xnv�{��ME�f��=�9!.E5�z���1��C�k���'���O �����������8!�:���fY?��������bfh}���EhO��Av`��P�y
��}C��/��I��F��sS��B�+��F�$^on��\�+s9����\8zZ�SZ������(;�z��	�������t�N���q�5%	o�C�^3�����	��S5 ���,Yh��U�+���d�6�V���\^L>��,:q�X�&��c�'�(���xT�sM5c�
?Y�'�FlbR2�z�������xU���+Ie]��kk�H�3#����
#wj�!E�O�n�D��vs
g��m5�8��0?43,=2�<��3?4���3����O����g��_&�/^�����[�_��]��&7����f�?��5r\�� b������������.�TYJ�)��z����n��
������9)>��x�Nx���L�/�� �7�3A�����l�H�B�����2���??�/=����[4�3x�� �����b�(?����G�[z��@�g�sK�_�?��?YC���o��?��?���P3�t��,cQ����tg�n�c�����#��,���r�_������ZG���vb�;����d�3o��Zk3����(���Z��r�J��������������������|"�������$������aj��O�w��-�*��\8����'iT,�����R����*�L�0� ���l)w�G�
p��3<-���i������*�i���)���'fK�l�xc)�P�{:�e�e�:Y<9� ��I�+$�c�OO��S�+$����9�ps��ps��7�ps{��m��z\L���(eP!��d���(
�|pd����1U��:4,QV����M������-�~�K���IZO�t&��u�&1W,�T
���E�i�Y��^�X!z�8V��0�C����!�)�	o�rSi����&������P1u
�K�"����5�no��z�.�����^�����.�Q���/5�<������5����u,������<"�7M��1�bN�gE����r����r.��	�gN��Es.���^��Sy�s��;�����j��)=�V4����hT�j(�;���Y9�A���'g����("
}F4	�H�5��=�; ���gX�0��t^�3��A4fXz�hT0����3�;����J�s�VH{���1�L[���c[2�Y�ms���V�9`l�����M��ej�bZ��,�ef�bzF�,�eb�bZ�(�g^�bZ��,�eZ�bz��,�eV�bZF�(�gR�b=dP�p�
$��k��|*M��V��o��6��u�Q�C����	`�W^������]Ls��2o����'B��r�>���5c>�uC04��s�!%�=�
:Y���T'��w���[
�����~��^U[�sW���.-ifK_������<�'����'��8�ifs_�	�����<�^�B;������xy�E��jKr���$����d�]mI����dy�����-�q���,��-���5%6��$���K�L���Vx\uH�4A��)B��q��S�����2$1�B&8���m���R�f����Q8�F'n|s#�y����O�XI�i�?����#z�����T>a�z�<V�����St��DN���	��i���k{#�Sq�C�?�G��It~GM%����L]zN�����a����<��M-?_����R�X�a
�mf0z%_���������1�M0h�T�������U�|�b�Y���XdV�����Y�3�b��b>�����Lo���+V3o���X����j�-/&�� ���f����������!�S�I�K����N-�P���K�JF�T�S��R�F�Q
q�6�=Y���m7A�
u�Zc�~��Pvb
���6�+������&:���f�����g�B�����X�"��$�D_\���,�[�o��`c���"�)
��G��'XL��NK��2�����_���ry!����<b.~�����P�Q����r�{����Nl-Ft*w(TN�o��H�w����.��"::i��Rb��F}'���
�^a}P0��FtK�?��dM�c/��2�z���U�^���5�_��u�?���_h�z@��O+?����W+��O�:A�K�Y"�k"���DO����y�0�C��abr<������x��w*�+g��c��$�X}����0�i F>B#�Qb
,�z������V��.1���p���k<�����	4����&��2"K��S�N�O����T�U��Z��H�.'�c��_5�j�%O��jI��sR���Lg-)X51)�?�O�t�2+�W�q��7���B��'6�Om
��N��3��(`�,#��������G��O������N81�����{s8�����<I9����2������x;�|�'���O����Q��$������z��|e���|g@<_���6���
&[(R��*���oH�5����-@Q[�N�t��������r�����N@m���Z�6_4����������z�l�����9z���ez$R���8��y���i����[L�,@L�
g9l<��S���hs���3������=����U�N4��#�������H�9~$�?�y�K�r�vG2_�#�����?�H��}$��>�yW7>mX�6�C�m�V|�:8"w
�M��%6�E(�Q�D���z��*��D�0&�����Qj����!5�h�����6w��D�#r��$�3�r��nUX��!�G�yS��/?�
B�y%�8����������j�������8z���������a��#�l]�`rL�`�����E�,5����;�Or
��)����J�^uw=����a/����������	��;�kaw�R�n��l%�`��i���"�3�ap�� ��?Jg��G��Q�>J�G��(�%��d����R�@J&H�)� %$g�*�L�1F����d�,�={t?[of��
�x��J������������C��xj����Ky�C��I����<���(�,p��$V���
H���:k�
	��z����Y���`���sj��)����b��f�f�i����+���i	}4�p�����A��1��5�5���z�����5C�-�'7���I3���Iy�K����6�Z+[qZ\m�L�6��x���P��9���Tvb��)A���W���'H�~�I$���?�Y�Q��	��?G�i&���p�I�s$�{��	���?G�	)���plJ5o��T�y��R�[���j�*G�T�V9������-����$W�(�>l��>1A����~��E%��N�8HCm}C�$S�}3�U�P+\(�C6e��`������w�h����^�Z�G��i��U{n�'�%).zN�p4N�����4�r��!5�m�N�K��wK&5��.A�y?n�[���'j�-P�Q�Tj 52�������x�w�?�9m4�����?��5�a�.E�V�RT���7*��h_*g��-��2kQYj��p��:$U���C����0p��2��C-�1�u�(��Cp��:��mp��$�(p��!�u��@=�Y�u�tn�[���!��:T)�U��r��� OpG�� }��-��A>�pG����
o����3�[>Og�xC�xI�l�HV(^��99W�|�;���t��yr���h*��B��j�\� 
3�M�6��u��h��d������x��wT�����&Y��q�����WK��)tk�������������/W������y��	c$� �����O5��/�*�����DJ����o
�� �����@��
z&���(�-C�XQ^���������3�y1o�`d����&�@��k�("���S�k_�J���p�`��fF[E��P�0%ta=�������d
� *3���2�4%}'eF��eF���q��)���%i%�Z��[#_Bs@��~�jad�������3��������F��������������������R/FmY�� �%J���d@���;���'�/I�$�1�c@��Z:���;ki�+c)z��B����~^�x����W��{�5
��iQ�A:G���A����.�����k!���"�)=��W�$�����
���Vh�0�([�h��V��
a^!s����*H�����m�[8�?� �6P�`�!YF�O]B�W���M�������v&O4�����q�h� 4������Y49�a�3��Y=�Z��o�9�
�
��7�|h��m%i���g���nt�v��������cX�sR�?��������I���K]������`�<��}P�Y��2Bv�A�g�_�
����e~)Kd�����|��yP�Y��2Gv�A�g�_�!�}��e~)�d�D������S1��	��M[�_���:^1��:4 H$����DR[�&�H@"1o$�V{��$�fb�iCz]SH$zi��DR'��C"�_��CH�=���"j��*�z��"����`s�j5P`+
��vF���9g�u{��"��g������x>�����n�o����}m�6���e��n8scl�g��X1��~���l:$9�1vs�'�1\������f2�V�����o[�tmH���������4���g�.a���N�Yc1��p� �����j�z�����j�%����5��NX��v�()�*��IG-�E���nR���~s��[�Y��e����m����ubZ�-����;��u����l-=H*���5��tvW%�m^��C������g�-V�e!�������Fw]����@�i�����#g���{����v3|��KS]����T�At'^R����hXve
�jZC|N�/�vi�k�:������� ��R^����/�f��Kz?��B�m"��bNZ����]&������	�Ob,���x���x#i��t������a���|������5Y#
���h�F/��z6�'$B�B�q����-�*���M��c���4@�4�����1<}����
���c�f���r�����������Y4��l��$����]�=��!�er}������u�����n�������S��?^��M.������
������O�o�|3h!�����)sF�Sh�>]\��"7i���Fh���7>�5�[2Y��y#����q�q~q�c�HV�i�]c�%Y�#r��W�8�VCWo���G��4<��\����n����/�/[4�u�|��I��,�/[4�t�~�����.�.������je�e��M����H?.�����H?09D�O,�~dr�&����-%4�^lyI}����K���T���.)���w�j9{���`�)����E�e��kh��jV�~h��i����B�,����4��c�x�w1R��1T��k�}V��s9�]|MO���B|t�A����:V���S`�$��i
� �\X��`VF�
�JiT-��rg��OM.�.��ZU��j�S-e�EKN�<g�<�����x���r���<Y�%�`	�����H_��?~�\^|������!�9����������P���"J������Y����%e���2�<�/O�yr���U�����]RGBFD��3��L�R��F1��^'F������D�������d�b�:O.~���n��v�3��N:������-�����d5���|���E�����k����*��N@C��t4B5$��G��<�O'�e�H^bzE���%���=;��}[>���n��0}�<�o�|�xJ����-�_��O�2��x��>F�[���vL����Q�k���e�X\�r�3� e���2����W�D5%��������\�L�,f;O.N6���egK�]����!����Hl�sV�]V���3��3�=�������$RN����@s���i�S��K��j�-#�l
�xF@-��x��V��'���OF����3�Wm��2����|rUrtHz���+�G��HP
�����i9'��}�Wk=������W��rdk?�JP�
���y��-����li_(�EI�W.q2,x�E��'RC"�I����x���Wl#o����c��I96usjNC4]���5)���6��������p�f�p���4qx�h�A�E�5l|���W�6�x�W/dnFY�Yg��m�h9�R������-k�K�K��^��.�/Y"����K��.Q*J��a�����.Q Y����K�.��`-2�{��F�2����)�f2>&{��%G���ir��:y��d_�
XD�����%��������f�Y�(�2�����6�-�z�k�)-�7!�M)�V����R�e�|~"�-������km49�j$�:�NVu���
P*h��J�t0��
��3�%�;�����9d�1B� �	"� �	� �I�7'$�L�Ht�H�o�z�B�~�H����G��#��~ H���b�_J�oZ�V'd��7�����6C<L����<���d��q��1�1�eL�p��jG�61U?���r#LE �U����A�V���w�y�7��|�r����\\O.U*���)�L����t�E��
D�S�F�%��hAi8=�R������?�!Z��z��������h3K��v��J�Z"S�Z�<>��G���X��Z�����PL�������E'��A����=���8��9���o�����lA�(����z��|M
��u���A����.�[��u�[d�^_-������u88�e��5^_�S��4�3�A z"�a�M���(�)���R���*cW����Q��ke�i��K��LG�y�1?4���d�p�������n����������S1�)J�N�Q'N��u�,[��yf���(�b�"�e��d��SW��P�	J��Q��{��{��p7�x��������q7Y��zX.3��m�^>�Wk8k��Cr���C9��Q?I�O�B�������KG\%��]����d�
�zU����]������\������3�C����"(���	����l;�����.��������������	m7���$�&)J�	N'��g��f���"!u��
��[Q=�����g�HA���o��k)v�jh�p#��	����.�jq�|6�������3h�U�G����|������w����5���G�����\r;���sk�s���v�24"��L�D?��'�%���d�����|�T!�	�*�z�wm�>��������>��h�&����X���Krrz>=�&�,=�����'��$V���S297eL�����S��$��$V���W2�2��2IQ��I�&k9�d�2f�:���f��XM�r��41e�>d�FOOb5Y�I��k3�hRgc]�X�����g�N����
M�X����3d������+����89����qg��(p=��	\�����T]���r\O�z	������g������oD�F��������$�x����2]81�Fo�7����
�1\�E�f�\r-������ua��r:2�����x�|D:{�K���m[��y�������3p�Vwk������k��WcM�=�p;�U���C�T-��j��+�2 ��.s) }g��������,�����e�������lk��������h������������p�Q��5��Oh����\��|sfV��4���V���Xb$\���X}����q�����&S�����5��a�O�|j	�2!�M������B&��������m���
J9�V���3=�9�v�
$:�$������T���c�h���S�����4������!HO�����!Hg���t�q�
�$�8�	��v&	��Nh]�C0I8�oB+���g-=s�:-�ah�4;����=�r��D{�\��!����Yb&�g��]QZ��3�)��=K{f�\`��e`�H�w�,	���r�={V��3���5+��l��d����`vV��Y�0fe�J0;�������]��r�`vv��YY-fe�J0;�E�z�^�cHg���+|5����[^����w����V]���S;��������^����
4�v0�<��<��)�P���zu>�Zd=j���o~���|K�|K��^jT�W?��=R^�g���X8{R�b���u����&pnUW�9�X>1�vyF�23/�>KM�i/w �e���b��Y���g�-��������V�ss^�S�-���U��yF��,�?Km,j%?w0�e}�������\7�M��ml�
R��* 9����l��f�86;$��U@r��R`���<�f���5+���U@r�Z$���!yF�
�>��py{d���Q����C^��%��G�x��9C<��t�����-��#G�h2��N��~-p����4������Q����|�W���9���=�q�=Z|������2�Y�U��$e�����i�\����.���=V3��{�f�r��uQ�=u^��v���s���,l{}�d:'�tN���#����s�M�D�����Lg|�������s�}�d:��tN���Z���P��tN������"py���"1��tv���n:7���d���D4%��l8�
����Z�e4�����n`7������
��
����3�
��{�u^V�o�����7
V�o��� �>>F����Y���+1P�O����3���/6$o����-(af�������O�M72��6���t�=��HT-]_/&�#��qx�c������.�jl&k8��(�}|5�6����D;p��P�� �N�c�����)��S��s�����M�����CM�v1x�k�����������,O����M�00�#�1�)�H3�s��\�<D:ct*fb�#P	�	WCL�'
?�j��u�(����������yDVo��$�K.�g���8�;�1���1�_N�L|u�wjT�
��c"�|�^K
�W��SC������Q�C��������R��Md������2�s2��8�.+��K]�W,o���M^����6(����o�D�9R|�>��~���
������h��`QV�,�9@7vr�x����,�"
��iAG}��4�}�:H����G�E �EhTv����D$�(��i��&RI��K��R���V���rkT�4�M�N�������it���p�H��	�/��
�
~�d����!Py�L�����
�I�C�M�����wx�r2�W�:�;��,5���D2��B�T�Xw|��u�=���G����������8T(+:'�#Q
 ��n�6t���2pNd`5
�8J�9O|�U������������(��$,r���2�W� ?��"��
4�7���+p������'��
|y���[�]�Z�CTq��
����KT_/��h����$�*.�4a"�%�����M����O��D��AF"�� #��j��"��d� �qyf��dd���������met�VF��2:k+�sSF�|���H>�]2b�4�g�dT�Wsuq��i���L�V����N��j��3�D�j�%�TMe������u�8)�xv��s�������
N�jv����N�Q�\�J�W�FgsS7|7��F�F��$v�	�	j��]��,eZ[�_s�{�S���(r�3�:����xQ#�7���L:#����p�3�L:c�������y�IN���a<��X�6q���o�=;�����*)��xG�S��S5"6�o�L������U�U4���S�0{-8� [����xr&���&�F��|hz�6�]=L��$�>v=�'G�����'�J��Q���Q<W�O��A��������t�Cai�nH��URX�t��%MdH��_���I�Ys���II���[���>�a���N�D�j�2"��	$���K%�P�������)(y
4W��}���\8��y{Nn�T��������4Eq���jGv�}��O��Wvy���MD
Vh�v��4���O��OM�o�C.u���5_w]�v��h�Lw�:�>�=-H�z+E��z3����H%R�H�a#5l$����� 6R�F��H�b#Ul$��t��0�'-3�V�e��o�M���vr�fQl��6�l$�\�����w�/�={1kI������3!|H��2��-F�p2|�	"���?�~�W�� �a��)�P�]D��_��"n����]s���~������O��T�pk�O�����2���E��B���Y��o��o	���Nsq?�/Q2$j������0�a��i����K|=Z�
��m��r)��m��;D�i<��2S|Y�Wt=�
�?���f[����p��0;�k�e�U��t��2�G��K�~�uE��P=u^�7����d=J�[���^�����A{(������A{(������A{(>!����(�����S�J.%L�+��0u��|����G
S�Jn*L�+��0to�Z��%�u/����
u������j��?�CBDMJ��<�K���peD��� ����%4A��J�N�L�����B4���^d��BPmT�?\�lc�����1��1S;8[��)Iu=R��������j(��"�O[������B��<V??���N�1V�<?���g�Wq���Jsc�W���v��"�
�0��Oy�#2
1���Z���<�V/��������_���q��P����?]�I#(�D(�7�K��r+�5��n��Ra�AER�w=:N��!t�@���#����8����
��������kC�������Cl,�j��;G���S�64�5~��cON����s�Z������.~��*�WtC�q���ls8�R����FM_��[8��d�ZQ�o>�(V�L!�v	���������������z��)W��GFz�E�t
~�����GR�K����sn���^V@r.���B�K;$���`��@
!/��|�K;$��
)������* �32;$��
9��'�������9�~�P\N�d����+��d(���R�`���}�uh��I��0�5N��p�'�5�ml�N��+�n����W��������YGd���:���3NsM�3�������b_5;���s*Og]�7��U��Yv]j��z���v;����p��v��ZD�����0���\��DN^���AV^��A�pdc��M��y�*OU��9s74Q#��9#8���U�>T��6���x�T
�f��W-��V�4S�Z�����i�^�8@y�^�C�Ay�>_���&��7��
l�6u;u���������qU�Ut�6v�����5SV�o�����o�:D��_!"t�Yz�"B���!"t�"BG!"4�a7��R�����l��`�b��7Q�����z�fW�i���|���X��H�q�Y�����2��`�:�Ji��rF�����AE�X��JX�DZon���_�����s&�Z#�@�~����O��vZ�����|SD?/3�Ok%%�f�&�F��~�6�r�/-f������mT��A��8���f�P3������eY�44�5�tT|��	�"�J�-�g�Kr�����i�Zr�����cqp��O���j(����n�>{���0����cnB�$GL�����w�x��������I:e	pkM��0C��)K@�I��h������}�m��*��=U).l�aS
�j�T��6��m����G���B%��m�riaO
{j�S��Z/����nO�
s?�-���[�V52�����5l��9���'�������?�M�=��y�P�as
�k�\��� ����xsmq?�]��;��Vq�aG
;j�Q�s;�O>aGmgG��~4��?d��hx�F�2����� h]�B#Q����7o^�S���#F����
��A���C=�i?%L�)�`B?H�%L����~0�u�)x692{l�e��y8COTX��lO|�q*�f�T�H���������������5H�*����A�\/��z���_{X��`��E�\�w���k��u��e��k���[A����d��S���N6��:Dil���'^a!b<����pV��X�~`L�*��'������_��~��9��A�Nx��1 ����Z]��D��nB��80�q�O���80�q����5V�J��������������!L!����c�B����Vh8����%!����'��z~�)�5@��d�L�F����,�I�H�\���3�	��~�t&�#�s�O���w$l.����D��e?Y:������'Kg�?���d��4p�B>����-E02>������N��I��U]A��*d}O[�W��R�B�ZH.�����h/�������|5�_���z�/H�����/H�����%|5�����W�${OJ�'L�&�����{Q�0]F��L����T����2���\lZ_�W[���~�G���Ui$��?����%��J��'�U�^e��A���[�X��	�]��e_�"����P
lCZXg6�6���l*�d�i	{�#�d�jI�������O|�	��������!x��I� �U�6������I[���j�$7�W�0iI*���Ik�V��4���'1���s����f���3!�>4���"�1mU
bL��'�qb����$��A��R� ��^}B�;�`>e�� �T��K�j�������2�^�Z���OJj����'�0O��eO��A��r�(��M�F��Q/�t�j�*��[;k�2m�MD�����/�+�_F�>[�^��B�Z���0wm�-W�eD�@��>����)_A	3:t���-t*���
3wm.W�et�@0��>����D?A�y(2�����9������h��QR�r%-QS������`����.��m��.��G��>�n���&_m����F���C^D�����v�\����Kt��f�-�"Z�Fy`�0�*�g������q4�E����uQ,��9BNm3���A��5�Ey|^�z��<%����w�^^\��� f�b��5�nsP���.�E?_��|u�
,W��^�z9�}������8��������//�GP�����#xK��I��Iq6S6�&��_�p���K���7�_���]����]��E�G0��Y_���8E��	�7~~�q��|�o�qU�m"/��Q4�� ��y{��<���^��x���� �;i>�n ��Ht��f��93e�Db����HGi ����b��,��`K~~��H?#s�
����\�J�YU��\�R@V��^�a� ��N���)�SRc��0��XE�aJl���`�4k�l�`,V$����&�����yj���\@>�h4��O���O��3=�>O�U��E.����CC�#/��#Smdv-U6`?��!i�s��xR��u<�y�����%���=�����132��&���� s#f��;�3>%���z��-c���y1�����*��@8
���������{YvF@<���t3��FB� &B�����JYF��\EX,����������E���yBA6,�����������w����<�_(��D��1d�����zJ�B�Op�)J��K�����|�=��gg�1�o�#|~����B)��s�����'�-{DOR�8Eo�G����)/�9�:�ldc2�Z�8�?�Jd��;�l����"`.��@y�<�������k��I|��zI���a�l	s�����M�l�_N������H��~���5�$�a!������V�MA%@x#�����7���2���}����MaG���z��
����(C�T��w�F�Wl����Nh@�jt]��
�CE�<� 4�	�������$�,6��Q�ar�/�#|ui~$!�_V�>�%6/q>��'h A�'s
~�����2nS(�0��I�42�p|�Lp������������%�PyM1u�:����ef�����hV��XZln�����a��H�T���!{��q;�$�;y���O��>
'�/����<.�e>��M����n���v��.��Q������|�0�(�6�����HE�
�S��(E5����T+=Lq��	��/0?R�N�(�{R�����"����������C�5����Y?|�#����)��z0���k[?6����(��c�{��3A�����]`(d�B�+R
��V�k�Qt-L�8t`0ew�2iS&�c�$`J��VS��0e\S�h��f���EaBU�S�h��t������G	gV8�
�eU�UF_u�y3��A�UQq������~�p�K)��F��i��GtY��
������'��Fv���SE�x �.&�A�5��9�`'���G*[����bD*E���yVl������������������Gc� z��Z��b�^�&��e��8B���|�-<�������(����85�����5;�/���6�2�XOd*uL��0��(�~y��r�'�N�V�����s��n�^��J���8{@v�}��lRr��2wX�-����q#2;9p��;�_F���-GC����x:q��0��\=�-�rAPpc�J'^/�
�hpp��p$&����0&���<�S���0eD����p7�x�4A�,�hsp7Y;m��\"C��z�p_���=���0�^M�gtl	M<���~������t��MU]G?~�J.
:��A�����^C"K��M�e�Oo���+)�J9�&l��P���u���.��Q��Nx���4BJM�a��������i��Jx�&�p,04[@�rE����n�������8��f�>@����O�TN~�'�CNQ�L&�����Fx�r�3( l��E���SM�����g�HA���o�h���M�-[Zf��v�2�%���R<Oxz����\���}m)!���{�J�/\r��q�[�+ST�w}�$�����B�8����\TPj+�TX$3e3~$��0������TN�]��'_�d;��;�Lf'��xAm��d���8O���,9�|T2�$j��K&g��O���S��e��j���K&�����!,�%'�����ib�x|&Te<=��d-��b��x�I}�I����46d��
�	8Ku�c�
����t��'�d-�[�>��A���@�z4���
�h�G�=:�h?���h�,xa'h�_�C��N��������aL;Ft����/���ZfCG�d�A��\�uw�Y�I��(V#�Z�p(��#�S�1����$��?��Am[��I�8���v�����#�w6]im���0~�#��:�W(�4������r��aNL�������h�*	��`kBjH���bC���.�$%!�	��(�/�5�����S-d���h6e���SF^�����������m�9%�04GLJ���d���Hh��2�)��#����#�h�u��1}	 SZ�������|JK��|zz�z>�c���t�9HIg]-CN����z�z~�LX�O���@�f��'�t]�O�8������e�iIjC���!)�M�
Q��qK )�')��>W�2	$e )������7I�9��5IQ�nVr�����%���dP6�A����J:2(���l$c	e%��TdP62�A�H��J2(i��lda	e%	)T�A�m&3�������+7z��=
Y��c�{�Q��IMW������V�!�������=�h�_�N�?U�@�v��~:	e�'����]�WU�e&���H��qo�5��A��$��O!@�$MZ7D~�d},��m���{��"�7��D��������P����Jt�c���%��R~k�f{����2wb���t&4���Ff��x"�'��V8�������N .mp�����8����qD�
�')mp<Ai��I+GL��xR�������f�'��
�T/�'@�w����P��5g��X��r�V�1Ka��&�����4��0z��h=�JH@8i�O.��5�e�=���M&���(yL��VV�W%���)�Wd�g�Mu�pwXj�U���)�RC�@+-����Z@�fF�U���j�s��Ia.Tyea-������y��:�;���yc2?��t�%����48�����V6B�V[9	Q\mc&$y�����6�B�X��
Idmc,$�������6�B�Z�8Ilmc2$������6VC�\��
Itmc8$��������#_���&��7������
��
��	,��qW�Uto"�sXE�J��
��{)XE�&"f�S�w�N�w�����S8��D����5���g�=�m$�\���*a0�C��_�����4y���O���
����H����AI���^��s�Q�W`'~,�O��C^�i��8��{nz(cP4q�q�������W��[��'!\���-���C�GB���a&h�M����at�m�ha�'��>4[������Ln���!������2����+���%D�F
���1j34�� �t�^�s��E:c�+� &\����p�c"?Qh�	W:�������DU2���������p�%�3��W?�_�e/����D�4��G�w����I��������v#+�H	��������x���u@�uA}<�G]����T�&��l\�V��U�����"/p���z�B���}��E�X��h�����>��/����7��������1���<������e�"�o�;MT<�S_{x>p������>�j�����w8|�	��}u������A�j�
hO7nn���0���T�����{��vb�O��'��4?���i����T�,h'��������60�+5[5)�)����g��K�:�+Og�!
`*S���p�=d�=�3i'��!
���pH��[2�����k]�����#p����Z�O}�g>�g>�����j|�&S�IX�H�Ue|+"�DA<~�KE|�>�a�}X]����(G�@/-���������-��e������Np��
5�X�8�&^
�:�e��Dj�AF"�� #�n��H-7i�@M7�H�����@�7�H����3[����o+�����met�VF��2:k+���2�����S����������U�AT
W?�5L��������|��.����f�p������||��T�L�>����=��e;f4r'?^��T;�2�����>=�Z�OFGO�42�����g�c�h�q�4�8[h�*
4�WI�,V��t�qt4��4h�
4n��D������Pc;py�IN"��b<3�m=��H��Iqd�x���A��We���=1�3�mug2o���%����[/����^N��ixC�#lK�����&�&��9J���C8�Q�e�$����|�O�=��#>��'�'���D�<�'���g��3}��6�����ZW�A���[h���Z.��}��f^�����K�yr�@�"#��K�$�����IN�{�p�1"�"I<|�����I�D���#�a��c�f���v�l�;��kn���&���@��>(�(V�*��e]����y�r�Z�U��jX-�ea�<����r�Z6V��j��lp���uh����1F=���Q�E �����83�gF�H��4#�f$���dl������&��7�j��	��E�]���H�`����/�]��x��Y�l���{p����C
�yVl1�@X�w�3Ox����9v3��)�,�MQ����r2O�qdZ��X�0�Z�P?|������O��$�S:�,N���9v�c/_��7y��o������u~k�\W��{��n��M����u���%G,4���4��3e;�c4t�rl4q3����3e?�t6���j��T�������8��|k2LT��#��$���a]�-E7TO�U�������JJZQ</o%]]��io%�]�io�O{+��4�Z�^���6PL����b���
#@S�b��j���d<h�V���������[1�Pz0��K2O4u�d�h�V����������%���K6=q!��0
x!�}��)����s8$D�������T^	8�-�2��"�����C�B�;�z	��L��0vRX�����R7�n�P)����go��\�l�����1�1-~8[��)Iu��i��Gi��Ak�N��N�\N�N'���xzVB��gtk>��c�*�GTb�����?$yO%^H?��#�)�Q��0�Nn�
�M=�Ca�GWb�J,���
����J4B
��(�"y�*��(�!
`��C���B@.D���(�!
`w�����|
QC@[A�$�A���(�!
`���`S7�PQC|��+��Q�_Gw��:!�N���V�_���B|��:OF����b�m��9L\�V�&�V7�����<�`T��@��2�o��I�d�v��/��eB���������)��Q��w9�z���/�$��H�)�������`����q&aZ�\@�~���8>-�����_�M���@?�=))I��q�;��+r�y�(���6�r�/-�u��U"S@6�:4&��xT����\�k��PG�����j�s�����N98�gX��*j(��^�c�_���P���BGE.��~js����V�J����?� hz���^��a)���OX�_�I���~v�b�'��	hsMg�)\&)K�[���Y�&�AZd;�E���6�Q���m��m�F�6��m�3t~p��-����6*�v���z�4���o�.v�]��]a��D9^�e�T���4l�^ut�i�P���
5l�mm����������:�*2l�a;U������Y.a;��v�>2?�}��#��Hq�a
{�_�j�P��a
{h[{�~���l����x�����]��.��u�t�D�n4���y�Nq�������������<u����0����	� 5�0�L�S���~@����������
�u�AVO�����n�
����6�\@��H��\�i�~��D�R�!�0���z2[�f��a�;�m�f�����r\V��iMtw�/�&��cS-����M�mN��a�W��R�����r��N��f�����/*�pL���e�N3}��kKTtZ�Ac����(�%�&����+}�t*�������e�W�i�h|ZQO/���������{��>�����.b�*V��yQ����E�����v����Ui�v�4�F�)����~x:��M��yU��T��KW�N	n���S���D�b?Y:���8^�'KgR���d�LRBt��,�IOHx.���3�
	��~�tQ��W�	����|�\-�	n�(Aq��?q74���	.����^<xi3�jv/q�F�
��%�8 w2��v��M�h��k��f��LYV0��� i�D�����I�N3��M���}������Is�����o/+�)o�F����S
wc|�5�OSA_[��,r�LP�.|o���#��,I�������6'�&>m����K#(��%|Y�t('@3�7E��W�w#i&hZ���O%
w)C�"���i��4��vt%<�o�����������V^s�?��9	�����5'B����U���p�������uk�g��������~�+�n�~�(���~�����~��I�H@�+HoER�/vm�XOyb'���a�w�j�G<-(�L	�c��/e�L�x�X;P�E?jC�`6��It(fAM�~=�R��!��uG7��i��jS��c����l���mE����U���*@o�� +x��|k{B�����Q��1���|D�����a��}�����������kwB����VN�������y��?�g�S�1��n#�q����
t.���B����po��� �[� �W���U��-Rw���mX�3��7��Q���N��������o�_��FI����DMAP������ku�%��o���6Z�-�=X�����M-���]����S!��c��.����E�co���6��.���2����q�(P�7���|��K�����/���(h;����l3��6G���._]�kL����_�G�h��h��������w��]����{��9�<�G��<7d��e��l&�lRM6����������Wo��|w	2�|�_t�E����d��
��>��(z��Op�=?��C��7�6�*��"/���>��y{��<��f�3X��7���}9y��'�#�
F���)��Sm/&���f3�����Q#1F��` w���4�v"�^���n�C��=�x~����v��r��J�]-��*�K�]1 +��/��j�V�Y�C���*��o�J��!������6T������.�E���b�����'!	��9-V(5���.��#�42E�Uz��wD��J��xv�p`<����tp�gs��)���h����A�xB�L���0�O!�� ��<�?Ly?�8�G;�^��l���:[��<b�t|t(�AS�	(8����_���e�!�����To9"?��i�eb�>I�8u����(R���s9|�(������2?p��F��<�BJ#��p���A;�� st�
�3��I�B������!��f���`�SzDPhg���� 4������������S&4 ��Q�#z;;������x+�I��\
61-u���$2�����D3��}���{ uZ���a�P�1e5�Fx3��r��7F5��,�of
*C:5�B�Q���8�#^����Yc�33e4���`�v���tXHlg+���p���%���7ft���a��R�?.�y�E��S�)����W�/_�?�4!dH �aN����(����#q����
�N���Z�����t���go�Ap����s-%N����e��]��$���/��?����t��L%T�q����yp�AY�k�V�	Q8�e&8M�����)K[80\"�	��S��<��M�K��H[�������:��fs�H�T����{��D/��%{
��k���F���Q#����Rqh���E�-�!�$.a�>^J=X�T��X�*���m�!�d-Emn��NZ�~���s1
�$��ei��n�G���&��mM��~��>�����������n��;�����vNc��!�q��	{�D9���G���\���8`���\�6�K��rI�r���&���a��&���B���.[m����F�n��]������@������������*J������q�#X2I��L��(D�H$�L�Q����=���#��PO�<az����*�"M��T�����J�Y;O G�����r`�
&�~���-]��JI��G�<+�W�riR�y}q�����R:�-#�����
=\��
$xH��%~�^�2�2!�Ld�
_jO��+����(*
�P�(N@Mx� ������g�ED�c^8��	]G�������C>	w������6��^����/��sR|5�R������*'��B�}Sx�)�Z�8�#0�^U^�{������ni|	���4wY��`�5��k�R�ab9zz[>H��������)>f�M�����s�*
��B���	9��)S��g7Y�k��\f��{�^>�W�ri���3?�����A#?a��O������Az����GW�E��C��]�=)@V����$o>�7{������E�hsO�~��Z������y*���V�u�+�jO �R7cOue�^�sv����c���:��������n���U����8h�f�>��";��PU��LO2���������H����p#Q8J4&��{��54���	?�{���b��4���xhI��Gu�ef}F��}
Hr�o�����X+�_<Ox����\���9n)!���{�J�/\r��q`+[�+U�w}�$���W�B���4�\�Hj��T��$3e3~$��0������TN����6!}�X�A��dv2���NNf�,=���$����S�}%��I�&k�d���'��5C��d-��LS���2����L_������W���h�@+Z)�J�V:�R-Zi+�k
���g�P=m83<#���SS]��L,e2|�������Y�G�o&w�B���)or�;j�HK�-
(��W��>F��	��4�-��h?�(
�-�y��T��
��
������wu8��M������Ic\�8�+�3Lf�����(�>LR/���<����U�w�q//f����$��}���h���F�)#d�x�2�!��I��<ej�q"8� ��L8&��c
d2�$��D'�J1�W�|���iL_������4>c���.��R:�}���u�4Ib�KZ���&����d�QZ
�l�ziBK�����S���4���X��G+�?��l ��'��k.WR-	�Z ���R���7��9��5�Q�K62����h�F�1(iVBY�2e#�J(+9��l����a%��cP6��A�H/
�C�Kt���IJ������++|���uO���7����B7�M��~'����Ur�����E�4�n�#Y��W����C=�U��v��yo��	���f����t����=�{��{Mf`vMI���Y��Ll��tMz7�s��r,�6m:{�{��"G7��D��������P����JtL����%�R~kT\{����2w"���s&����F�����
��n68�p��	d�
�'�,p�f��6O�Y�b�
��j68�P����L3����:1��V�r�O������-�#k�p��:]�b�<99�7��;��N�IQ	Qw/C����������|\snF����h2�'@@��G��@e��`��"(�TVL���"�(km�����R���Tz���v���o�WW�:5=p�����T����N
s�z+k��D�������y�y=L���a����(a���E��Y��%d������F�KbV�.�Ym��$h������F�K�V�.�Zm��$l������F�K�V/�[mt�$p��������d3�(Bh�N��fn`gn`�N`����S�
��C(XE�P���`����L�����l)��c��_���8B��g����!��H�g	�M�d�`v�\������������
�f�~�����������
��s�p�+�?�e�T'��k��<
�P���l���
����:��a�����-RyW��:1#������D����e�W�lU��qT)JI'��R&�W�nK8�R	�������z�g�#������_q�J�����}	�S� !��._��}���^�I0$������uA}<�G]����T�r��l\}&
��]V�v��Z���������(�U-VQ�p���|�%���y�7��������.���<������e�"�o�;	R<�SO@��9@�cc�9��\����Z>+x4�4���z.gW����<A�(�����9�d'�����vv(�?� ��b8W<�s�a+�SA���t������5�Vl��f�P\��x+��lt�h�W�Jg F�T:03N��-s��y]��y
 ��n�T����F��D��n�#�9���|��|��=�������HX�G��/�[$�b�C*2	4W���h�������@��zii����u�Z��/SqD��F���TP(�D��/IcI�
pj%���b��D
�AF"�� #�m�G�� #���5�#}Ff���N��h�VFgmet�RF�q���p���,_������^KS����je�����9���7\<{����������M�x�1)g��}<�����1��uLD�N�S�QQ�vb<���h���ZgF�������n3��1s�0���h�^4�-G�LN��m�qC1�8�h\K4N$w�-�!���1%x�@1x���4�;�����QT�G6�`�������1����7B���x�������(|����o���6�1��
�nR,)���S}�L�|�O����@�O.�0��S}��>y�O>�'�k���A{���Y�U�-xp��s���[M��j���]������g.X���-��������?�m��,�����2�w_4L�&����������Tb���;��w����]�������&����j��r�ZvP�jB-G�e	�<��)�r�Z�P�j��l$�j6��F��&�@��#�c$u�����1<�G2]f`�f�h����|5�������.V�m$)3�������.^^<{�,b�����=��0��!�<+�x��������'�������CWEA
��9|S'�$������.ES�x_��@��_^���I����:��6s��^�`Ao�b��D������������>C�����J�J5�WY��q���]�"����SF�*'�F��������e-��U�Ku=<�N;���<��&�D��<��J2����RtC���ho3���H%��V���y��VR��%��	��i,P��h�����m�7F��d1d�L���bt9I�F�������Z.��hAD#&S`j:�����IX��J)h�,"z�|5AV�R�B�P�WU}�������(F����*���:�[�ewD�|>�	��-�wW�t}yacA��$��`)�+�$u��sg��>(���+�c$�`�p�gS��*x�����Vu��8�����9�6��N~���4����w���[�c��9P=�"����R�$O���������������Tq���
Di��@��&�	�m(Xlc�j��6�K�m{_�mx������|�+�)�B`���V@��_���6!�Ml��9o��<��6����
���+���]��������%��/7�~���h)�����l�Wa�b��7Q�����>e��%�q�"5Q��|���%b*y�Y�����D��c0_��4�m�9����]�h�j��0���+���|��o���3���j�c�-�~�7�n�����|SD?/3�O`HJ
���n�� ��%]%�[���M�\�K��t�|�������p18�+td������;����������x%����SN�������J����E/��{�W��x�^���|�����=n&n��R�+a�D��=���p:>���8���I��'9/�����"�7Hgg;�R\���Q��Z�
[`�w�v��w��BE��m�ria;�GM�h�6��a���6�����-s��8��a3���SM�fh�6l�a3lk3�?N����c��{�����P��V��f���p�[a��|{ ���	�����?�::�4��o����������l}���E����3����@��nF��.��Wo��|�x�������~����0�����0����	� 5�0�L�S���~@t������������u��A�1���k��n�-�h����)�I�_����GdN�u�0��^��z��K����=H)�r��X�U�������M���D���z�4�����,��W��h�@�	t7�_O_W���$j���N����K��2�1u`4��P���e�:��U�s�QO��:����Mo\W/��������b���0[�����B�Om�N\����>��;z<Q����q�4%��M]�d�,���i�����10E>1������t��j�!�,1	J�~�t�&�`"�'KY��r�,�OGc/��1nB��z:���s���D���G��=��.��\�ci������������[�f�F]\A��.�Df�';d����?m��O�v�YE����c�n��1n�����1v���v'�������u��E��m>��a�]���9W7ya��$Vv��E������Y%�l��U"5
W�����3V�T��79��x��`��[�k�	��������A�;W�&K���W�o�������"a9�;���o�����6����4����L�����2�r�-��z��M�c��>���B���a������58��s��6���F��V�)F���y|_�
�d���>lEX����m�L-�_w��TY�3�4g��y}�����z�=�f���=����T������
	�@��W���,l������s�-��zx��a��"����Y���W�$]]�����b2�U���`������|5[�0"]W�SRU�p������������'x�P�-N?a6����e8�{t��&�p��^�.O��B�
�x���\>	�x���
�x^�i��{:D���x�9�����]^���K�;�()W����)����}^��M~�=,�����.z��rQl�O��!?A�k�f�/����.��c8��+��w�~X���`V��1�l�k�_�t�yQ��	��uQ,��yt��D�r��"/@���{yq����h�E��vf��A=6��ls�|�����+0Y��{����p~��&���p�?�}����_^�GqT�����G��<��J�������v���[�����?\���|������.A��o�+�s�h�1���j[_����8E��	.�C��7���*e���(���^�U�?����D5���(=r+�
�R`	(��7���}���~I�z�����'�A���2��n���'�t]��z�]������$��~u�������s�������s-9�qx�?,�-�<�vHu��<)��H�N�D?)�����Hl�"������ d��K���Q�C��.����+2�&Ev7��%���Xa.}��2+�]�3�O`G������j�l8�X
���_���^�7H�ju��~�uA"w���K���{!����&q��1��Y`+�+0�����B��A�p�6>;�0�=�7I:�0���%�a���I�������I���{�of���e�0
g�^�]�L�U�6�|�-�b���f|����Y%�6jW��wo/^�|����{������kP<�0��/v������k�=�u�F��7H��F0���C��p�n,�?����1��6�m��W��j�A��v��U��

���]^<,������@ �jx�]{�a��:w>�u<`��O~���W�������
z�p
X����(M��nT�>�j$4�3�u%��k�
d�.�����%��
Nf����~
S�' nPG0�{�/�&����K��=;��1~��[�Y��	z�=��g	~������~L�[�5"I����q����e��f�"�4gv����p�q&HA��&������_��@~ahE�s_�bW���Y��/�~����k������-����_�!�j�X���g^��l}��}Vo�YUb������Q\�1.��%c�lWeP�������1����e�����o��F��f����+i�n���h��C������a���;���O��<&����U�z�P�X�����e6���KT�S�)������_�z~�IA���@`������Q��)G�T��p��^�(Y�*��/�b���go�Ap��"o�3+%n>S�����x��O�b�e���C^n��h�d��I�%Vr�hF��PiY�M��+�z���~F&�e�8��= �J�\R��p�D.���J{
[;���:D�f��/�%�!JQ�8DI�!�@Li����T3D�(/��Q������C��"��VI2$��
)��v>BFqG4"��7�	�J��.�4��F��/l�0J;�!��LG��[�"�O��������F����������F�tG���t6��
�@��8]7>l����������#�1�asqm�-�1C^�	3��=��Nn�<��'�5B�x��M�b~k��k��s�	f��=�GU��
����
��0�
_���
��1������>\\�������I'ad{����qu\W��&�Tv���Ee���@i�O�.`O�{��M��������G<�v?:���=����Ztp�k�D�s��)jE��7X{X�V�Pk��4)�Cdf'��Gf*��Od�'Bq���cB���n��#3h���l�j�Q����y>Z�Y����^j��/._��x]���3���c,;F�#��<���~�^�2��!7L�|�-<����JGGQQp�Eqj�kU�'�v�J.��v\�X��[+�����`
_��x9��p�I��r������p�r�ub4�t�g�l�>�����%�� ��(�����c��^��Q�c�clQ�����zz���
]���O�����A��E����������B(q-��������27V�t��B�V��f���43�Q��n8q������(�q7�8c�xDG���N<gJ�D���R�N��^=,�t�7_/�����
��U������!a���$�'!�T��`���!���a��	UaK��`�7jY�&����v]�rI�)K��Xu���y��\5��:;H?��tLez���9���(�{4s��g���b����<��}q���>}��8;������LO2hu�����������[+�+��m�V��Fu�~|�����> iz��P�����R���������o�������s)��x����/�w�5����[ne�j����_���=�����7V��~�$��IZ�F���10�g�0!������Jj�l��df���6Y���iT^�K�lV�g����t,^��'99=��N�x����x���Y|��Z�(���2��Nfg�)b[�Yr��Z�*������g����i69��d-w�LS��C�l���$V�����6��&u6�%k�D������p����$�u�f2�2����&y��V���3NN�^�Kk8�J����a*�@�B7���
�n t�{�q t����a��~�Q�P=����Xj�atrM-:�:��c!b��3��������?�~3������"X��:J�O�t-�y��NS<�p���+v�U���e2�Fc:+��47[3sM�9�]���Gw� �������/v�����
P��uEPd��e�"�3��s����cW1�eW�<�:a�?��U�Z�m�q���������h�����������A���F���*�����JSK��9�4���0�x��},���"�+
�52z4��E���7��"�A� �LS�D�)#��x�25��r�
v�8�8�\��`�s�lSL�6S��9����`I�sw	�?�`�	�R�B�T�)c7R%���)-P�����umJT�6���ugC,�M�9���P�v�p���M��I�qkZ����3�2u:�8TlZ��Phv0�A�t��Gy�V�0Y)�����j@xV�V�+J�7+E����RI`������qU��J%���V.�R����`��n|Tc&�12P%��yb F�������01#��@��R	��Ib F������1#S�@�Q	bf�(H!1@H�G�5'����v��W�9�V�oaW����n��!�9�v(;�i�:�X[ej0;�c�*ut�Q��
��3���ls��W���[�LB
�`���OI��o	�oI������6�|�W?9�=R^m�6���zGI��H8'B	�q�hY@��l7L�1�b���)	\V��o.+���l2���������RY�w�)T/��XwU���3�������Sjc'+y��9��)�������zFq���?sJm�d%�u0�P}�����ge������+��U�a�[���A�����<�e��8,-��\�!x����TZ������g���e���+�W�!x�J��;��.��[%�w��.�,F��D�����5g��Y��r�V�r>J�(MF��i}���j��AKF{h2��\��k.������<�d�	pd#���,�����AgW=����Rv}��>[�s-�����RS����n��i�����uD}��y5��]W���������U����9�����e�;��	7�y:W��������C���s�}�d:��tN���Z������tN������"Hs-�"=������M�&}5�h��"�-F������F8�7���]�`��K	�v�����1eP�Y�j>��@mg�2��\S5����_:��l��2��(T��*��#Q�|,*�Z�F%P���8��>��%n`7������
��
���O��%����^i�W�Ut��f��*���Uto""?+���~G�~y��C�t&�f��p�������+1������3�`�0$<����.a�b�������#O!Q7!��6��T���	R,��yRp�����x�������{v?#��[��m1���&xm�&�����i<%�w�����MO���v��������u��4�b�J_�����������x�J���%�dglP�p;i��j��D
���(������e�	W�0$
_0�������VL�G�H�������Y[�!6���Kv�����8�=�(��D1�_NtJ|u�wJT�
��c"r|�^K��W��S:������H�C��������I��M��i�����7�`�H��������o�^�H��o7yQ,��"+�x���C��m�zI�
��>�5�6`��"�>���:�yDY�����I����z��3��?�FA������y� �{�s!�s�`��qW_7*�|�G3E���H�h91u��(���&�|�/��{� ������>��+�"�#�T����NTdv1@���
"A�C��4�\���
�Ii?v�&E{1(E(#'3��&w�����PS��j�4e���HS�*��O�%�t�2������h�3e(���bm(�R��2q��8C*�F�Sg��3��3�y� X!=�\@��=RC���X"QV��P�L �^���w�H�@z�{o����n��<�KC��_�v+�����qd"d��x)�~�\Df�n."�P7����"�������$f&��������bci�M��3���\&��2m%��Vr����Y+��s�5<���m.
���5��Vx�8�P��TM(W�������V�ii5k��T�����G���|Z*I��[�~^j�X�����g���Vi�
:�����f':����5�b��I�nt6�6u���O�ntp�j��HbG����f����q)���9^��K/T?��S����@�nt��-:�8U%��@
�����G�@�t���9���h�z4N<[��%�NmL"�!TlP�8qi������Tl��`���^�d�7�j���{:t�9��Qr����*���D�F�\JzP&{�jK�.��b�R#�W�=���&IM�c;��GI����w�pR,�%���&m�I;���4ig��s9�{.��5i�&m�I�j�N5i3M��&�\N���Ys����E�3�[�����an��3�.�j��!��	���5��^z�������R'��3�9��U�C�\8��y�mN���F�����:)��Ip���j�h�r��O��Wvc���M�~�K�n;�U0�r��B����7�c�:������n;�~4k��\](���wev��
v���hG+��
w���xG+��
x���G+��
y�R��G+��
z(m�*�TE�V�BQl�[�EmQ�t5/�D���_��"��y�4N��5/�������8�6��PG��T4�Zf�d�h����|5�������.V�m$Y�1y������.^^<{�,b6tH��=����;����e�[�6@�@����{��}���A�uEA
��|S'�$����}�y�.�[!����>t�������!�����^�>%jR����4e�����W��7�}��Z(�v���R�rq8�/)��1Z����:�L3U��Y��/����s��m��O.���Z���`��-���Y��/�W���������z�l�QSc���4fg!+�,��SU���7|���wXWtK�
�Sgk��1�C�yX�J>�4���J�������9��6Pl����b���
o��@q��
��[�����k%������V�L`�Z�]��k%�����V�v`�Z�4�.J�)l�R
��9����2��=�^-8�����!!�&�nn��0SB�)"GA]YD��� }�q$~�	I%AmFC���M/�VZI�6X�w�.C���&`��	����-������+�i}�����c�v�c
��)�ps��W��V����c����Ba�V��u�+����3`�8��u����$����f�J��X��2��db&����?$yO�^H?�5���_��Y�q��P���Y?]�I#(�D(�7�K��r+�5��nbnR!�A��w=:N��!t�@���#��S��3���
m��q����btk#t������Cl,����[����X�ur��d�8��iEQ�y,������X�:�^8Nji�9��6������������j�u<�V����7��8�D�]�|�4����������p��^fn�Uq���^�F�������/����"ui����uB�D�Z��E4@p���BD=����X h�2�[>����k��� ��)w&���cj!��3�Fv�!ljT6������~m=\Nsd'�����d���3�`��=�u(�G�����5N����&�5�
Y�Nw���n�[��W��������YG���p:����FsM�3�������b_5;��s*Og]t-�YS��Y�oi��z�|av;����p����ZD�����.k���|�b�q7��<�����������A�Z5gA�PP����3������s6e<h�F�s�f(���s"(�J1��v�jd��	��jz'���W���u�LOU�IM�/���I�������3���Z�a\�j�7����
��
l�v�v��'��qU�Ut�6N���{�5SV���Q�XE�*z�z0�w�K4m`T�7L��[n��
�~C���7���
�~-M	�~���5]��a6C�_�`�h�����U��fm0q�Z��([�DKh��~3'��4PmE���[,MAb$2�������D�P0�eg�4�m�9����]���F�,"yE5h��77���/XbQI�9g��e�F?f[����|�{���z�k�)����'����*3���t�p\?g�l���;oY�Q��6�MB�Cn$����;;���~n�9eYVs

|
/�;�j���R�J�Y�����~~���\��,�X<���S����a�}�������6L�������������hKd�i<�Y����X��$����&g	K�����% �$e	S���Y���g���v�
Ra��������=4����	{�����P��o��P�cg{�\Z�B�������'l���B������rly�����i�M�n��C�M}�	�i;�����#�S�cM��������4��
�	{�������n�<U��.�sh�@���C�@}�	h;�~p������Uv����c��>��u}�D�n4@���y�Nq������f���&4
u����0����	� 5�0�L�S���~@�������������}c�A�!�n���tzK
�M{������*�f�S�Sa���Y�5p+������3�J.����s���-h5$q���A���Oo��r�h��;E�^n�7j2s�������R� }h�B6�z�a0��m���"[b�xxL�5�:�3��u��$���C��4�i�����������E���?��FN������2�Hd������BL��I�����v�5j7��k�Osv�5k��8�A����7��B�Q��R[c�,���z�F4AWU�:s�1����kd���c�����^��:�-�+r��71���"�!�L�O���9$�)������D2e?Y:���8��'Kg� ���d�L:Db���,�I�HS���3)	
�~�tQ��W��������P^������m�#����$_�>��eZs�"���t���[\EU��{K���T	W��\��4�j"����_�K���u��\�V�o�U��������Y�(���z$y
��O��T��3�e#SG9^mYz}aLx���R�Bc�y}Y�\���G|�Bz�)d��z+�^oE�_�z<�E���M=�c.��[M���DFn�����T�
�WC�*W�9��-����X���^�?Oi)��5�C��)���9
5��8�p+A�ki ����n��7��}���P��Lj%e����$��]�}���b�z��Fb���� 3d2C�+�Y|	j�V�U�&6mA�X�6�*6��^m*�d����:��j�$
�W3���.���6������mU0Y�6K�����
�.��I�PYu�����SY�Z����w����_}���$���&m ���Y}��� �
�]MZ"x/���3�Z��;i���}t$b��PS��*���������u�[|u �e�jE����d}����L8��NE����j�����U����e���_}��2S�� 3u�V��Z�>q�;��>e-� .U��R�jq������t���-�I���)�I��*�2�������*���K�.�]�K������[|U�"�K;��l�tc��m�i�jl"��W�h����������������_
�k�+���!�\��iy,���J���|���p�i�����i"��V���6L���2:m ��W��iP��� ���������]^���K4��()W����)����}^��M~�=,�����.z��rQl�O��!?A�Um�%��e�r� P6���F���z��R���m����|:��(r�f�P��X\/A�E��F�l�B��5���j���f9����{yq���h�E��v�	��A�7��ls�|�����+�$�
{����p~��&�s3��?�|x�.������|A��s8����-y>��c�+q�k������s���7�_�� �o�wQ.F�"�#��E��`���9�G��~���aT�!_����
���b8E�1@<�z�g�����"�Q<����lt�X41|����0��7���}����K��d���:��l�A�f�DMn�&��_�p���K�����f�uE�2��v��^#��n�C@<�x~�_{�v%�rj��\������g���W��
��i������QN���T��1+�Jw`m���r4����
���n>P����n�:����^�������#V�qj��S��hy��M/P��XHTM��i�����x���[ja,�~�VCxl����e�x��l���32|"��3pf�M`�.��z���p����!��Y�Z�Oc����2/����G�	tR��@����x���w/��ha�h���[ELf��V���8X���|�[/�cm��|���ybA�-���G����p����E0�]��|���!���#�5L9���-�`
�_���#��{�o�����{�o���	z�=��$)~������MX ��t�� B61
-����$2�Z�4� u��0��)����as+��1�(#9U8�������3^!������%L$�Xz���V1��&|9��7V;7�=#
���o��+PB{�c;������*d�w�����
����N��2��������Ma����z��
��&�C������O��+6S����C'
4�25Z%�����D<� 4�	�������$"6��Q��*������K"������y�����eD��! ���K@�ev6/�5j�I�r�Q8�e�8M��}M��HY�B���G.������N�S��V�|��^e���%�����k�6������������������E�����]�i8w��/l�j�/���m��Dm�v��/���w�&�����������Aw)���mE�E:(Zo�.q�E)�1q�IZIS��Z<�k��������
C���������C�5)���Y?|���w�I)��z0\��k��54n�"�gl��dUf��*G,+��B+��B�*������J�'H�;�������H����������	��g��lG+���b�j�-��wK���$T��.`���@���Z�M����
s2)F�K�������B�X�	E�$��&?����0���Ok��]����[�TX��DF���D4�����e	M2���>S)����h�g��*[.M�#�/._��x]J����{�1I�����)6��o�k\Zo#d���������\F1�<:���;�)�P^�E>���L�r��)7"4���z��g��Q4|������O��&�b7���]*�nr\���kYg����N!Q_�]�1��OT��0�f�q'���q��2:k�!���R�a����&���X���Or���=�`�09��n8q����i69I�n�H��dU���r�A��z�p_���=%�N���A�gtm��!W?I�O�B��y����\%qS���[�~�
����V�H�������}^.��lY�"�s(]9g������l��=���������?��}q���>}�k7;��P��LO2�a��y�����#j�	��B��3FlP	s�6��p��wo�Y�R��w������4�.�V~�/;MR;F����R<Ox2	��j�\�$��C m	�����+W�~�������B�h�N�;�����"B
u�H-�0������TNeQ�%"_�d;��p�L�N�c��J������t����<N�t����XM������1�v2;KN���d���j���L��)c���=rO4���j��XM��)c�!k6�xz��Z���X��D�:��uDs����ZVW�_���@��}��@���0���hX$8%��_G�*;���k����E��x����_3��;b�4����1T���q	�����kW�&�eU$@�2���Y�`��5�
�F!J���Fu���Vc�^����n�u]�����N��:W_5O���7���%�N��rV�EuU���W�@ZP�r!)��u��f����`�z�����4<��+>
��F�����p��S�I�g�����Id���@�)�5K��X��VG=N8�Xlz��04�DKb��J8*E;M������LiNJ+��3��t65�r�pD��V_i�$�(�	�Mi����.����VNSN�8N��"�3]&�T[^-]�Q��Sw�R�$QZ���>��A$�7t�m����7Jum�P.����Ov�d��U]vkt�_0���P+�������� fj��)��Le0#�Q�4�.���(A�T1R%���� =�$D�W
(�4Hd4W��T�E"�wx�;�[%�x��^�������.��$�9��:��O��7��E2�@�r���` ���z���g�!���@�,O�SJ�&	���{j)��56��~:�l �IMDb�5>�5V�cj����H���5�j3B�X���J�����i�r$�B�N�<a��h=O�h!�mx��)Z�D1@L*�!&Z�������)�{G��I2=���Kb\�\��9����t����$�y_��t4MG��d=�D=����)B	�ICrN^�9�#��5G�`2�'@���G�H�c��`��"(!PVL��"��km�u*�����j�\�lE���������6�k7q����
���t�Alq����3�c ��(�e�0��4s��ii�ZE�!p����s�����[i�������*���,R.	�&��A-�.	�&��A-ZF��K�1	�& �@mB2�,(�A-�2	�"0G�k�,q���M��N��fn`gn`�`|�,�EXE�J��
��{��]V����{)XE�J��&���"_y�R�o��'����c��A_��g8kn�@�l'"�BH� 9P#C�
d32�\� ��]�,v-Y
���u��Wc-�G�W;�j��j:��NF�WZ�k�g�Z�r"�cD��5��b�/Nv�[����TJ�.��(�^��L$$y�Q>���.R��4f�
��T�,�1��T.gL9M��1���A�4��A�6v����$�y����cZ����ju#������x�w�y;����K�������^��M�������>4`c����@G���+�:m�|X�O�=�"[.�[���DD�Ek���|�*b"�����Ob�wY����j�b^��>n��X�WE�x�p���|�%� ���7������������<������e�"��]��]��AG�M���t�R�Kn:H2V-?<
!��&��ZY�`�6�mW����Xs`�N�v���,�O�9S
�D����s"��T0h'�F��O�\c
,_l���t�W�BR���N�*CM�rWj���P�A��
*�8�e�uS���@&��gH�(�y��<6B���M��%�d��B�x�$$,�#e��	���fe���Yu�j%P+�Z	�Jua�S+O'
ty �������8VJG�R����Dm�7�R������S�\D��n."�U7�F���t�����%i%�I+�L[����\f��r�J.��\��:}.D��3}��������&>`��I(2T����WE
���Z��T�6������cm|��>��[�c�o:7�����[��N�:>
���������b��Zo7
ae72�����4f��M�@c=�X;4�^s����y�1dh�����?��J�lm0O+��OGK����+`;:�V�j����VX��ZE������������r����r�x��G�8�b{�Ug�*����D�6��M5i����&�L�v.�q��~�&-��M4iSM��&m�I;����ip��5�T|MX*k�o���q���TMaj��bW3'_����/����8KL��s��pi Z���f�5�dF������1�/�b�4b^��M�&�te�O��0�y������Ff?����YZ>K�hi9--�����������[Z~K�pE�i�H�h^(g�"-�ya4�S�E�D�B9�����(�+�J�D�E�Bi�H�h^(����q��IEm�%�����&��7�j�����E�]���HRU�^<{�,2��#���~��&���}��+��-zh�����)k������!���I =�c�i3�V��"�%����o
�����&���w�Gz�8���&�
u�C�h��}��������f�j\��� C������#F��i����������`@{c��b]�JB��4������6Pl���b���
���$!�3K+%�S+%SS+%�S+%�	S+%K
C+���>�W��+R�)y(��9+��
Zo����e���Z-���A�R77�-��?%"������amB�1���4�%�����b����,�R�E=�
��G���zN���^X����1b�E
��lJR���_���giV�����(���+�
Y?�erj�[�]�G�."���|�?$yE�/����_��a�����
���0��+1i%��E�l{��Pn%�B\� �q|>qB\�@U��!B\� �h�rEe!.@��EAa!.@?����j4�!.�t.mj}���B\	,���mW�&m��o��o���t�&���I#=_�_/V��6���j�o�lu-�Z���oC�D������|�YG�3"��v��/���2��~0��4�m�9����]U,���_@�
�zs�������-7��`A�~���5�O���_o~�7E��2}����e�N���=��]��m��2_����NF
[5��}~X��:l%'�f�SY�J,XBz���/
HN�7����8t}-�rX��^�y���!eu���~��o�W�����;7O�����y�0���9�%���}r��9�������%��K�)K��M��% ��$e	p{�����6Hgbg;�R\���Vf�[�W�y�F�����P����n���[x����L.-�da'3A�����<z#�dm�v(;YW8��q���ix����MM[����M��������qkk�Qs�_Td����&_aK����5~�[Z�(�{�S������>�1}��>�1�R�>�Gl��-��rc4�_#���[S�`Z��{$����~����w�C]@�kl}&~k��Rg�@	�A�?��Rc	S��0�L�Div
�M��;��V�t�e�0�L�	.{f���K����[DS/���Zv�>x�yQ����E����K�B�:�1���y�-�[�r0��*���-HHl�9Z0��u�kAps�Z�&Z�a'�-L����$��:���,9�+
V�i��2[Hg47���~�tF����'Kg�9���~�tF����'Kg4<���~�t���+�.���bI�������.�jS�e��� �U�b*�z��h�����������$ �~0+68_)���/��V�N��������i�Xgj��X'�������&����'mp&����L)��6|5�E7Dwa�8����.��)]�[�5�>�d�aCa��m[e<�����s9\�30.x�q��F���i7��0k:�e����CD�\M;,Mg��t2<i|D�!.��j���Q-��I����%�."�V$�nb�]2��k�E8�J4���v���
�c�9��C��K�������;���<�g�;a�����=���E�/w8�(Y��U���33�����	�����hR��� =�� =��L�������Z�*����"h����	W���7�a`2�w�����,&M8|���2��f�_��v�[����*�K�m0O�l��p����?G�{�d~��\�JZ�� ���o�y�
����<+�"����E�����f��z�X$/(����(���L������|I{tf�0���LRt�G����.�E?_��|u�
�0��]����0;�F�Dh��8��������//��8��I|]�?��!��)�9�G�K���km�%�-Jt-��C[t<��80��KF��F�lJ��������2���y�J��>�	�D�j�}�`���$�':5
���d�]rt�/���d�\7���3�����V�1��{�ZkZ��R���
P��Gx��u��H��Wu������]�P�]L����8��tj������E5&�����D��]f���0��@�z�h�DD��};���M��d�h����|�cS>�K>�s��)�����/�]��i�2��Q�@�����`�A��~����q����
X�WX�y1�����(J������������'�aWB�j� h�)a:�
BR��� ���q8��T���D�Li&1�i��n�8=�R��f	\�0��yUVO��$�o.�7�����M�0�:��������P���u�=p
��f��p���d��@�t&9����&�)m*��u�������eW�?��A�9��w�U]���u���*a�4�w��]O�Q�Qq�09�G���b�L��*��B�������crSBI�&e��?4��x�>B��C�i�4/5��'�
;t���v��C?�z��qo�>r�I��N��Em����;�:�z�������2n�zV�Kx����p)����k��^��������v�����K�p)�b8I��.�\����C�"�>�:9�/�\�
@�� �.c]N���\�ZtyU	��7Z���.��N�u`'���X�*��J\�J��R�+[ ���
�$r��]Y�#��H�`�yj�@�:(%�V�.�{Tr����a�8s�f���}R���k8�U���|$(��b����VS[����
qWr�e
W��T/W=o�y�x������	|e��
}�������[*����������d�y��y���"����HW�����&T4�GI��y� P/U%8�x�����@����������_���F =�?Q��qi���kYb >�+S@�+6��0@�I<��%� ��5a�8�����$y��:�2@^�!<�kV�JL9�1@���P��P����K@��T�c\��0N�����cQv��`��|s�xO>RvG#WY:��>{���BC`������W�r[�W���Z���������^.REv�\���H�w;�Wl�5�s/OI��s�j>��l����F��/����V��^�TS���|�Ee5�,�wR��0�wH��d�
�j W[&WO���w��%���E����A��W�)/���'����o��.�t6�?M���L��3(-W��&3M=�e5���*<r�1��l��w��@B���4~���6\>��uW�!��/=�NHW�E����|Ip�cO�E�?�������rhC7W���3]!��L`8��f�����VG���Q�`{�����H�x�n4�{�&k���������`Z'��N�yVI���:�N��8�
�u>�nZW*k��M�d{w���w����4�Sa�U
X�i�j�uj����:��)7��z��E���i]C������:�%�*�Q��������?�"���h}{���r��%T�/)	?{�gt���]���Ac��*YU\	���Xk�~I���w�������WF�/7i�Rx�VW�RN�}E�����]/��xt�
�*-
�����Rx�|�x!U�RR�=mh��-({���jAin�u��,(��#���'<�����b�$��2�Uo*M��I8�
GS�-M�GSg�����M�P��v�E��v�]��]��>�~�i��E!zQD�E��3v�X~X���X��MT,n�(�����s~�5����R����.����|�BS�o�"_��-�����f}OpB}��ACW�-��E�-�!�,.� �,*Qbc�luC��8Zon@�]Q������?x!�q<N�Qp#	�%�I�`�I	A��)��D	�����y��K�@�)��RsH���C���Iz������L��Wo���C�������.MM>��<�������h8�&s�Sf;w}�Ni�������a{P3,�������%�3�����p%�"�N�fy�������87��V����^�8(`�x3B*q.	x�%���"�{�1�2�	�Y���0�uLh��7ffJz;���m���M�
��X�&��pd&}3���]�|�w�h���q���c��Q���m65�I�o���=�����*���9�M%���7���9"��s�|��^�Uv�a���u���y>���"/�4�����6f]M��Lj�y�PT`��(GR���|0�?��,8&���c?�T���I��$�T�r�XC�c
�T����<����T�2>���"<aUgb�^K�����M��9����+\Z���;�?GG���d��8^���3�s�	�I1L�`�f�`x���a�3C��%��������9~�������/�/+��6x��|8;p:����cOW�.�y\�������������������� � ����f�aP< CBBL`>�} �>TaD��Nri
�����xV��2�uY�uO��qb?[��ed��J��[���veQ�4��_[��k�lpV"l2?\B��>v5j���&3F�
�������p��-O��(c�">FM����3�[����	�_����CT��������gCe�Q:*K��x6T��sm�2��tT���g�,�/JGe�a�*K���	���E�8���������(������%���	>����S�[��ST�������0}������������0��%�E��,�/L�����0��%�E��,�/L?Ge�Q:*K�;�'�(�������O���O�o�/LO`Y�_�~�~az�C
a��%�E��,�/L�����(�%�����,�/L��M�/JGe�a�*K���QY�_���@JTo@���O�o�/LG8J�����o�/LG8J������tT���#%�E��,�/LG8J������tT����pTK�/JGe�a:�Q�_��p��������}�����3�[������t����(����t����(�%���GIT)D���QY�_��p�������0�(�/LG8J���QY�_��p�������������t����(����t����(����t����0�(�/JGe�a:�Q�_�����t����0�(�/JGe�a:�Q�_�����t����H~VW��Z}����v �3'����~l���3]����\?D��v/����E��T�9�m"�������d��i���{_-�������Q&US4W[�W���ef8��C$[���}[.������a��Y_��ba�(c2�8��A��5Od�Q��h��x4n�X7F0�P�#��I���Q�����fX�G��P�$��I��Q�%'8���}�d��H(�Tg`�F�Q��V(d����t*�J4U_������^�����5�9O��x�::�~��?]�;���F�|v~b��P&�]��<�y{�����_�|�V6L�/�q3�'�7������V�K
����_|w���^������v��Y����}������>^������:�q������~���}���AK�7P�g��h�����<_��4f���^�IJ���>��-5ld`���N��'�'�Vj�L`��X����N���4�hK�!T}�c�"�X4�m�G�%��9	�S����b�=�~�6WR����0m���hxk���c����U����V��2�E�%�=.�����Er\3k<
�X�����p��a�n��/�����P�����6���L���46����L#�0\96�}��a��!:��g��k�X���Mmlu<�1c
F�A�X����K���T�>�2~;����I��'VF�A�X���������O��T�>�2~�>�3h�@ ?|Q�d�/u�Q@.x����Q����L��YZL�b7�aN������|4��4�i�#Ro��N����_��q�e��$
���hmR�M:f�QRJ�I�����s�����L0��+%M�������f����y�b�S�>(����s�3�����gn��%�P}�
^���L&.������_�*����aK�a���s0����%��>7�&����&8�)���r��iMno~y�er��:7�%S����Q6���BK/�l2�hi�����t���x���`
@1@Iq��������S����"�&�+����,��g�MQ����W�q��F�%��w�f��F�\��zK�+�ch��f����$��|��f������3��)��q���I�s��1��l�F�����f��q���A�(N9�R�5��k�n�������n��9���s���R�3��5]�����Fk�����5%w��h��<#y�Pm�E����xVT��j���	��&17G����X#�����OxF~r*b�����H�@��h�l���m�H�9&F�H���OA��������gB��%�4���������u�({!�*e/�����a��k�����G��D�_Z�
{e���0������R1��G��8�>�-�J���^K�sIC��K#��"\F�8BK���=�W�4�W;z�`:�Wq[��������=4�N���J��I�p	]N��r$�La_k� �N�l��F�'A*O�tG� �N�v6�fk�Y�t�p���D���XCPc�
�k����pA��C}�X�
�����X�"���dr*\3�zzvr�&�o���L�q|�J����LM���xC�j��x2�nd"��8�g�2��7�ql��L����I����xC'��M*�x�&��������&��t��I����xLO�>9o(��!�t29Kf���d|zv~��v�u���s����&7���h�Pi���()�������eB��&�D���m�����fa��lH��:(����^�y���P�4S��������',���C]�����t5Y�����f���5��?pNUwj�[|���� �9X��Ru�f��Sli��b��^V�{��,���o+l4���������xL����#7]�������6
g�����������G�&����&����I��������1��1pB������p����������7������4����4���xJ��������GzD��|7?"��4����4���8!M�w�#&M�w�#�I�������	�n~���	�n~���7a�=��1��<��[�#>�Mc���G<�Mc��G����5="�S�������I��`�#��&`���G<!M������4�?6?"��4�P6?b`��������������c������I�_i�#��H��9��A�� �v�2�/
�6��QpC/M�K��
�4<"�0�
�4="��>%�1=bX\��4���`'����#���������~JczD������1�G�kJczD�3\�i������HcL�����4���`�q��IcL����4��a1*G��4>b�3�tF^,�|�V�^�l��lbtkz�������4<"��>%�1=bX\��4���`'����#�5k��1��(����`Oq�OIcL����4���`g��3��#�=��?#�1=bX\�3��#�=��?'�1=bX\�s��#��4;���K�#�����7���aO��)yixD��}A�ziz��g����4<"��>%�1="�	���4���aq�'�1�G;������1,���4���`Oq�OIcL�v�k?#�1=bX\�i����������1,��i����������!�����aO��)yixD���H	!b|��g����4<"XLy��1>"XLy��1>bX\{B�,�<RB�1,�=!D��S)!D��S)!D����"�G�)��"�G�kO�#���GJ�#���GJ�#���'�������	!D���?����G�)�	!D��SB�1�)~:%/
�SB�1,�=!D��SB�������aq�	!b|D����B���aq�	!b|D����B���`1�1!������B���`1�1!������B���\WN�
E!B2�;��!��i4��+m�F��Q���eE\ZV�G��x�7�0�nw��t����AW��~��qRF����z ���EjT-x[^
�Cs|8D�
�E_�hn>��
�Rx��	0�L�S���~@4,������xy��tc83�J��M^l��������������{o%]�I��s]���X��C�Zk�P�#AEAM-�����f���~�,����s��W��j��I*��h�
��������8g�/��#W�K7�XoS(_���������Xp�kc'�vML�j�9��������-��N����N4�^���&R��<��g%XU�Qm/G[ZC|�h<.��P�g�Cm�����`O���#�\��\��E�o������,�+�S������K�Sb��~K�n&��-�4�,�t2�)�0*9�O�J�0�(���2 ���C�ys��P��2��.�fq{Xl���b3�zv��7��M^_�6�r�/���]~���v,V7�o����,�''��y��c��t2��t*k���������zJ��i��fH���	42[�3-����?�\F���/`]�����}�M����>��E�q��2���b{�~�^�7�|�����z{�o��;��6�UA��c'I7�j��~����X��p������a�G�|����� �+�>B=�X}��3��!�����M~�!n���|-�������A+�������������@
@����
x����e�/���.C0E����i�PNW$�����kK�N��h�����*��.���_��t��H��NNn��<������M�����ri�_.��7��|���c0� ���K�a�{6l��E����������pFK��y�m�2�W0�U~�'�v
����A��r����?���&�~��"��2�Vx���Tl�	U�]'���
�I��v|�R���������U,?^�_�V�~�� �`{�+��V7X���,�P>�����E��6��z�����W'8������b7���&���� �f����f_��kG��n1�kZl*�~�b�T��V�8H���>4X�v���6�"�hnZU��Mg���!��
�����q�F��4�x��YC����P��P�e���W>����1�Ee{��r����}�M; ����=��m���X��3�[<JF�h�5uZ)��-���	x�y&�
H���?C3G�>����v�*��T�7�R~B?�M�O�)i������(�{���/I�����P�������,�
#M��v��G�2+�7�P�.k��.�"���X��F����|������H��*<m���)4���TM������	�%�Qnp�D�����R�H�L�f��y49*q-U"�+�U�Z��������JCc~���1'�2*���S������$��,��H7vP]������������P�7�-����0[���_����yw�Z���������!�<+���#D�!or��s�Ky���C�����(8�P'�+J������
�G�!�m9l(%�^�}ag�9��� J�ucv9!����r��~��:��{.����Vw��"����m����^N�����V[��c��6��Hx�R%�Z&�wT�vg�V����k�U�y��y��a
w�����vY��w��w����~T�����q���g���*��s��twZ��w����.��n�
��'��ew��{F�;����v�(w�U�y������PF��������
�D���&�z��(����~J��|J[��D�Oi~����A�����R���
���h���}�)DgW��������
�0��.[�K�L��pU`K���zb>�`�����&�����;h�XRi��>��+�������&�&yT�5�T�8]03z4����N?mu����������
m�,pP_�`��Z5�N��	8��C��'X���U0X����{�����8���`���j������>�v��V��	8���,p����,p���vn����y�T1X��X�<���R�,p�N�V�����:X�����l��Y���3$�Q��s�����xL�tP��v�p����
�8]G7:�����(����`�Lz�IO0�y�M&=��'����`�Lz��t�61^�`�Lz����S}�`�������H0��w���U+����Lz�������IO0�	&=���`�s�2�P�`�s0x7����s�IO��e����'������)�����`�Lz������\���'���gU`K�I����},"@�";��7�Y����'+��*�2X��M�]GL]���������=��y��u*F���f�Mg�2����g�5�A���R]������|��$N!���7�[�;2k��N0���!����w��N0�i��x'���`����`��w��N�*�w��N���=��}+���}�n0��Q�����l]�xg�]��}+���`����
�;�`su�;�w���~;7���2���w��NZ�R[{)��;�x�O+�[��u�w��NV���p�w��P�)�j,v>�-�w��b��o��/S��m�g��oP�?�w��E�Mxd���l��
o������ �����.�OJ�L���
���G�Q��6*m��jm�t6*T_��j�Bu���M��Z�;c%]� �:S������2��=�/���4XDI��`,��ET��
Q�"�yW����"*XD�(K��"�`�,��ET��z�n,�������	Q���`��j������E�>�v��V��
Q�"��,�����,�����vn������T1XDu�o�ETdm�kk/���"*XD�ie`K{���ET�����>���E�z�d6N�����Ut�~X� �(d2E5Rq8#��	S�rl#�#����,n���_�L���H]Y�L��Q� �PL\~��w���V7]Y�����%�b��Qx�2X�k�`��Q�5J�Fi����l�F	�(��Rv�F9��k�`��/����:k�=w����5��{7X���Z��`��`��������Q�5J�F�_�k�C��8�:k�����e���Q�o�qU�(��O�����k�`����}�-����(��?��[z0�(�y	���E����#���^[��W�Q��A}~"cTH
�������[�D��v~�'8����f2�
��ze�e4�
�3R�W-?�Mo23��}�<�f3 k���H
�22���������F��@�Z�]~4�ed������
��I��_F6;��&���lH^r�D��(2���/Sm� �����~[u�A��`G��������g�_��&c�~�'�g�����]f�����\g����<�g����m�'m��T?C�y��~������:�52�=����������$
!1*Mm!��
��-�����j�j��h���k
�_�����6H��r�&*M!Jlo�0��h�z��u�����:���95�b�vu�ny�`vF�7�3�!�g:1��j����}��c�F�f�v�Kw\s���<��_���F�qb,�����"�����2�����j���D��5�]&��2I���v����j41�������jk;�:������6-����_~��mQ/�m���,��b5����z�������H����Pc���"�r���R
[��r���x�XCm�4T����E��z�?��Q��y~��f����e=)����v���.+h���&Uy;�L������t�"�_:_��-�}�$��I=<���1�]w-���I�*�a�:lR�O�Ixc�]�:c�)���������:g�wZ����A�4F�Z����3����k�zM_��E��>S=kPgg:(=���wM����y��[�z6t���z���
^������������7<��k������A��^ou}kQ����]������7M�`������QS�b��Jq.CY���j�s�6�I���E5�g���^wj0������oL����5���b�'�m���u�J�a>�n�`�P
�����	�p�)�(�}��������3��!�m��xow��
l�8z��s�|��4���<Z��Fd��������+L���~z)���f�������]����.��ZY	[�}6���APZ;��Pc'S�=�b��k��}~�f���������%����=K����>fml�������?�l����s1v��Y��J�	�+W�k#�=�����|��!p�Y������a�^�Dm-�-n����*��u]v#�-��=��� +�Y\����7'B�%���U�!���U�AK�1T���~C�f��a��Z���f*�h����!��v�b��H��!���=kG���;����~U���5���M�w-��P�p��[C����&?���b�z�I����>nQ}kH/��T�������?Q���o�����v�����7tZ&���
}�����-|C���=a���gO�Z�b�V7�k���t<>>����&[}�[�������5�F�-�Fk�T-m�y
���H�N[3�6������:�[s�`n��Zs�`n���`n���s��6 �[s�`n���s��lD0��_�������C�C���5��G����C�C��5$�[s��2;< B�gf�����>R}kH/��4@�����>nP�kI?���[T������&����c�z�M��!��:��6����b�z�I����>nQ}kH/��T����!����<�^��|����z0���p���O�'l:�l���	��>{������C�����ZX��[fe��z�s���&��#�Q��5�Q����qi�z@&�O,���@��AOaW����X���0��to=�B#�6\ ���������oH���H|��#39�P{L��

&��jL��	�K��	���L�{��`BL��	u�L������&����~UU����~T��z��~U�����`BL��������?�v�b��H��!���=kG?����A��%��nQ}kH?���T�Z�����6���������T������&��v�b���E��!��nP=kG0��PcB��{�S�qX�N��|������?���9p���O�'l����P_��
���XM�[�����r��D�_4����(��+��l���?/��Q�F�(�54��"U[����1�F��0�>����x<Ji$�	�sb��atkm���0��h#_���jm���l��u��L�ZB���wO{������j-a5��oW����kf�������`�,��/4X���E�\��.M����Z�\�i��z�\��=�O�\?�F���U:X�jK�1T=���[C�1P}���]K�1T=���[C��z�\?(�"zf��X���a�#����b�zH���P�������c�z�E��!��>nR�kI?����T���>nS}kH/���T����a������b�z�A���rB������~L��a�:��	[a�>���{�V����?���v���n��^-�-������z]�V7�z��S���XgV��Iz�-���G�����m�XC��(L��x<����O�Nz����jn�ZT	����������`��Uh���W�`��z]��z�_�`�����`��z{P�`�{��V���t��=���c�z������c��������c�z�������`�{P�mD��������G*�o
��@���Y;��b��
�w-��P�p��[C�1P}��z��~U���5��m}�����^T7�������-�o
��@�p��Y;�U/�:��'�k�����@5tZ&��P
}�����-TC���=a��g���W4������be+�����}��E���{�k��f���D�`���/�|C�(��B������-�`�����./Np>��M������M�-��<+(����/���aM����`t�w��w�����PzU,n�
���w�T���������|p�E��9��������n��.�`>w�^���7�0/>��A�@�%hq���@��5���e����w�"_�DoH��
���Q5n�m�W��v�bX�)�����d\������\��Y�1�a��U1D����q�|o	���
������&8�)���r�##��V��g��1���0��`�s���?��??��*����!��#�jj��WRc�
�{��It#���D72��\F&��Lb�d#�hG&����f��,��`}~��H?Ks���^E�v�st[I�v-���	���z���+��qK�����D?n����v���%����[}�
�j��+D���Hi�2�����������6����4��A�P��:[��<b�e�n����]Y�N%v�c;)�s^��$�yA[-Y�bR=�LO|gQ��s��\U��I�[G�a�����5���-�����K�������4�����2t�;�(����T��c*3��������b~�:'%�<�6��.���m��2�o��&�%���N����������|�0h�
�\�"��h������R�7=�D�J�w�A;��O�j��|���z1���H�ye��.�V�����D�_�G����H7�O�9��@������������zT:K���d����~[/%�s,Q��^`�� P��f>�lXO��u�Fh7tn�V�g�qC����aYN�y���s.tL��\�o��h�n����#Q>��#*�����9�A?�,�$���M�<'���#��GT�&���9��E0,0�t��������NR����oI#������I������A1���/H���ezPlh=w��a/���\��n������hU#~$w�#������t
fz����[K�j�t�NxN���!5���6S��'W5��Q%��O��wF���7��9��eu���Zy.1^t9JG��ttZ����W�5-Y~���`s�qB�c��$��m���!���:A�ce:�0�a\.?S������.�qu�����aq��+=L)E@)����o�D �a�-��gt����W�P-/������<�&%�"�o�g'�����
gd?���Fv���w���CG:^�.tz"��r��z��H�����(����{�<�Z?4[Pb?�M����vU���Q�2�U�R/j�F�J\A5�g�BK�v���^l�*���Z��.�C7�����n����,��b5�F���
^<{�,2��'~�S>��=R��xO�3�L��m_?(�r!�M�\J���;��E��Dn{�t����g���6�m�������D���[�eD�����C<o�;�@I����04�� �}BM
�� ��]�������Wv��A�u�l]"�O4���S��U�"�+qkUJ�����U���3hz�Me�L4���y������v��Jq����j'��d<�q���������nn��<�5��oS}�U�@{���q��[;�������.g�����}Tk�~=IS���b�&~k�o���~�p^���Aj,�tH�>��&��L�=�	������&*~v�m��������O��J�f�b�)����VO�{Wyw��s`����tO��J/Go��d�����J��/��g����7�i����z�[2��$������%y���AO���I�3I��4)f.���h}m�%b�Nz��q���5s�Xz!C����L��d�� �,Z:�����������.Z����E�e��|��s���������
��wE�j�V���+jF1xWt�d���+��=ql��J����b?�U���+����b��+>�}���G����b�;nh���~�p����e��w��]1�\[o���4t_��������]�����o�i�6xW�5�Z?����0xW��~���#k\���+��>��t����C��O��A������ ���w��]q��wZ��]1xW��w��]���z\���=0��.��Sq��1������+�eR=��y��U���=D����{��q�(���l��Q;��������W�9-��B[W����6�]o�M���G�������il��q��)��rS��is+�������<���f�]�-�ul��cs�n�L�5����R��K�q��#��"� �+H�m�Vp\AjF1��t�dp\AW�=����JW��d?�Up\AW��d�\A>O����G���8�� ;nh�����p����e��+��
2Xka[o���5t_p���c��
������i�6���5�Z?W���0����~��#k\p\AW�>���7���C��O��A������ �� �+��
r�� wZ��
2��� �+��
�=�z\�����0��.��S�u��1����:�\A�eR=��y|� ���%	52-8��7�[��<?~�[�����r��@�G����������
*�g������1�E�9
�#�+���U�����+���3���<��#�f S�������uY��g��;wl�����w������96w�X�V����e��4+����f�XI����YO�%cp�\2��;-?.��Qp�����%�Z���1�d���
.�K���1�d�C5�K�'������(�n�%c�
��@{��@w���tp�\2��`�k�������.�W�`�\2R��8�>�����&�S����������/�d|d�.�K����g��~����vh�p�	55w�p�w����%cp�\2��v�%�N+\2���%cp�\2����A���Y_\�����z*>C�<&\����{�K��L���=��%�z�d~dX��s�~X���p��q�/�>�(xX�|��^1+��7G�m��1xP�����t�K/�S���O��"9c
���I��r��M�b5=zo��u<*����b�������Q����I��<*���G�~���Q1xT�G�>T3xT|2��m��}��<*v��������t�
��@����b0�
F������h���Qq�	6���"E;�����jm��Xk2<�~;�a���I��G�G���Q1xT}6�G��/Mo�&w�PS�p7w{�}A:<*�����j<*��r��b��<*�����:������c`��]N���20t�c�U=q�'xT���z*���<*R�����2r��)[>�'Bp��e�r�@�wEv����'F�Xfe��
��!c���v
�R����A�I�/�`���`����N����q<������h4/|J�%��s�X=���]��.���+��a��;I����)����Z��	��������?�x�t��tbi��M��"3��.2������m�(��L����"S�Dp�\d�s^��Efp�\d����E����x�m{���C��"����~�=�w�;nh_:��.2�u��������C����L0B.2)�y�N#�Vk���Z����Cp��Y
���N�\d>����Efp����?J?�ahz;4A������A�������"3��.2wP��"s��.2����"3��.2�d����v�/. C��rZ=��{�����=�Ef_&�S����"s���2!I����[�&a��j���K&�z/=efE��F�<����1��j���f9=�{r�9���F�O�If���cv�so�.����:��.�����2���.�����2���.�����2����S��e/*|]�_���e��|]>W����G���x��.;nh�����p����e������2�Csh[o���7t_�u���k����������i�6���5�Z?_���0����~���#k\�u|]_�>��t����C��O��A������ ��.�����r��.wZ���2���.������]9�z\�����1��.��Sq���1���8s�|]�eR=��yt�.�����_?|(��7�����&+���������x��c���f�������_3��F��f��4�g#�I|]�?��!�;�>�9�g|����:{w�m���E��f��&�G}>�M�S�\v���F��6��K����R<�Q�.Kq�F��-u'�'y��IN�1�6���s��~��Gv���9�����|w|��_21O�NN��GQr-�/������K�0����a��[��S�t������k^L�H�{�F�K�e���7
��_�2�|�|����`�M]������7O&�[��y{�����wo��|o�����c��H_�����G�eW<. B�
p	���R.�p����O}SRF
���55� t���p�.��R�l����x�7���������a�F�|#;�vC����	F��hC^UD����b��x G��:)�.L���B�b��D�*q�*��J��x�A<J��K�H���c�C���AV�:�=j����'��qo%���c�v�6R��M=��V�l�����Q��������n(����j�j���n��:Y���&*L�#���<6�_^F�
��\�e����~}{���r���z���d�:+��m`#u���77Q����������]�J�����JX�gQ��J�-w5	n�X-�X��Q�����gh���Q'.4�)%EfZR��
2E��:1	M��g7[7��h$��J�}��v���^j��7R:]����S������3vy������
z����������M>��8���o5��r��n2G�z���:��F��eB�|��0����1��K>g4�x���AWF�����.WF���qn����__ao�B�
��`z�U�,E6�\�$sFB����z�O�D�M�c,�h���{:�s��`������0��������*K�M�6��c�D���*,�0H�
��8�	�G�}���������5OP��`P���������y���C����<��������
�S��������O����	��q�
���O�����c]7�����7AS��/P���������o"��(tT�>R�'%�q���N�	��	�����/��|�o/8�l,q7!��DO�~r@�f�<\DFx�m��[��w���1���Xo�H�!j6�@3�D"K�eC��	^�y��
�D���jZ������_��\Y�P�RD:��q��v��n�sCn�J0�6��L������4�����k��������&5�l��se5,�:�q�<�3'��;~=�3D���b0���K��	�7�������`bE��6�AW��vS�/�ks����W�q��F�%����|�C�����U���"f��h���m��e�j��!������A����D�s?Nt�5�|���:&3���H;]u��L�8�v��G��q"P+K��o���A?��	�=R�Z���-eY7�#�����yMH^Sr7�[��������ns-�������h���;��p����,ar+��Q����7�������7���h�\���ILI�E<9{:�q��&�<�D�|��S9�t'r���x�*AO���
�S������V"N�(xOa0�z�^�0����,���@~����z�F\����(��S��xw�������%9p5-��2N���'�*t��{����h��N�O����
�!]?\e7��n>^A�����v~�_C�{,_!������q����������~�8A9�e@�wz:��x6�w���$=��8MN��q���5N����E�]t�C��6Q�����a�.�LpU���*z���||3M��I��%7���|v=�%�mv�������<�������}��*��?���y[<���)� }�-� �3���k����6�|���ts}2_��e�=����=�E������:N�����N��i:���G>��)���/.���(�������`nF�����O���O1�Dxz�>�/��r�o�O��������3�����A�����Pz�o?��*�~^G�|I��#����l�]~���>��s�~X�'��������[\B���v
����A�o"�i���!<h�~u�
 CA%gL�K���9ApVCh Y<���g�m���C�����'���?�F��9z��?P�A/�������sX�5�C,3'�	W�,��8@�h��^��Zfq���5��%W`�/�q��EI��a��{zw���V�n�>�KG`"���Q�9�������������%_����C����5hTJ��d�F�>� ,�q�;�Evs����2�y������7�9t��0���=(��|��r�������6��0V��'Qt��X�pl��� C��Qt��EQ3��%��a~�a��<�s��@�[p���EsM���T0��a�]P�����Ar���?����D@!�T@�k��v��o`���q���M���/����U�������QTK4a������5�I8�����F<Lt�p��R$�N5fu���6Q�����F���:/V��T��<K20-���D����v4��
�����p�`�L�	g\\Y�	���$P��V��shgb�f"�4fy��Eb��	F5��)��
tr�3d^��G]�+Vi�}^��P<dK�LQc��~�i84n��Qb1���s^�������M~��w8-���/o4���}F�14��aL�E�U�6n�y:��:���~
���b�7���/dRC��"���b������(���O��p� Cz��\Z0�-��aB��K�5@�m��o-0'~N,0R'��h�������1Mq�bF�:�����^���o���}G�������[.x��t]����$B����d
d������.�3�-�oh���7��F�D���P���R�G���}G��0v���r#Qk��V�rq���A �����
��PUL��p�\�WT�������~04����o��{�	�_lt�
����1������p
�����O��B������H���������w:�5M����r��O�)-Z���n�E�U��Z1��QEg7�;n��Ep����A�����[���N��HMs��#���w�����+�s�J�>a�@W(@=X!�~(�����pJxhC
f'�#h�{���n��l���].��fC�vA=�;�]��S�|@(��,��,���3�d��,J��
������!��h �H�d�@�q\
	���9�0EK�J��cmK��
t!�M�
��XV��SH�dc��������1)lJ�3Rn};�]=�8Fe�9"��q�3M)Yi�����n�l�� �K�]*o��	ss�9�cs�1�z�@��V�2s�}�����5�B�#7���a�s���PD�V0
�CTS��5�������uq����Z��/C�z�E7��V�E�35���:�����B��}
v���y��� ���S�L
�l�9ny�E�n�' B
t�gXK\U'���
�����u:���1��p5��xFb|�*
at��	�.-i����K��E��h�m�~�.����0-�t������Y0?����(h"�����f�p����l!�
	C�������=e������b04}��3L�@9��*#81�}4�m�E�=m�Pi�]KI�+M��7@�=���P����0� 
O6 E�i��7QF�	)�M����b��iPv<*�L� �^�D��h��v�u����O�P�}�)<��@���1��@�'�X�.�o��`��
�>]P��*�R'�U��E�P�@C��k��Ky1z!�A����B�#�5�l�
��h �� T!��0�%� � �"����1������_/?�?
���Q��}���W7_��m�\��M�y����x69�^T]b~��[��w�9n��&|qr��y~�86}��],����V�N���h@� ���&��iZ��!���we�uJ��0���r���>���`��r���4`���&�P��M����d
��2�	�G�<���?�C�����
L����(?�����''7���tr����dG���?�����������+tK#�R��y��XmO'H2�
rpy�p�p���p�m��������L���`;�}X�G��������}�F�#�E����E�zX.����P2��a1�U���l9,�/������+�����5�YYC\��������-�����Ao7�W��� ]ZW+���bx��V��b�������j/�|��g~����Y��+�����3��r��$0wM����1�Z
���|���)�h�+�Y��^��`i(7X��+_�T���{P������Y�w\}Z���:|��E`WH&���+�[��G~���/I�����!mV�!��tT�
'��C=���_�~���MJ ��@|W��$"���� Y����F�	�|C!��k|z�
r�������3�xy P~l "��d��I"}�Z]��D�p���wZ��U�@������B�s-�@��B��pU��,wV�c�h�J���\������P?\�6�-���Y�7U}�g!��k���
0!!Hft�q_��B�������
����_�4~�Y��������Snj�
bCx��'KM����a/�G���d����1�TyA��M�k<B_�-7�8����I�#C�9l���u�Y���N�
d�#��������^y/�&��]9M��A/��� ��^�9�x�R��"7�[F���}�bJ��d<,_z:�����)bL@������������7�A����w?�?�9=��^^��������}=��?�+���ct��yX>�����>��o`���@^D_E����|�^�ch��ONv��wf��D�}F
|���()���%f���}�@��m�����b�#���l�K�2���oT���CyQia�A�z�����B��A\7��l[~���.�0���+
�s�=����BF5��Tu,���Q~L�J�W�(���������'�[�*��n��<��t�1�Z���K?�� ���-��������nBJ��Q5X*x��'&�|)%�G(SD��@��+9���J�g��#Yg����iO��tw8?|�N�����y�EP%k'8=��5��Pv�Zi4dZGtj�F����B|(���$�A���}��"���|��`��E(��<$����5p�b%���[N�
VU�	�po�ITO��]�e�B�:��C8�����/h����_P�,���z���|��l��z��E�S�Xdz��JR?��T�]Ine�����D�'/	��x������T�lp��������QY�=����o�b	��z��p��"�[A�s|����9��6���e�Y�lFN7��a���;h/u�L���_"������Y)/��������7��LA�.T�|��0|����w��?2�c>�L�O�x��Ge����3~��o�7�`�����R���6R��	�l���-�����������?�������@�l4n�hEB�����bKB���_��//6�5s^�����k�^~�#a�$d�_�K�F�F.k�E�~S.����T�Jzc�NT��Y�����x��U]�J.�=���'aDp���
��)��;%N�#>��E��bO��{4��?/�X	%4F�B�b��>l��L� ���o����tP+��?��	R��PS9����:��D���<��|+���t��a���@�Kf������uM` ��DW� ,����D��T!�����V���\y�a!2��2�k���	�}�'�QRB��cL?��=��
�$!����t���G����"=�GeY��>-�1Z���S�\p�����0,�p��Y��"���V#���@m\
T��_�p�����i��Z8��o���:z���\DYOg+F��E�>��0��u��!��T���k.'��(_���� X�`,Em�0�?��m�!���Y!���b�$�TjH���}=:�eY����E#�G��S����u��X�q�����g`����"Jx����&SP3�6v(v��R�L��HY�&��%)���*E��&��$���;��| *^�)�0u����Z:#�,���a��?�����i�����at���2F�]���o0T����%����
��M]9�������7�&9$H*��i`�"�.� ��S\<��3:*��=�|��m�T����a�96����:��z1����EA>t�F\�J���;h�)�H<�������M����23"�2���6t�6�9��|
i��p�)���
{�S���dj��P��~G���CN;�jA���q"<2��F+�(�0C]�K�������A��!��o��]�`�H�I���+n)t���i��L+ZE�8/2 ��i1����i�J�J������H}fs�i��D�%��C.�;~��pn���93>�������H�?XGp��;'�{g���gl �K^�������]w�����}�Y�cM���-O��R�bN�M	�Q��U������mh�6Z��xp�
�
�Z�mp�I�O�I�	V1������;�B��3��:r�Yr
�w�PsH��o"h@����j�u��,��0�w�3��H@�uRu�)��������aw��%�,
��*�-�]����v_ef�27z�_�|�.�4��/���`g�$|9�b�v��K���&��
��������"C
K�bs�aIt���q��Y��o!G�@)~&���MS������i�?��d���C�_�}l�������?�(Oz�$&��,$e#��;�o�����AJ����r�q.����c>
��:��s��1b�V���C3!����K}�����j
��KV}�x�S�������uj�06N���R����������T�����%�#\����9rjQ�7���hD���Q�}X�
����cL�;7��~�y���k5T�������b	�V@�����U�(��le��f������f�<���%�'��}�����)�i���tTxC�G�HW\�����
r��R6�a� :a�sA0����(� ��%���P?��F�^	:���E��#�����		8��P� 7^�"� 6�Da�|�rR�W�;�#s�jW@�l0o�A9
A]���/�~�1s��+��l����"����E�"��pN�����_��V��<;-.�<����PF$�t%�����o1�f��!U�,?)lmRC��*JXv~�1�K��������m�J���o��wr����	���u������h}`_����(zX-��iZ
>04f�'�m��Kehr��x����4UU�����f��
c�_�J������bi����,��s��Z:�AZ����:��2�4�������GZ'd��Y}������tij��8��0��,�3�����UG�b'M-�T5n`��n�Y6b���	�����!V���3gs��Q=�Nt�U�6V����9��m�j>��5h�a�FAtn2p��"R�*���3���y��|p.H���sYp������������8R�S���h	P�+,���=�A�Q�<R+�~�B"f�j�L@h{�m�	
M���MF�gm���6k�fBs�b�L"69eMJuu�`�pk���
[�k����x��#Z�����m4�.�?|K�B�h�U��Id�z5�Z���GI@�I���2?Z��h0�ebe����;�
8G�E4. ����)ho��?6m�Q?+ p��	��D�����	G���3����\{FI[	k�ru��s�14��j�B������-�������y|Z@��>C���G�H���!"+6^�����=�\=��^�s��;�WU9��G����Q�9����t����n�!��K���|���-�#����`�;]���o1���d	��XGHC�W�
;6F1n8R�������{����:�d/��%�A�q�����s�@��[����8������-�j�h�%����:T������_Z��l������9<e�G���M����B"�YQ�'�j��]KIcU�>U���l����EVG�"�U�Eo ���|/'3�gM��l�Ba��S��XY�#L�t�l{@��(U��;��K��nrij��np���`�5^��� �=�n��������f�%�Jn0���$)���9����,��CZ�#�.�=Z'��@cq��������0��v����r��y���hY�\����h����H���
J�#U�gV�L�.������!��l�-:���?"���,?#�^2�SH,,�0�r�EV��y|�9�����N���4�	�k��<�7�O�K�#\Q�J>���T�"�^�6gr;���H����&��\E�[��lE�2.��y,���X(giNEc�t �B���ZMD�2����`j��1�?�V����W��hn�b��
��B{Q��%%
�zC
�'r�����^��L0��|M4����C���&��Z���z���u���Oc��T��2�=��!��x�#P:DL����-L<���@h��������S�R�>u)U�|�oi.8B��,T'�x����O������/�����,	Z�U��\I�S`~	Wk��F<�c�oD���Gx�!9�E`Wy���b}�-�eX��d���L�����_�!<��\�1�XqFLf|��a����W��e�� ��.���[(+��p"�CbA?�<�� �c�8���
c���d�+��������~���P�����Ig���C�DO*���?�e�m�1���Ks[�Ni����wh(e�g�0��C��Nh&)�?s�lT����E}cx0�
TXrt[(`�| 1��L���4�j���s��7��l��6p�b*.d
/�5�!�W%8:�T�R���/���r����R��5�h��T��|���r���D=��R�����m�&X�DV��V�b��/��(�:!�DC(e@	�Hk��Nx��^s�$�Ts���.S��L�jN�n����{����rf�`\6+,`[�a�at�'�i��#D�hB�\���e����3��i|�A�2�3z�+�>�������2� �7xc�21�e�m��@$��x���C����/J�9��D{y	�������(��M���.J���Q�N�7y}� ��!k�aE��dDb ��u�\n����z��<��*^s
�G�/��]�Pla8>Q)N&^����x���<SlG�������(�XS��T���,���U�4X�\��&g��3���E�{Y�M\]�y�������h��J$���L0�y8�$#ut���0h6(�X��Fg4�#\���+p���X`�m�^A�>�I�*��A��[����1m�x��U��eK�����8��Z��M��L��9W���R�?����4�J�M)��#�g�"�!�1���<2�~�Y�\6�8u��>�d���.������^��3��> B��,���LQq�\��'�O�W�5�K��YH"�����.���T5b���<p�l��C6�*����q�tC�*9)O�M��+�H&Ukb��a�,���m�f-��C���1��j�����&_n������	^��@�$ZF�}����#Y!�2$��`4���"��P$`�|2=���1�tV��G�	CK��4�hC�������NW�t�����S��u�#�B���}
���ky1��Q��t���:��'�1��o�cY�9[�!~0�"��I�	���)�^���S.l���*K�w@����m;^)}�|��;^�}����#
u(�<c/K3I�����|���
��mc��E �0���`%e��Q@�g�'��B���N������4p�1�F�������� �)�=p��i���q�j����IQA�����2�+3ag[(?�B!��e�;}�e&�C$�
���`V�E$��T�����<Y�4}�-e%�[����1�-�c�Dyd:+@��U@�+���Ne�����`�B�m��@:����S<$�=��8e0���DT�G����*��k�8���t	�"������_����%!�] i����:	�*��H���%��
�	O��j��]<*��%<�I����x��&��\f�3����������Q'��IN����G��T�� :<Ef<
���hu�G5D�Y�P#�~�3B�����N+)�S�����b	$/0�}��B����x��U:O�$�Dfv��h�����,����sB�ki$H���� A��	�+\��r�dv���"s�$�1����N+�+-�M�;a,U�6���-�7�����O_�Fa�Pg6Y�)�ZE�N�Y�j6�����d�)�i�cE+�eKM�!���V���:�VS6����,�/���0
�K��0F>����9*���l"���s�x�����zk�X�Rt�J�\�|���R s��	�4
j��+x�X��T<�3�������G�=�Wm��8i�w��v�����4
�,hoJ��������c���� ��OU`U]�G������ �{(_�#5���%���,����
:nDV�rbL�(�P�>���	e������yd����:��jV �u�4q��n\1L��c@Nf>�U�S+����U���V^F�m�d����TFT�&�Vc� |�l����)���%�����eE���b-XT,,���d{������uUv����*1<I��Q��A3M��:���b�C���R���M������B=����*�b�"_8�9�#���"�@�`L �f�.
�:���bm4�X�M eI�	��h�0�@,�M e��*��i�=4������%�sS
 2z��Pg�0*����+�
,+w#�^��)GvP��I4�_g��bB���.�TBwq����RRJ��v
}VO�5��:�{W��v!L�S0$����Q��L2��pv�m&Y��x��u��;�|S�-��/X�U��,����F{�����r�
3���V�����d�/���	���cC��f��cBp�'���R�����������+WI��KR���O+e�!O�s*[��/����@���?s#-�P�<?c]���X���|"iHgJ��Dy^O;V�/�,�*�?�u���������!�$$t�����xrA�������,��$��<�TL*����#��c�}�� ����jL�[/����M�9�/Y�(��H��+Z���H���;t���pC���;���������C���7Y!��(�_�/�(a<_��LG&?J�2I��N�t!8ez-;zAuL�v��3��c����	�U��e"4"��<��w�s�@��E��<4��1DX���;�"�����������n�����/���������U��k�T)� ]<Z.:�zr�<���l�L��.aCUB7
���j������_����!tV!����Z{�Zy��;hW�k��
/�7}���a_mA5h�f����4"M���1�j����P7�b�P�k18i�����'����f�:\�6�E��gR�k�P��
��k��tf�0���n0�"���Y�,JO�T�L�A*~*��i�V���b���%eQ2�-����4{CU"O%(��QmxEh]�`V<	}�-�p����;�q�>l���f
Q�?�������Kkj.E~�K�Is	<����Z5�G��%��AJ9tG���n���MyF��b��#U�
�X��bAU

�
��>"�'�
�������Qh-*c�@���Mu���S����8����68NmPF(����x��X���0���#0$����M���D9E.���4�-

��H
P��~���	:��b�-����NI������rA�~���q���4��]��Y~A
"�"�0�N��yy�����J��g��t��k�JI�_/����������>�9��O�[,��v�q�Q���O�,����84zIiYie����&[�i_�M�TnF����������X��(_N�a%F��.+��p�/e���As�CxB�qwdq��y������B���nE��X���sy�R�A�v����>���bF�e&,�����E5J�������),H'��6����U��{���"���6��[����E������/����*�����GX�Z��>b_,H��qa���SC�f�����ya�>KH��PD�&$k�2N��V���|���2I��.�9�����?�������Ylh��	��pj"8��p�E
l�B���CjI��7(&6i-BLT���Sv4�QpQCN���pE�/��,!8�IQ	�V�V:�����W��p�����Er�P�#��C�{-����B���9>`��2��`�F����
�hr�3���xR,���R��cN��Q���;��6t��O���1G .xb]��_q�F�)���@�7P(T�?!=�r���n�C
�/��6	�'e|
�q���%���{�>%o5�b"-/8""�|k���r\JZ�E,QEc���6����KL��F�������9
="�j��U�nr��F+n��E�F����^a�T5t�@s����@�dn��"8^r8�-��DEq	��L����y0~B�`�!~�H�/2������fW�~����m�T���#N=�	���V�*'����q*��D��Q�<��/��:�jQ;���XY7��v���9�Z�?���w�����^"����Z�4��A>����b��iyFl�C_`UJ��}�����.D��4(��,~��&#8FRt���"<�VDk��"�4��g:���M}�VH\C+�T��vWIT��]�|�qeGZe����D#?9v�G�]�~"9�����J���Y�^�5����?@�����)��1�H^%��m]�+�!u$�l�[�@��%?1��<�pb���O(��o���/�A/����?����%��9�"��J���<I���V�<X"�tQ-������O�8�+�GLf �=���[�j]�N3��IlYo!N������*���N��M��Q^���e�X�g�������u.;�5����?�yS�&���Yy���q��?Yy�@���������r�������j����K�7,:&�b�2�9TO���W�%&��[��t��|��*��T���r���j4s�1������N������a5Rx�K"�4+�*J�J��?��Uez��+I���B)(�L,J&'��u1DtQ�"vQ���wA|d�>���Vr)�u������ .Y"��C���t];��[��D�>p
��j���		Nc�,��W6f�%��&�if����%��4C����)��G�F��yUPn���t��p���d�B�VQCt@l`���Yt�H���Yl1�*eI7HNJ*�Ys]�C�zC��
:#�-�������Y�Km�6�����5�~^f��8�L�����{����$Qx����w��1^L���dGI��g;��9��{��IH��"5$e�v=�����h��L9Db�����KWWa�i���#)��K�a(C���+G� R4o=(�u�����5��zy�ZE�?�T�K7m�n�T%c��4��*���c�g%��x��'�dhS!Y6�YD<��K��dl��dI*U�3������G4H�m�t	mQ�>}B�BUmZc���w

	�8�����MpC|�1x�I��:��Q��l&��^��V���?�0k����#Pr���]�O������z�aB�����C{?�h������b�g�p�4�\�{�sp��+������J�[��+"V1����M1�/���o/���w�������=���M�D�%�2��h��j��kk;\\�!�ab���%cZ2V6���E��s�E��g�K��B2�����6���}PdKH�����=���%r���=�CH,\���F�&DET��~��D�v�=(��e���2L�X`�O���_�w@6Q�j����U�hK�@���s�{Du����Lo0�-�����U��f��v���MJ}4p�{��DG���;��F"mP�Z��UG���xn���x��`�
�)����xlk��'�+���*h
�������5��|"|siQ�������F>H�sb��o��'�k��������d�
#���a<*�AnE|uvW�0+�J���}�d\����c�r���1��1`g3�Cl���j�- gu�-�EMWN	�m'S�k�\4qy=��90F4+G��W���k��e� �Z�|F��0g��L@b��A$	�E��8�h��f7�I=YDnj�a��15��#����T}H,�%�f��H��^��7�f���H�P+	�o8,��w��y�&����2LzL��Bp������h�.7�$:22PNH�T�V6�%���Q_������L�^�}�`���m�{������c�]��t��/����`/��=�{��B	D�-Q�c���(AX�m
@�=	��9rI�n����f�Q�}4�����;$��@
�����}TM=��>�&5e�kH��_,W��~�@L�m�� �#{*�:�+�r~3����_J�-9;���J#0{��%n��m(9K^8�g�P����fG�����Nr�_2
���9#{LMA��86}M�[�U�0��.��F��U��Jfm��;�y��MY(J����kfQgpn�)o`�L��6 
�B��6]���oT~c�����dg
��H�t��-����T~�E��m�5�'�B�A
���b��!��,a�P��\��{U�������r��{���Wx�X�&�� a��fk��&�T�P�g������p-Y��SX��huFf��MI�F���`A �	bV'��u6���aH�X��mg�	}��y����*I~h�y$?��MA
^r�X��".9-�CL����*/f�<������49�Up�bv��K�DE�a_iZ?�v�9 ��&���	_|LI[q�q���,�;������(���������(ls�����dV
`��B8���x,�\��0<�]����?s�
�e{3�2���^����Z���=_@�nW8�9LH3�����b�����.�\s�<��)��muS�����=��))����n1��/�������9b�(���l-������b����,��Z������$G�Z���(��ZI��v����[+IM�~��c��TT#^�
�G1�q�u�@��1\	
�%Y�:�+
����	�j]�"�l��<��_�����v�����V�n���3�<��6V����h���h��v�2�{�E�N����Z��k�F#��?�	��Ig��c��_���-'��F�R]�
C_��K�lKoy|'�G���YCtJ��
M��6���9�d"lC��������]�����������wRC�[k���H�^�B{|�u��I;Z���h$Uv�-�o�g������n��[�������]��j�U:[�!@�1���r����wd[�d��|����9�gX_M��	�'Da���n��'��7��YD�����q3��La6QAF3'�����H��?�8]gSH����n}�,,�$�/�#�'�(/����n5Gi
9�'����`���-�����x7w�
���R��d�dG�	T
f��o�)	]����r;'<Gp���s����t����o�v
9:����w�2,���b����5�6�����"��nk
.1?Fg�_�G?_����l��9i�u��$q3!+N2l���Xq(Q���P^�o����,����X�I}��B�w]h�1�����q�Cz��K������<���A,�M�2%b��8��u�=0���� 'T�sf�`H,#a+�Y]��)g���&��lg��N���v��q�U2kQ:�Z�p�7MBr<

=�!O���W��D�����8���QS�_���\�s0�w�������y�����	����tdqY\J�0J����e�0<�����������MF�����;����]����<���?��k���^9K�XZ9c��t��X���UWQM�k}c��������V�-���REC�*Z2���%>�*�8A�{����� ED�p���#�X�5o;���� �
����IF��rJ�T��^u�S����=o��C������"�
1JT��c�t��	I	O�a��0���,��U�G���GpJ�E	9����dtRy��"��4&�����e{]�O�YH�K���V!���@T`P2	�W���9��%O1�����X(=u]GTA��;~�G�b�z�O�T�G���o��o(�1����2��O���1h2�pj�,�28�+B%��*~y�}/F�di��xb�?�|P�J��;�J�S	�6����������t1'%�����9yng�����^����n'iF�� ���5~�?�Sz�$�n=P����P���W�������j[�[m ���f���|d�������C��p|���u���7��l
�D��=�����I�������F($>���f�^��i�{��\O!8'�#zBBu��0�N�@.6T�ba���D�o���6�q��6m���%�K�~���m��GK�~ ��;<����U�HT ��3~����K)
Q�Y�x����5�X�A�f@Dktw�6���5Y=<.o��"��<I�z�����n���d&��
�I����(K�m�M��h��On��w��?��:"��N1S����^���!^�0k>��b��gY8�����U�Z��] \�v����������s�^�Y�;q]����i:��;!�jP���(JV����p
��"XL �r6�J�����'@`�7������N�PI������5���	��=r��t"��P�S��8^�����-��	��m0W������l�:���bP	���-����e��WS��4	���*���
�S<s��4+�SQ���p���v1��{��4���T���(��>�dO�'�E�r��D�+,��%�&���TT�f4
�}����y�2z&r��%q-Z�~�C�$87!�q~��O�d��f�t6W�j�Y4�kK��r�DM
*��$H�-_	 <��zB_���1��0�r�\����@[���O�8J@�i�/��N2�E�)��L��ze����@�B�)*��n�,�W�$)?�����a^%�M#e���)�<����m�wn3�RI�����C���������=ZZ� ����C��J[���X�p��":(���v�����d-,G��{-�7qO ���GO�pKoiN�S�����q����nZ�������7#�j��f��9E�^����Ft���#h�p��<
��`8�G^�(\e�}�P��P77@���O{���1e2������|����ZD8�������0�q������3���mBdm���� t���+�]yX���g$�o��w3���"���:��	g���8�2���-�9�E�/��O������0[���vE�K��a�m����T��{7�� �p�5KBl9L��2�S����6j���i��*)
d��>���W�����p�S����i��L�v;>�v&�Y2�#�J�b�?=��bX��!\�`u,��}�v��������_M�~�r?�b��0>*,�5�A�T�t��_(v
NQ�X�*�2��q:�����qA��j)�\e���%���+�*�%-�� O�QS$�B���R���'Yr$���8j���hYbr�S3�ib�{�iPM'!�;����"�m���r�$�&�����&�YC�������W�&bQ#�����@)Z?��Y�f>fT����M����r:�F~*��$f!��8��vVC��qp�_�_���)?���+�~���������}��ZC>��8����n����n<�������}�0_����������7��$`n/�''1�:�A����E6b�O%�m��&{A0�����c�
Q_��Qix9$��&",%!�������>����@_71�h�Z��O��Z�w�K�n�F���hzK��WB��g�N'n�O�JHt�����������[<m|k|��K�b
��hzX����g4�u�O������$�w�(�z��$Q2H~�:1w�K��]����,���rE������i���x0O;���2;�gI�Jc,�L����������w�-�_;������������6�h�����=���au����t��,o��a�����w�C���l��ECu�w���5:��S{w7�[6�<G������~D���/H�&���*��v�p\�$n^�hw�+L\�j�y�[��@s�n �!
�;�7_@(K�j�F�Nk�F$����}������-%
�6�O����������������i6��]9"m�t/o��/z�����"�)��"�D�4+�d���r�{h�.���.=H@=����Fb�g�T{�&�����P����Y���f���n����~��6n��[��
���l���m6^���eM��7�S���x<�i�q���~>]��]��|�Z�Gc]�T����I�V�6]�&N�S��lS����x���$����/������{�� ��a�s�@>���#�n�!5�O���9�/H-�aa�C&"����$J��d���#%�F5�8��*�?�d�b�\y
Z:%m.}{�6,��
�1�����%���{r�E0���
&�6�,�C�"�Bb_}\�n���^�yk(�8F6�1���OX��n]>��
�0���z9�ax1��9��n�k&]�P`�$g}��O�XU�g�F��|~F�R�IY�k���;G�v:���7C*<?����44{x����������)�.`9���k��Y;�M����P�������m�\ �kPg���~����������{'t����M+��u��&�7�����bI���]�'�@�F�l�:�C�X
��|��h��W� "��8�|������4��y�0n}�\'�]�1��+DZ�>k��_����,��G�qc%�b�Z�*+�M���w���	
j
���������O[����3�I���P���77��BS^ z���~��>8�:�"T=�G�liI��'O��fU(�h�_8������Mv#�I������H�RI�V]�����'7Y�9�kml�T�����?�����n��������U����|�k�s���
Z��-r1���7��OG���U��|$����!��
ec�)�W�Z�xKrF/�w7XRY-I/u�bR[����j�[����6�{}R^m�c�>-���bcP������UF���$���eI��x���Q��1�����<p,��Q�b�2�%�'w��O1����Y�����9H�$����H
�<t"����sq{��'�����
���jbY�f���S���n�E�i���B�O�%�Y��:���p�M����4^v.O�m���?�A���0o�Oz����pg���jLXT
k�i�!]�$3_��}>jW��=#��A^��|���\��]�ImSv&���i/l��6^P+oq���y:���s�%��2L�1���,]����~D���*������N�}e�k�l�0O�F�j���h����4��2��R�A�0tg���W�CZ%swx�hTR�f��K����)����6�4��&��F������V�<#*�������}�QJ��i]#��80��n}�7ib����������
��Q�z���4b����������D
��'X�)k�z�Y*K��_����*�Gva7�b*^}�<�IkWK���-j;�C(�v������CH���!j�'�
�G%-r-����1�X}4[`�E�t4��t���;�/��P �
�D���bD�[�4����$��H,Iv��	 J�Q�fy;��!�6�e2�f���%wW$��KF�����B��\�4��e��{�<mg��`��F���-�&qg��D {�O 1�6K����K�������
�4IVI��ur���`��������M��:�5qO��X%i����O�^u���:"B(^6��v��{�����:���� 1�;�=���Rr/qe	�s�
�K��8��&@��1�`���5W|��l��E�������f74.*IP5Q���l�C��?�{4��F�W�s���3.fl�6}U�/�����N�3nz�>�jB�'�q����R����)c��K�b@5��I"��h�=�j{����P,��'�$�B�����v
=�������
aM��$z.�`�����7_.�y��2��H�����Y��d�^��F��fx���<���ZK���$��u�}��w$��"!��5��Ld�;?�m�Nd�M�E������ d�9��+E�+Tv�Ek����
���[$��m�I��6xH�1����K���l��3��PVa���

,��~�^�v���,bQ?puOA��]�x)�g��a��O��jM���0)��� ��3���	1tS�r�Pv9�E9�BD��-0����1��.�UmB���4��C����������I�L��asoVW��/��Ev�{���t�����d�%�=x�d�K�}��R[�b�R� N�����f��M���/z�K����TE��c�X$h��������)^_#}*��5�J~j(�P}���?��{�~�[x\[�VD���8��iI��iq���+�������@������{��y����z��d����8�&��xp:L��pE��~/�t{��������O����]��O���n��<��(zN�~G�;o�Z�"���w7x�/����X���	��&����W�l�]�����C�sM��
���Pr����F����s����	�d�`x����g�����h�<�3�R���~����t�nN�i'�������>���6�h���GN��;���8��G
 ��Dk�c����{8��f���1(RC�]���?G���}}�n!�C=��&��	�OH�y"s(�o��H��z
�ct�Q
�}I�|{�����?��C//�����Wo����-���h7gt���Pq�%	���08��N�A�����7��/��Wo�=��������G����
�=�R� ����/����B���&a����^PNOO�������B`[
��4��;>$
n���-<M4(h��7���'�1���?�~,C�r�Z.����r��\��K4���u�r]
�������v��G!����Hv��`a|X�L�l�7�lV��m|O����?�{6n��w�"��E���.-��N2�]D��O����Pbs����� ~�hcA�f�m�!��E����H����Z����G��?�1��A���4@�\�/��b�D-�h��������B�1�����Q�{��d�N���^��Nn�C)F�
S�&��X���8I��r)S	�>ss�]n����5�Ra0�w�y�f�'�J�j~M��LZTC������WoP�����������6Q;m��	������N<Y�7�P�O'^��/�4������c�^|~�	�`�A�������_n���������5\�Bj��:}8���w���������9
r��
B4�VMV
7�J9�V3�D�|�\L��:��	�kL������r��v[��.���f��q�)tA"r�&���������^��X��C@������"6~�#H?�������J��������#�1	�'cZx'#�������Rf$[�8"9�N�����a=��ry�y����`�^�WDE}G���E/�!��B!�	gs��\�}�c�1������u�Ega�e&�K��������M(zP^�{������XbC�`?9tpA�����?�z���?������?��b�C��H��8��*-?kZ+�FN��-i�v�?�,����C���#x��	?T:E�J�[���/x��?���}���	����)b<H4�bi,���&��c��H*��0��F�� P�*~	�;Jcy����d��d�����1<�g;���VIJC�awJ5�C^��#v�k
�f����R�\����~�l�`h��z��������?�Y��+&~w��Ayv��z�~�����4��cY>�������2I�C 7����Od�.-��0��LW�����x/���,���{f=�
����T��C(�n`(�>�i�DG������"VN �Eo���Hy��$���O�1�
}*��d�{�t"sB���w�)���IE��6�i���+�)S���m����Q�����r}�=�V���O�����;���(�!��=rm��8��*��[�&a��A3�AH�n�����P}]������d�����}r_	��<C�I�y/8�\�k}��pDdL2� u���G�����g�Rp
b��)g�f�D`��C�,��i��x���W������	ramt���Mj:�Q��Eu��|����*5	�������J�S
�����}z�^�����y1����J��q���1����H=���$���a>I�;��N����w���=��g"u��t�[d����5�Oav��������*h?T��~��o���z��dB�2������u���K���GC��&(�Y<����(^4�U���L>�+�rJz�����@<�z�D�+�L�c2f��,�	2����)\�p"���+�Xl�uP���:��!�=��hB�F����XyJs_eF�~�O����=��2�D$����i����i@7%��)��{1Vy�������#���0�85p�K�|W�������sz�%P�'�����P���U
a@�z	�T���_���x5��2���	e��.
��}�[3�� �9�5]VSD5 �8��t:hF���c�?ED��#M�~��F[��B��{Bx�����9��E!�����FD�����Z�%�����~v���/Sr62oHL>��`�u����!!H�����I�ivC
t�ayy�F�0���i�	vQ�������*������7�X�+���B�d{�*&��v7��L���4Pq7�ec�lL��zY������^�K�t5���<��S�F�G�A}������;�~G�����}V>���+���#KS���Dh������J���L�~_N�v_�u�W��eZ���*X�!:��l�a;2+0��W�}s�����ja�l�2	�4��V��
>;'/���*}��c�\]��y1V��]������[N���
'g#�3������G�2�������wJ�o�~�(stUz`�����2���|������-��m^�9
R�#H��rL��L��������
��b�7�i��b/����e7�V�y���-7��_YK�!���/�@~w��v��8���t����]W���;�������d�^����I�l���az���7[
7% �1'l�-�j��?�?�18csp$��R\d��g�����g�����Z�r�u1�6���j�]��{Z��jyw�VP�8S�7�.���|6/ g�������CW����M|��S��0�LJw�l}�Rs��1�LG!�{te	�����
TE�v�qy���H!�7E����l4Fz�(��@���6j
���p�S������T�Oe���<�j*VP��T sV�p=D��
�*�H|[2=��L����,z��t�b�Q�V5>��\��x��8$e��w0(1�X�C����nq%|P���	�%>:I����t��t��Iv������9���w�
e�<��p�f()r������Xf.S�0�����;������J��f/"wr���-6�D�?S.t��}&�A3�F/4���m���!���&J?\����X�����7o��*_���!���4�|3G�a�l���f�(?D;�/���N	1E_��);�N�������t�����]%-�����p('g�.�T^w��
f9U\e�$�,�@�( �Dv�S�Sa���L�����>�����1.|;��1Ql=�/�j�����2�����z������f�K�o��� ���5T'�ua��4�1�v�3��i�,�E+�K4��^!$�����e���]t\B��?#@�po!��m��#��x�m�������������,����s�Q"��
=N�[ct����W���am��N���\�I����um�=��[{su��[�y�AW����`45���g��C}�)>�@��������i����C��Fm���������k��
N��S��������l�o������|v�Aw��
(�����z\�|
�${q��^.V�ie�mGT��K�����C�d
0��.m�zU����n���l^����C��=y�B\����z��e:��Mzu���.�yz=>��D�U�%f��.��U	��XoI@/B��D���[�g
���Ba���w�� ��A0���U�7k����=v0#!�wX�4]iF��0#%����\��b&����b���O�(O����g�zC�_M(��$��+V��$��13�B��^�����z��5��sxv|�������9�=�_��d�_�)����4w;�7x�0�)�9u ��WG}Nb	)�B�%���&�cD�X��4oz�m�������6��(���WG��{R��'!�|mr8�K��&V8�q\ '}�Q�h�p�7��r����|d������H���1'���.g`8;�*�C9���<���Q�}T���v/<����Gb�V��2����|9���V��:�{�$�-�{|�G}����/����
$�A^�t�2�h���x���p��}NB,*��GN�
:�����@$.�"lnr���&��s$�c���A�3q�u5�wo�^�����N��uNN\���zS�2SJ	�������*��4��7-�2�^�y��iO3������_�>M,��9f���
�X{�5����_�1TUe�P��*
�Td��{1����h��t�_�5�Tq��n���V�G�^0�l=�������\�bHq�����H���O���U��*��P�CQ��\�B��wy��^"�=����9W����s���x��O�M���c�N�=�Ja�p�}�v.�L��<[O�:y*�"��wG?�n��B�"�t�����<r.~z����rG|��^����PV���z�p���"�!	�S������bLcTn
P���R���Aa�F!��\s�VJuM8L��Om��R=���C�}�����T�������<��-$H��=��G���8�C(��<�O��3g�r\E����3o�\Q����s��(2������e$Cv�#���+O123����<�x^2�&�O�����(����X���%`������g�v68n���gd��[��V�Vb��hoy��-h��}�9u�"h`����c7A�	:v�\QVs'A;!g6w�VA�n�v�
!��
A'N",J�`I����D������[�����X���W/�g1����|�����#�
��I�I��N���s���k���Z��3X9���>Q���.+�q1�8�V9���<���eN���
]����K�����A���Y�
\�5/='j��������,54�a�;3��C,(
�>lyvA�^�Z��d<�xc�8���!�Q���f�T�1�Y�'����=��'�'��z���O�	S��v�l
�QA�zl�V�]^{����k���BfJr
w�),�u4slp��9�j��zR.���en��*>��,�������i^���
��i�
]^���A[a{4l����g_�=y�j�]��m�[�G��=�v��W}���xpjY:��>��4W!��Q3�u���;m�I\(�1�d�X�On����m|��U�&&y�(�;��g�}��^�el��s{)=���������������gV�c+��M��H^��QN�A@�:n����L����j�����z�*����@�J�d�����~����O��X]��h�-�8W��{1 �:��Ce����@��b����Hbd$r*���D���f�S���Q/����{%w{���"�^����j�v[��x���5Y�Q!{}�E�/O��/�E�a��z������@�#
��#
U�$(��c~I]KMjq`�"OSF�jg"�����V ����y�
9�z������}�Zm/4�m%%�cu�q�4�L*���}B�>Y��n�nj�����AW����|����M�	^P*�����`���MO	^���Ung�[s����#���5��X���CW��7��(�4��jW�[���zW>����xy�(�lvj^Q�����d4�b����u��>�K0ZU2�y�ALk�'N>�������
���|�;���Z�����NWNO���,�f�C�An������1��J�)_�D����^���x��'��@#����� ���c�����)�>������hd��6��H��������6y�o��c��9���=&������Mf��m��!fo���1y/��\A�y&��`#
�,SM�����~X��!8h�����1�ok�P�G������U��wICCIC�CMQ{}mg{�E6o���Vlw�����^v��^�������i�p�)T?"a�pd.T?bY0�8��o�~teA��U�'
�[I������of�1,m���5�9p�@G�@�������<���	��:1��0l��3����pk�#��	Y1���������Xv�H�Hi��I�,�t�����9iB�*�%�y
��.g{��U�������,<T
���)���,�Q��rD����������L�������M�����D���6E^����*�6�{���"�M��$��K�J�����
�#9������hi��<CZu��I5�������Y�C�[�p8p�g�[h(tv&���T�W�*�T�o#�
�b`�����6f��t�AKah���������2xV��=�����;�5 �)���3��Un-��4p5c3N]�:���Xd[stICb�D��A+���93��Yn��,x�T>~�4�4�Uwo*r�R����;vpcH23t��D�\w9�	�fB�2q4��G3��L�>����B���lM��.����n>O��L���F�Z���~����y7��_��o�_3�h���u����m�`�t
 �	���I������nD������k����
W����o2�����>��_����j�s^0{qK0~i�cVG�IS��0�Q���L5!t$����A�z6�L(<)/ZO�g�=:�D��A��<�;����q��k��c��-�
`���l{��vOE��&�LM���,i�
���!/�6u�bx���CT����i�I���S%�II�C����fH����)&M���4������� bO�h�F�C��OI������u��A���l�7Q�2����`���k��sT�D�����M=�Z�A0$���eQh���<i����_���\[b%b�8c�������)�:����bOE�+Q�����hQIk���Z�5����FXdT���Z�_��Bc����	Nj������L�/���5�o�W=��X�J�#x5V���Xz��h�(SZ2����$���e�.c����b~�<r�5�bK,3�B�O�K�*f3�A��^�
X �f�����0�4^2j&ay�����Cm"w�|_�^^�����Uv�n2����jyw���z�D/��J��'�[,S�Ks������t��J"W���j<��VNp����HK��������b�>�9`{{�������Pex$��������#�n ��@���'6�!v�r�y��F������	;����������<Z�B�#'j��>a��-$lQ|�;��� ����H������7��OhEm,w�
���)4?^��2$c;�;p�.Ut�4�c�R��kA����e���"���>*��v�EN�tBC/�5�,�����h���d"za�w���usc[��-���>�R���Nw�b��(V>mY�'.k�(b�49�m���_{�Lf��;0)�J|�g�^��kn���7��������0�t�``����0G�s;����J�31����53q�+$����J�]��[+�B�%
d6�a�kV<Yxcf�c���9u����6��!h���\��._�����-�J�f��X��jB���Gvt�Kk���W��l�g����nf���~���yz�~��Yq%k����lB�9:r�?�`S��J 3����K�b������Y��/ \y����0�;���F������~W�8��2�0�g�e�hB_!N�@'��8�c; Nf��N��V!�W�t�����j���/�@�rcJ[��������(�Lw��]*/',��DJ�&���k��cF�J�|����	�[��(d[3����b�X�y��h�,F"T�����q�2��b���
1���)2�0���fG�Q�
1
q�8�q�
�F�~���X��e��PA7��-*4XY�n:,�z��$yG!��k������4������>��1��~/��Jk����"�]Th��UW2��`����>����]�[g%�L���z��;�������Bc��6xq@7��*#���0��Q���b�kLpO///_h��9���D�e�p{e��f���
/
?dPL��#-��K�=l�m���2L+B�2{k�*(b����TT�|
�?.�~����i3@O�4N�=JrUM�M�*j����;e�-�����q��Uy6U��2��ml��CZ������-��@��pb��+�k��U��a�,&��J�.��v�-I��W
r�$�z��%	6��\��jl	�6v��	��d�-7�B�q0�z<�r�����[�U��ZL��`]F���o��U�A�7r�]�$�p+sIK���2f�"���%v9�!�Bf�i��I��22YHf�f�|���3�.
U*q�"=��6�@kj)fp�9)�\������,B�0��8�Q�`��59��NU�r���|3f�-�,F��lnco����a��,s��m�%�h�*�����uQ#�v�G�m{��������Q�i�����K_,B�[�U�f��z`Lsz��x*,;8N�_`g�����J��������h����Cf{�;������@{=F���a�+H<� ��U'����
���� �^@�N�_`���������5���!�)]ke��9����;��	�m���� H�w���y�� ��`�V�������O�� ]�0�C{�Z�t�r\~�W�2,i@������H�M8{}�C5\��(�]�6�}I�FoH��"W:�;�J�������~a�>�NYa�|��p��s����K�%�w9�{��C�e�
o������y�gKt��o��9������C ��
;��E�Ax�����R��c�c�������B�|LbqLbqLb�U"�Q_T`�"�]A��`����m��F��(���u#��Y|�����?����Z������[��(LA%g�q~��R�����l^�&�������~��4�^#�����y��I��������~v��^���z����O�;2�fs���?��bL8}BB�a3N����*,=<�8���
�p��q�%_k�]��
���`���G���Zcd�RRTB���#�G�>��'LTa?����O�wThH��-@B ���=��w�QW�����8��>?���M��B���*E��j3��0bl�t���g~��q���^�l�n���x�L"�k���b���z6�P�������lZk|��g������\O���������~�|v����Wx����Mz������>��s�8VS�"��N/���	�z����=z��$�0����3�D3�W���:]����/A��	w�����l��V�l
�l�����:�U���������������C�7�V\|����)�Ot/�Wo�`b�Q�s��E�v�g��6��������1h�5���ig)YLug���cW�����b��+�I,�����qT���?�u�~���iO�l�����z�|V�i�	���A��������c��F�Y��Yt��������g1�q�3��$�	C�^�uR*��F�g�����������mQ��m%���n��V��w���AX�D��p�"���(s�|Q�2��_�����:2�>�
�����[b��/�U9����I1#��O�=��u���O��A��|R�
.mn]|R�r���b�pqss���b��[�k�������!`w�R�i���M?���(L�=-o���]��G����:9��:�g����1�at�Z���x�� ��l�Y�:{%eb+�4��NS����4[���UU�`�����&l�_D��
T�g��Ir�Y@�t�����\T����yX's���t���wT��
�h�4���:U������$o������4��mW�)�&��S���J$��?��w��2�HL�"3��~	r� +�6q�+���Zk�K�0�Oho��0�W��f f��W��w5�V�����%���x�]
c�U�j�"
d��Pn�v5�V��0��d����Y����)2+k
�TX����n!��JrG����T�����;R5�w��i���fW��b�9j�a������]�v��k���B�K��2xN�����P
3V4���&���0nwMX��u�&����]��kf��I�43O��Y���k�����5itf�g���L��I����13+cf46��0�NZ�{�+��iAg�C��G��l������*������O&������M�DsOD?�7E4f� D���j��3y�v�d��H �U��\� )�O��('�-�9�^a�f�;��;��-�{�b)�?\v(���<w*�������0(�&5�(�IA���;3�.?�>)�����L�GQz���R�2���:fF�Yo?#�y�(J�kt
2"��$�������P���2�������z�H��^�*�mY&������ ����{�8s"�5Vd����	M��+p��:S��l���0�g�=���}�K�U�_����
�KL�}�)'{I�'������/����wy�o��+Il������gq��:�
����������������?�~�FM��<[O�:5�tNPC�G`wN|�����i�����|^a��g��{{����w��p�{l�N\�q�,o�+e�����Va������&��tw4����8�<��c;�x7�=�L��;���Qe!{p��������I`��V;[�����}��������Z��M|���+\����f6I�C�5�\({t%�[Ti�o���:[�8��n����3#�@\�?\�s��n�vs`�]�����h���g��d����3�G ��� �,�Xa�3y��L)9&P��zZ�td1��_����c1�\��=�Jg���1���|+O�q����>�?9�������n����+P��Sn���]�J�	��SH������U�#���bi�ZY���UV���_e��VV�����
����0���Jp	&��,�&x"�4r7n@,q���[��
���t�h4G;J�Fs���1ix`r�����i�'@���=D����=���!��2��eHU9�����o������<��9a�V���Q��-#�e�� �2�QF:�7���A��r �rRI9I?��Ma�NI4����3=���
L�l�i�1������&��_������x��TL
D�S��08=��j���:�g����	t|�0���7����@M��
8*q^�;���m�E\��w��	
`�Tl�@a0N��U��);�Y����`����OT��('�������<Qq��r����*�33Vwf��p��@���'���#��Z����ec���,&����-� �����e���

�Pp�(����[��� 1���?���,�T�����VW*��'�<�5�����P��rQcX
$_^=�A�"���8���������*�:��1��6��w�cm�c�0����Pr��0:��l�c�m��0'�0'�aNJ
s�4��atB��j���^H��	��������Y�hy�[I����H8E|O�gR��r��;��x�$����8�u��������(�s�c�hXF�_q/��{��\���G}Ky�d`��\��|��h���3�����[g���f9E����`�&w�����f� �J���,Y�C����<�������-������W�
��v:��i>b��<�@3eW�.��}4Sf�@R0�>0���u�Z�.����X�}�'K9��������s>�z�H�y��4n�DQ����_��u����?���;�Uz���u�AFOXo���&J�S���n`����h���v�z�("�����w����x{��]h��,c>:���c��G'������Hh-)F�;��������d'��d-g�Y�^����(����l5Oo	R,K'��d�dys;�6���`���;���PM8��$��M����C��6�zw��������98�����9X2]H����>��Y�fV,��P�<����ar�i��^����r(��	�s�M��:��#�
���G�����x�I�����.����<�����Q��q�yTr�@K��3~o�$�$�� �C������C�-���0��I��$o�E��!M�t����:����D�=���������G	����GLT��[o�'���	���U�����b/+�GB���,V���;��x��A�R`��������Ee���T0R�cU.��G�x�>����e���T�C�_�Ye��E�!�	����1�u��U��`�'7�����9qMs��>��w��"�s�Q;@�,te����q�#q��Y�x��x����*(k�KE�����*�S$Q�1�n;^g��U�gd�)K���mA���������u2���"�A����C�:�e��{������!����������������Cu����/I�|������UK�2��X�p�!�!���8>�;���x�e��|6,��nYl	�H�$���gWW��*�d������-#���3���=�=Bc����������,�V�p��p��Y:����	�������ov�C����xtQ��F�0�h�.����8�6]��44������H�i�V��Z�����Vjq@�^b%�V�bn���1�RKp9�V~����Fc�E+-�q.���~������vxX��!;u���-��gKu��xK�y�R]����[h�<��ut������Oa����;;W��
�a��w���"2Ji�P!/(1��U�(��C�3����}/o������M����
��;�!���i������*�/��~����m�G*�?�r��D��\��-��X�pyX��������'��T� *�-���*[��oW8��>�G&�q~o�Y�n��&zz����@s2������������V����O�1��y��
s�`���f���nX������=�7`���O����q�Y9�G2��QRrT�M�cz���~�"��O��+����4N�&!w���DK�N��&���|87�nL��+�����2@�����E~�x�����^����=)��R`���q�} vF����^�G�F��$$��\�s������#.3�x��!�+sU)26�y|������W�lB���oy�����`;���p�w�(1�#�����N,=�����e��/�����M���8$�h�I�����2�;�9��4�|�cm��d�*���k�i.��Xy�c�f�Y�iN����e�:�Q�1�T�����J�unLJ���\�h�iN`�A/�F���*���q������8�)>��};����n�����F�n_�p�?l����;�<t���q����}���)����.��������RN��p���?�����c��v����A����.<�ow�������������A����iu��9n�W��<�N��a<�
�q�?�HFZ�q��P�P��r�<�B�W�U����%]+����f�=�/IN��@�p�����|�1�:���I���x�����L��%�P�w�����&���@6�@6�@6�@6�@6����l��l�9��:�@6�@6{d��J�������u���3K���:C�;�j��"VbQH���ee1w��
\�Bt����L2x������x����y+�����WUa������
Il���c��GJA]j6���zp['��u-����rH��g���i��g�_�i:�kW���/����q�>���A���?h`������-�n�1���V�c4�:F���1��g?ew��q��q��q���=F�lu���y�y�y�qw�����1��1��c�3��<��<��<<���l{��
��r�
�����<��s���G�ni�;�B��fy/��BqI����0z�0����=`r��
��h���X��5���?����S����-�=�����>���of���8�Bv�I�b���o����F��Pa����
(����<$k������������n��{�����[�q�-���%�8}�?I�����_	v�?��o�>e���Gp��I��!�/)���k�����=8:���-s�C[Zw�#�������T�M�]#���r�����n���Y�Bx���s���s���s�����'G�����c������h/��{������]!�w��_���*~=wg��u�>���t*X��-u�\�'H%�����w�-����	Y�s���.�|1^3G�������w]�2��7`�].�{H���U�nB�M�7�2�@yXS����%�v�&e�&ExS�J�6��4q]�g�9gh��R`����)m�l�xy������f���b'Aq��t�6�i�Q%.��"}rt7��Q��
�l��:�`��������f?n�rv���ho����������a��bp�R�>�`=�`-�n�z�z��\�:lV~4�!�2��0�cVc@�1X���1�1��b�>"?Ir}.��4�x%��[�G���|�cP��)�0RG�@����0�"������Q7<V�J�a1���Qe��H��cV��r�q�YF<��/���\���w�9w�+�+�\���'�x��a����E>E1tp*+�n~�8�����)���T�c��=�+lxx<�C��i�c�U���.Hc��CW\t=��B�CC��}������C�5~N�r=|=|=���9�>�~�z��z�Z�e6��5�����{+����+ED�^�Dw����n��A1)�-�)]���3���7~Qkq���a-�"���� fa��=�=�(����r?V�'�>����]Y�.T��;���;yq���=��x����"l��q�ib�	�E�L�z�
n��>:Y��nkZN?���it��)�-6M��1���F���7?��q���p�O��''/��`��O�Z���;����-�c$h��A�� 
8j���?^�IT�@���>���V�o+�U�����=Y�;=��=A3J�!`-��LT�~a)�.���=Qq������>Q�{��P�'*�2Q�5Q1LrI�-�F�X��q,��Yt%�?�jDWx���X���b�5����JO6s���:{�=w��EdY��r�-��M����������������UBnHmD�h�Xk�c��mQ��j�a5�^�����\�JO�3���2���5��i9K�trb5���8�����Q��;����+���0��0��0��a��a�����a��W���91�9��9�s�
s����0'����f����\ �yEl��J������o�
?R��C4>�q�!
{5n����"U�� �M��H�_�D�&a��H�Fb���7a3���x�(��	[�^����������\��}���������Q�����~���s����V���j��B�	0��&�V].Wh����j6`�Y�]�h���j��������Ol��-�f0����g�>��3�<��5u}�������}������n����!ko��������� ��Q��F���{/7�$�M4�]^b�\l�+�c{��hi�v��S��:���M5�+���U�3x�01���y���� 1��
�-�>vQ�/O	��]T��BU��c*8����xTo��HT��[k9��?nW����������cz��c����S���@s�������s�w�.��gN�w�.��c����C�>m�������,]Y9�:9[\���M�cz���P�����EO��+�A�W��orG��71|��|_���c���|D�%��r ~S|��������f��^�*)kO�,��������68�'�V���^��D
�vOzifM^�P{h����p�1�dG�d�H6���j�����Zg��+q���If�S�_e�J�gJ�k9����'J�(�o7�d��V~�F����=O��O!=��&q�����LdLMn{������0�r��\a:c}:�}Og��S�I�
fX�2�	5/�u:jo��t&0�r=y��Z��/u��T���=�����-|6��^��.�M���.l��Q�Tt��?�>�K�G
�N>y��� �����T����C�}���xj�����yaG�`�������6��v,?6,_���y�m�xa'�`�x���{n��vW~�.�8;���z^�=��{`�w#��6�{a����l7Bxn���@~�l,��s
����`�!<�����������F�����)g���v#��v���(�g$�}�	�8�~>}5 Ul����l�G�����Gr���f.N�^������Hn�In_R����fy�y���
b=�H��:?{��c��� s	��f�6g�6A�kp����m|n����}����z����^�_�z��d(����(��4�G��$���CU���Z$}H��1��#����Q�l�X�4�!���a���1��%9���)=-W��e7��{�;����ZQ%M,4���_��j��D������-�Su���Dk��a�4�-����gY����C��D*
<���DX�p���X��#`�����c�T�-��R��y�K�;w�E�I�!���%��z���NX���UX��MXa2����B��~`2����P�*/T�6.�z�+�J����������OB�U���&�����~v�	Ny���[��I3����������X���^����8�{����G)!X�����rN�������B*�����M�����K5�
1����'x��^"l�#��Q�;��Fd8yV��9�j:�{�~5��N�t��.��]m�����1����,�bN�O�C'9��)8�Ud�N>e'�s�T3���]V$O����t/��0�{�LU���T�50�p�^�|��c=��F�D^�Q
i�.����� ��I^X�����������-�m6Dj��l���:��t=4���x�'9�b%Vs
�9��K�%{9oRD��C������nc����7�)�������� ��"���6�;�#�d7��h��l^�r��m��]�(c}s��y}�=��E����^W�`j����_(�[������Z%s��o�3V��d��]�E���*[�Q=N4��`��0�����mg�R��7����~7k���h�n�m>f�B	}�kv��ZQO��&��?R����6���S_9	����������H���k�}��f�����!(������k]�}S]����fM�X�9��U�Z������XN�������O���>{P)�b)�b)J)�H��C;��\aI�}�%��
�������������{0Px�wj��Cbf��zQ3�0O����)�b,7���ZA�u-��c�Mr<���>�t��<e��=�������it	�y�Xra�+]�.FQ?����m��\���=��c�����1�|U^��P�b�G�
�S���N$n�RV��*�aq+##�.&��x7�ni"VN���[�I���(�z����0�(@)0fDa��(��Y����0vR��������2@����;�e�^�1�j�Q��z5yv�g��
��=[�}�L�����o�qh����Ii<��q�}��^2����n�X���h4J�L���{��?�SJT"��F��-�Y���.{aQ���6�"��E����!�>�+G�N�sb��d�le�������
�M��@��9�$=��/"���#��3/�H����H���#����A@���10���3��(�LtC���7v��I�3(h��7���'�^}F��*,C�r�Z.����r��\��K4���u�r]
OM�����` �wofk���7@��G~ Y!�a���71�Z�</���Se:�O�����X�y�;v�n�"��F��@q'�8��:����#���An.8��L��o%cT�H`�-�&��]d�h� [������M|���Z:�lZ�,,�r���_N�xh���������+�������������/U�������������-C�e������X��l����\-�s�)��M��$1���x�L�����u�Ax}Q�����a���E����2K��"�bm��,
�J7�*�*Id�I�ut��s���][�!5�������Tm�������j����`!x�+����*0���n��n�+�;�w�'��{��=��
���D���x|����]��<�c�r���S��^��
�y������5��UF�L�|l0T����k���k���&�������Ll��������w���s�JJlt������Or��"u��Sa�#��P���=���&GXu�"�F�������k\�\"���fU��Z�RPT���I":P�P[2�X�c���v68n�����Y���������D{���nA���`qi	�HZ��u�QN��+�t�&i'�"��I�NH�j������H:v��VI�UH:q�a)P��wK�4�i)��g��I����i�y��������'�7�$��J��tf��z�fIe��3+��u)=�U�HZ#�
���,?T����=���m/����:N��v�;A�S�����
3�G�e+;G�t�QIMt��m>d];>�����o[��]��]yU��s����T~'L��j��;��MeN������6�["1�,p����3��
�g�3�:�g���R��ih�K]YseQC��N������KJA�=�f3W�I�`��gR�g+�[<R�Y���R,������X������
��4��EM��n����n���K`)D[�fo��`;yw� ����^�=4�swQ�w�T��"pr��k#2�yy���Yj}\���%~0�����z~��1�ks�����t���i��jZ�HIk9��{���cZ�G�V��g�n��uF�������k���O;�z��R�L����d�uNvDst��#y��8�]
�G?�>h���w�1�6?>�-���0�x�����Ls��(��b�/�������	��	������ES�D��EI�$�SS��SZ*�0N3�G�����y��`�:?b��9������!��|��Nl��f#�/�;]��G����s�'�f�7c�x�^b���~K����Xk8��c���^���;`���Z�v���8q�g��1�����?��^���h�&����tx-z<yr����� ��2	LC=<��4D�L��O���s���OV���Y�p�����77�W����}^�eMS?/�r�)�/���i;��rRf�c1�n+L�a���3���:UnX;��\&"}J>�>`z��V�TK$��d�fE�&��`"*��dA��q�7�d�mF�^�Q%�|���!��A��)�D�`���Ut�o�QV�W���VBj��te�tq��'������Ihz����=y��v���t^��4���ah0w����(�6������B���
^G���"Q�"���2i�{��m�U9������6i��V����h%9ZI�V�����_����ZB�f�*os��1���A�}[b�tK���6�,U<.�~�|T��P����X.O*a�!�G��WN���F����7������I��IAa"�7fn�4����s�U�B@�t-��,/���Z7�#�%}��i�<�� ����Q_g���q�BN������v�Q�����K�����Tz8,����^�E�G�:�F�Pw�x\���4��E��iH��=^��;���Dh����c{��h����h��3@��q�\?�~��/�^J4��SCF�YF���X���x��C��Tr���w#��Xu���$=��&����^������C�����#~Q�M�E�A�����~w��������-�3��2�3�Zm�M���5��wW��l�N���f��8=��oa�A�����	���.|F�^G��+����EI��:Q�~�I�d���<���Yb�~����6����\��Gz���h0�D�$��i2���$����nvz�I&���L��4J�OX�}���h�:������������6�h�����=�������r|6��L�M�{,:>�M���l��ECu�'���>jtzI���n�7�r>G������~D�g�_��'"��r��'EN�����f��k�sX7����k��n����{N����'��,��^�o�X��Un����hy��,{���br�\��k��_�����<]��{�[�.f����L5y��'�:���aD~],?��H���I�KB����rk�'7u�r����
�g~�c�N�'�6��$����38:��H�Y��5N'�b���v3���O�z��������$KP��9 f�3�lr
�s��$xX�2B:So����`���a�Xf���3�
�v;M��q�����!��`�eI�pY���I�h� �g�;X�z��\��V����.����
���#�N���'O��0�uVq���A����?���I6_^��?���?$M�{-s���[����$��Q��W,�����ob�O��f�~�����:4��
�;��O���X<:!B��g_�?�+��X��������H	K�r�:N��d�����
E��6Yu���)K�]���<R#y"+;`�����d�SV1b�ie$������.��f	��b������8�"����^��C\��w�Y6N�N7I��$����ew2�����w{i��Q/�������8Fo���P4$�O=�?�O�~�D����Y��&�`����4���M~�q�q�X#�2��I`�&�[�����.�����K`�����`R��a}���v�y:��mtA�3�'�f���o�VH]�F�#R;f���
���>f����s��d�+=�(w0�l����t��U\��d�����}���M��t&��r����h�qeW{�E%'NN�l0��8�����#Gg��J�{��������%d3#�"�\e��1�#0��f���J��%�W�s����n~W��~��%���

�?���������f�%��7WL����[@d����0I��������x����f���|���Gn�~^k��jkD�:�\b��k�7f��y~{�Y�x��lM�`��5�j�����C9F����O����d��=dw����Mh�,��X�Xo������_K#��������?�u����Op�����<��{���<�5r��8�����3��{����D���&b��]���!"q�d@��6SD+��K��8C�1���3�h�Q����ks|r3	/���l#R�j�m�Vt��������E_�������^J���^�F�[�w�=�P���@e+����9^�i�l�#��R�@� z�������w��n����ZX���%bZo�]i0�J�F3@��Qb'}�G�7�����>�h�b;(4!�._j�7��'���Q2���q�o2$�:{�)[r����dv9��X�D�]�k
{-Fx
���p6��sD�A��P9��aZ��]j�������k��w�&����%n[Fm����V�V��tV�h���L��wX#�%q3X�=�0~�whI�uv9�.����������������Wo~0��
&]�aL�kLu����2:�0n��A���a�~n���-r�=5��5r�$�A�a:��w������oF���C6xv����2�L}�[�~M�Y���?��~b��*�B�*�+,�T}��d	�0���9c����3�����3�Nk�fs2Y���O�3���4���6����5[i��	$����������K��{X0��+!���,��,���J$+�5��Q�5��8�?B�>�2j$��������/����'�G�;���nR��0�z���!���B�f}�.3�@����V��g�9`
k�!�S�����(/�d�j���l��
��i�R�QN^Bn)���_�+������(���Qj�h-@(;�k�b
B/����F�1*��cc���<�	��$CA�r��"[���3�*���o�A�ZZ������&,	1���**)�TMR�d$
�����H���I	I��1)O@��
�$S>��(5B���%a\���TL�N2�Q��`�6���\�^[*j��ed���`���lVw0/��{B�&OR�.��@d/���w\8c{OX&�����%#B�����fcN�$v��Rn�=;`���:^��x��u����:^��x��u����:^��x��u����������0%
#31Dmitry Dolgov
9erthalion6@gmail.com
In reply to: Ashutosh Bapat (#30)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On 3 April 2018 at 15:34, Ashutosh Bapat <ashutosh.bapat@enterprisedb.com> wrote:
On Fri, Mar 30, 2018 at 7:36 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I am working on commenting portions of the code to make it more clear
and readable. Will update the patches with the comments soon, mostly
this Monday.

Here's set of patches rebased on the latest head. I have added a lot
of comments and revised a lot of code to avoid duplication, causing a
net reduction in the number of lines.

Thank you. Unfortunately, there are already few more conflicts since last time,
could you rebase again?

In the meantime, I'm a bit confused by `merge_null_partitions` and why
partitions with NULL values should be treated separately? It became even more
confusing when I looked at where this functions is used:

/* If merge is unsuccessful, bail out without any further processing. */
if (merged)
merged = merge_null_partitions(outer_bi, outer_pmap, outer_mmap,
inner_bi, inner_pmap, inner_mmap,
jointype, &next_index, &null_index,
&default_index);

Is this commentary correct?

Also, I don't see anywhere in PostgreSQL itself or in this patch a definition
of the term "NULL partition", can you add it, just to make things clear?

Another question, I see this pattern a lot here when there is a code like:

if (!outer_has_something && inner_has_something)
{
// some logic
}
else if (outer_has_something && !inner_has_something)
{
// some logic symmetric to what we have above
}

By symmetric I mean that the code is more or less the same, just "inner"
variables were changed to "outer" (and required types of joins are opposite).
Is it feasible to actually use completely the same logic, and just change
"inner"/"outer" variables instead?

#32Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Dmitry Dolgov (#31)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Thanks Dmitry for your review.

On Mon, May 7, 2018 at 1:06 AM, Dmitry Dolgov <9erthalion6@gmail.com> wrote:

Thank you. Unfortunately, there are already few more conflicts since last time,
could you rebase again?

Done.

In the meantime, I'm a bit confused by `merge_null_partitions` and why
partitions with NULL values should be treated separately? It became even more
confusing when I looked at where this functions is used:

/* If merge is unsuccessful, bail out without any further processing. */
if (merged)
merged = merge_null_partitions(outer_bi, outer_pmap, outer_mmap,
inner_bi, inner_pmap, inner_mmap,
jointype, &next_index, &null_index,
&default_index);

Is this commentary correct?

I agree. It's misleading. Removed.

Also, I don't see anywhere in PostgreSQL itself or in this patch a definition
of the term "NULL partition", can you add it, just to make things clear?

By NULL partition, I mean a list partition which holds NULL values.
Added that clarification in the prologue of merge_null_partitions().

Another question, I see this pattern a lot here when there is a code like:

if (!outer_has_something && inner_has_something)
{
// some logic
}
else if (outer_has_something && !inner_has_something)
{
// some logic symmetric to what we have above
}

By symmetric I mean that the code is more or less the same, just "inner"
variables were changed to "outer" (and required types of joins are opposite).
Is it feasible to actually use completely the same logic, and just change
"inner"/"outer" variables instead?

I have added handle_missing_partition() for the same purpose. But I
didn't use it in merge_null_partitions(), which I have done in the
attached patches. Now the number of such instances are just at two
places one in merge_default_partitions() and the other in
handle_missing_partition(). But they are different enough not to
extract into a common function.

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

Attachments:

pg_adv_dp_join_patches_v9.tar.gzapplication/x-gzip; name=pg_adv_dp_join_patches_v9.tar.gzDownload
��n�Z��kw�6�0�>_��_zEJ���K����W?5���q9��9�,/��med�#�U������ .@R�����L��r����x�����}9[������d�����4���4"��i4f����?=�ON����������Z��"��7�
����^7����\��=M/���Krr|���*>N��I����x��7�����W�w���K���6�������-���6��������������x'���x�����j���OO����d|�LO����{?:�NO�<?������������\�O��N�x���?��U<"`��&����x�?�s�nJ���d�IV/��:����[>����x���=���C�/04����w�G�G���`|��u�{r��#���2������?���ts�bs�ts"�9Q�9Q�9Q�9��<�e��{\h�(���@��<[�������'h��r�%z^�n���	Z>�M��S��c��'<%�y���=������=Z%�1��x<JK�C�q������O�lC���
<U��f6��/	��^�q�1~$�����dM��?|B�?\^ �{��}�������'t��������?��\���^�����3�����W<���,� ����������w��=��^�����9���������u�h�������$��Ww�N���*y��Y�;�|��o�
����V�W����������g��~O+��c�����FwO�����p|����dEj��
���*!���OAV�#:m:#f��rtt?{x@���x����!4u�{�H������h��O��2������3�y���'��}�:���o��������/�7�����������np���-?��/�����&�/I�|�����x'P�.nP� e��6�
�3��^��IYR�n>�X����g<��o(\�>��>i��y/^_NH�9[o)�J��O�A7���o�e������B�w�,���y�I%o.����Y���v��eeWk�/�5�G�������E^l�}M'<E�d5�t�yBO1����C�:�0+�e��%���Da�|�5�)������;���e�^���4	���x����brs��2�umFhJ���x�Q�j�~�\�|����
��7t=���� ~/�q�2f
|O��x��6�������?������!!'���gjLa��
�o�TO�Dc������y�:B9�P�����_/op�W�.�S�����tqy��
������c�[�4�����	��fH���,B�o1�%��1�E��C\�������A?�����_//]��
�����������*0Tuf|�=8���/�����e{PJ�}��@����lJ��A_�}���=���Y{0��A�-����@��~J��V��S�~���?~��|�B�?~���'���P��?�]<"����w���;�����x#:�m�T����hr�y��~���?]\����pg^������!��+�[2�����\m�����M����#i������$�{�C$��bi�g:M�^�o~\.����XH�
�������;,�a��_Z�����RJ~6��M�gs,B�n��0t������b�x�����K���H�����|�-��������H��d�A;#
���1�0ih>���4lg�a���HA�V�/k��Vr������N����(���k{�?g��#=y$�,�J���%-��$/�G�2�OAV�|$/CZ4�j��x�r1��'������L��i9O�q9�J�U��:�H&��wty����G\#_�g��_��|��t9����VJTG+���#��n��:Yc����
�������i#��i2��%`C���D��:Ra�����\�Pg�|(ht�1��������DA���e�xN�_7'
��xNx���D���D�!4�nA1'� ��;�U����~��H���J����8�|E4o����</+��r�7���DV�cje���R�Gr;2��]f��}��kZ��p�<Z���<%��{C�@G���C�#��Z.�p�� �O@�l���	Y�!-2�I��(W�>b�D�N��&'���y�7��9�k�~�%��X���?���m_?�a��7�G�(�/?�N��6���y\q��?cS���V�A�@��AX*���r<
fD��meN�|N�����i0'Zm+s�s�����i0'Zm+s�s�����i0'ZM;'rNj[���t0t�s.w����G�W|��O�����-J�*^R�
��HkF�f$�I��yI����}^�sQ��2������g�Wkw�h�������y��e6#�g�O����|���%��yU<)��~Y5���ls�B)#���.�f�2� ������Z,M�|XG�q7��K�nZ�cj3�{�����8����+t4����SC2���i4� )�	���J�v�63��@����c�T��)�����g1�*���954��3�>3�R�O1��tN}�9U�v�������9
�9����9
�sX���7�dN
�z��i�����T�i(���bNU�e'sjh@,X�������D��a�W�S�e��+�>eN���TY���s�s���z]���k�@>�8���y�fo��&FJ9���������I\X#L�
3���p����'�cN����9S��}��l}���f8��5��������E����\��K#��2;�}U6����T���#�&_�����%Jb����h=���6E�^,g��3�O��������LD�K�<b%�[�r��gU�����>.����sR4���I�)����ML��)�hM+sBe"Nj2%V[�T��)���	�!����7�X�*;��}���B��N]�5b��������6����1��+�������G�Y�1_	j�<����|
7����"��;�>�O����������NP��R!���9��S�<;6����*���>��(wP�$���	��~2a2��0nm�F�Zg,7���N������w�����9)z���Q�g��\?),�V�� e����q���9�Ef�|����$����9zc�����c�]b:?��9�!>�Q�J����\f���	\�{�n���19���Vf$fg���)7#�0#=8`g"V��)%�6��jiO����dN����,��4'�6��q�SbC�[�?���{���I�!��Iq&������3 ����Rx5�ms7��x�N���6w�
�-��O�$�C�K=�D74���Xp8^fc��l\�'����,6T��v���q�<b&����~}Ne�����?��i�;�#���|b=)��H��$��|0�����mH��]oSzmG.�B��f;R��Z��R���~-�m�$��/���4�@��H#� �V��/�*>�����-n7<��;
��+	g������q��g^�z~t�s&Ur&�(�����oUG�.'7��K��N>�_�I��?��x�I�g����%�Y��z�d�����kF��^��mhU�Z�[�����
[��Nf�$���,xJ��/�M.	4mM�)E�+���M([0g�����T0����Q �*x�U���kF@Y��f�����M�kPg��(�M�����5#�|PP3���g|��&v��n��Q��*x����1jF@���f����0M�����s	E���~z�L�(�������]��X�Ax.����=V-+_I�������^I�-2�G�5��Rx:���p�����*6�R�DUN��Q�JRu'�e�%U��o8bNU�����\i*b�ODxe�XwI9�<������V����*"�r=X��oc�����n-�^X�u��Mm�1�����iT��t��^��}L����������1H|o���l��[XI��#&O��[cK��+x�
S����k���KL�y�N���%S�X���U����%�rC�����BCl;��*���b��D�����O~K�9���6����WWx>��=���|�a������j���$�%��m��XQ����|%H��q�3	W����50�e!�*[3b�&����K�jmC�z5z�-,�Q��a���ji��#�������������(d���&�fk�a����7G��>G��Ai�M�6�$�n�0l"�����1w��������E��jT��Xx���5��	O���f��|����
'�����[^�*���g��&4�M�>!p��Y��W�Q
bTk�~�g	��S��,�1�T��u.R�.��� XY���`e���"U�xe�^�����7
�
B�e{{'d��f���w2��,�U�x�*�a�`�*�a�H�*�a�0�*�a��=N�������@�[S`��
lU5�-V`��6w���������+6d��e,�h��Y���CVz�u:�I@��P�X�2���������r_AM��#!C�8��sp,�����l�p��g�����>��]��}R,�{D-������R�����r����	��>-��=���r���--����1��}/�I�n����1��@���������YvC��{�{���c�A�0�n�X@;��s
_���x��-��^,�{�1������:+����������t#�����NB0p�SwlB�e������A�`g6�������0�vf�x����2�`��������P���D�=s������ V!�e���]��K��~:�5� n	}�O��&�D/�o��|�d�����"�L���a��H�!�%����c[�.�.�b���QA�8�+�3��K���3��3�c7�xF�t�-��K��4��3,\:��%4�4\Y{I7��F����P���4�e4l��V�..�/�a�Fnv9�x1
W6*������eO�\Y-��T�g/�����I��S���p�OoA����0N����0Nz���0@b��$`8��$`8�I V��)���X`8�IoA��iMz�Nk�a;���p��p���)z#�S^�GN�r��4���e��rzXb=XN��qp�<XN��qp�;XN����E�-s�`:e���NMV�-:m&F����)E:l{)�T5��-� X������xU��:
BVM����'JA��d���_���
�+�U��;t�
lUf�88�
lUf�88a
lUf�1
������VU�i��������D*�Ui�;t�
lU��]�[��9�v�f���+�[X8V�*u�A�B.K
������Te6�����Te6�����Te6�7-Em��)j����t�
R��U�e����F��MW��(�R��6cA@J��+1l?�]A@JFr����2���a@JDg���S���p�OoA����0N����0Nz���0@b�#~`8�~`8�I V��)���X`8�IoA��iMz�Nk�a;���p��p���)������w�����y:[���r�F���*A�'��D^��m�xwyw�zNH�d|O(��2U���2��P���v��Z�f�S$r7�F;S�������i306q��-�.H�.�,���e�]&�M
�O��9k�t@�'��wnZ,Uj��5s�v�4`��`4�z�zH�
������
�"/�V���4��j��;u`;D.N�QU��4rM�v��b�Uc4���zH�
-�[�p�Rqh���#zi25��5����v�^�4���zi�����w�hj����dZ@/��j7������G��dj��k`�����8�tT�����.��������0��*�!�*��^lMY�nH�Z���������T������!zq���hE<onE�S�=x�2���{���"�S�&�Hp-�~�	?�	?]�~��c���c�=0���?`��~���Ie0���?0���?��`�i�c�����>���p�y	��y0��uT�i0��v�>5]��}��c���c�=0����>`���}��fIe0���?0����>��`�i�{a��K�=X}ze�i�th���,������.�lT�	�~\#��������~,QInF	�n2�I�q���V�vf�E2~�d\n�;�I�q���V�vf�E2���tX�L*�5�������3k�-�	s$�r���L*�M��6����6�����M�W6!a���P�R38	�M���@����	�Ap�@`�T;P[���@`P�2X������~*����V��$X}��S�V����V�����������������_./������O���]&������	J��Y��%^mf����<����l�����u�)_��_~'/_���l:Op�d!%E����B��}��h��c����w?^_Ln.�YB����q���������2���p��h�]�'W����%0�:�,��=���5������O�����*}���h0V��L@�)?RA�M���)�h���p�����8/��!�������uyK&g���y�3��Rh��"Y���v��f�zp���Px~>��)H�,���\M.��b�V�����T6
�l]|�u�e�"�__�.(�u��	$p~�x?���F\H�X�lg�U2�"�^�=�9�-�-� #9
C���\�z���y��h+�g�������tU�9������k��VZt�qh�[�^�g��s���MdJ}��)m�����5L�����yZ�N��i97�U�Xr�<l2>����2�<a�f9O�q9��<$�dq��!Q2a���o�ES�3��!�F��S:����z�k�c
����#Oe3{�������(;�#������V$��`��1�G��<���Q����P�t<:��WD��Y�����r-7��,�AKd�<�VViL*ey$��a��2����SD_�0�[���^��a(iV�::��*i��r����fYZ'`���N��i���O�������*�NJ����W(�$��Z����W���p�����4C
a�:
q�?i��l����@V����x�4o`io���&�xg.�z{��a1�t�s����Q�#�+�E�����T��o/)`����5#Z3"5��!����\�2^��]�%�{��9^��Q� +������}��*�a����}VZ��xh��~Y5�mC��`6p��Y�����2���<"`p$��9���8�a����=YI
!) �� X|S������6�d���h��mF�sl��KHV��T�^������uG���s�$N*� ��S�N	������v~W`J�����{T��,�� �%#��Er_����a�|N��l�X�
s�`����������������{�����f�]:�j�������d�;LW4C�);�H��dC����=��l�E2��W������qM����WNg#��g*�-��w��1������$���O���$�� � �^�
:q1}\$�7���h�!����S��MI-+��)���+sB�n��d��WoQ7�����Z��������Q������ g�(�E��4���
&?+����qV~��G��#�/y>.��M���������N�����~�<�Q�*�T���T��1�B���<���4���:,��3�'��*���oWS�h�
�0�Q�i.aY�M�j��[OF~����u�:v2%�4������b�K�f�N&�E���V�A!��L��pQ�-�y��nQ�-����'�H��fdZ����9fF��������L�c�:�H��;��(�h�>|��^���;����|H��)�A�-L��O����RmsRl����v�Bb�yQ�)J�c�z8L�c�N8�+}�����T�?>���x��hJ���)o6���c�g�;-s�C���x�.���@�'���*���&���`�r�n��m�C��z�\g;���Z�q�T@>;��p�I�F�A|��e_U\!��;��T�|��#���+�d��Px�i��/2�>W�����[�S�����zr����'�����C��I����3"�F���$>��R�N�����p��kF��^��mh�?���&��F�3�m�����&�G����Fe������7;i\���f�k�����8	��:����lO*X��k���J����p�5#����5^;� [���v�~.Gp����O/����7�?�x�,�����7��K�K��Z���k�O@���WR���6�3��B7�����Q5p��*��R���V�tT?�^f�D�r���q����G���b0���c
(�r�*��;���t���^���E,�+����]�S1����^���Vd�e�v�\\91�x��cr`�n9}6;0Ph\�d5�'pr��R���N�������C/QT��0�z2���$�t;`�d�d6`�puXh�m�=@�O\I�K����s����'����N������I�&5����R�v."�@)a�>v�R2�]�M�������i�-\|���b�+�y��`n����bH5����`��Vk�������V,�T��k�e��l�
��*�,��=�<�����r�Ph��%``�z�ZY.x�s�n�`���9�e��z��8���oo����-pY{�e��]���0X��2���V��20U����U�WU����J�@w�k��Y+�]i�;ttW������.�k.�,����TP���S�^e���[h���������^{�������7��8	�����Ca������
����	Y����=������n�l���N�P<E�	����Q��g�G�0�n;m��}���{�����}�Ed�GJ/�n����U
rYj�����`f�x�����UP���f��o�{�����V�]e���B���{X���K��~Z��l`�:g����[�6!�)��G�����c[�Nr���G�/�*Y��=�(�� ����8H�z�|�x��.�b������xO�n@��wGH���|��;�\���H��A�Oo�7e��0��|PF�C>(#k0Y������|m���?}PF�2�-)#y�\#A��40��e�AyX�1h#�����-�UPG�Ji��o�9�Mi"��R���A�U�[�lVM���^��eu���IW|���tW�������0��W:����tW���
�/8�������9`�vEh�����-,�]��J�i����R����8+P]�
�M�T�%��S�O������>�uXJ,8j�[��@���������C���@����~tie$���������2�l��(#����5	�,PF��H`���|��s(#A�we���������t��7��b���U�6O=��&E���yGwgNH�dd(��2�E�Q;�:��v6�������Q�F��R����+Q�u�FP	$���U0��A�V�J�R�%td5xYe��`SN�5g�N�y�jz�&�R�����">��"���`4��k=$Y���Z�vC*���i�J���v���@�C���eP��K#sO�Cs����hj����dZ@/���j7����Z��G��dj��k������8��TI�kEg��Z{*��u�W�I�]d��i��7�N@u"O�:1O�:1��N@u�������T'�:�	�NJu�?�4'����tj����o�s���*;
���.����k�8�	(N�(N���P���'�8���^���7����%���&�����nn���5���`d@{���z�#��%*��ST���T
Wt(m�og:[$��H�%jhRL*�&������������A��+M����{� 	<P��I����P�8��4O�������d���@��M,@�
�����z���yAo�Xo����_���/��j|_���'S��.��N��C�:���x��� �(^��]�NP4��II��Y�h/Z���h�F/��z6�'��S���E����B��}��h���X���1���8�������3]h��b3�[/��C����S��_&�7n>����������x���l���S��?^��M.��D�������!���c(�����#�
���������!�S8'���O�7M���F��f#4C@~����L�g}��g�a}3���E��7��:Y��������|�+S��!h�5��\��_�����[��l|���N����WJ���l}%P�������~�x?���F\H�X�lg-,d .$�Xx�0�4,��r�#���$t)'�hr���nV��u���Q����3�;Yh�j�,G��Z��a�l%�:��@mi�zn���v�?��H�|���H2RQ7�4� m�n�����oMi���Y�0k9d-K����L/��bz0O6{G��}�m�0'��'�����U��K���0�}v����j
v��95���eb�v
k���r��������2�A��`F�������A;�`N��:�������DA����S����i�:��M��=M"Z20*��G��PC�����q��e�<Zn��&Y4���jyL����T��H!����ey�����5(`
����(��P���7tty:T:�</���q	'�
���N��	i����r!��4+��q��#�ND�D%��w��U�=�T��(���J�5{�����r[��+�2w{(V�*��]��&�
f��Z���4�]RM�������9���%T�nN�D���h��������a��t��K�}?J{��������������%������fDkF����8:��:�����<�K��~��?���;�de���<}�o�]E7��2�C�]n�W�C�T����nJ��?��g2.���e�A���[��'o��r�!#���e����=Y�Z�]�O]Dz^�C��!����4r�T��b�M%9;��zP�'Ns�1s*e8s�I����SC�������S��S)���S_:������N��Pa�����J���9
,�T��t2�����o
"+�io	_���;�Sj�2d�$���:e:Rbn`f����qdGf��u���.!k��Z>�<�l�X��4�D�m51�S����$���e��g�8�F��Z�`�ta����=�1������I�f_� [�a��N!_�7����dC����=�R�E2��WN������=����WNW,��g*�-��v��1\�	7~I��V���;���/@��;T�b��H�of�I��C<_'M�����Z��S��TU�����k�����bi?t�����D��W�������E�J9�E�,b���c>�W�1�Yy�/����c�<jy~��q�n
���uoE>�w�}>�����y5��h��Z'��B���r����yvld���W�2T��>��(wP�|������"�0�Qgq��5u���B����Qcrr06u���7C�9)z�����%L�J?)W�z�e����E37�L\��>9��Lf�������&V�d`���V<I���4y*�A��'��I41�v%��$��4�<>7#������132�fD4���g�Dt2�<SJyl�k�l�m��6�<jWl�3�)�����isJl(O����b��o@y���#�g�3�)����������i���H�x[�����Atc��.������0\�U����d��R
���������=���9���#~���r�e�t(��Wo��?>���zR����<�|h
���I9m�/�]oS&iG��B�D;���Zd��������?w�p�#"����Zy��<�8�U�w�����<�N0d�4|7�$�M������s/
�����4<�8�
9�Dd�xnz��[�������zr�S��'����C��I����3"�F���$>��WD��i��p��kF��^��mh��ZW)G��nR:,[�j�[R�$��R�s�U�fa*1
��OM�}���Fn��L�61�h�'�h�5kt�
�e�j$���F����e81V��x��l��wZ?�0Y3�(�V��������������V�wZ?��[3�A5#h�~���4�zd2�A(����}T5�_I������i��j]��cm�Z\�a��
�k�+����K������Qa��Zhx���j���
j�HD�/�M��*i��+�+��Zx%�858s�"o7S����Z�Ex�z�a��lgr�������X���B]�x�>sF$o7;*��"�����W=r�x��cd�n9}6;qYXO��&!O��U���xu	S^]^]�DQ�'����Ur�e������#�l���
�� ��p.)b�B�7����'����N��.���J��6����:�v���@Ie����
�����e��v.8�vY��5��Z�Xnv��R��V ma��C�;{�Y��=��,|�P�j��6�6H������j�s
.������2�W����������M8���8��DO���(Co*����t_^Kw.7�(��������1Om�����gm�^�����7
���e{������(
p�������L��1�|tD��Ag
�5��M�qp�5����qp<5�������*`�����.�nt��S�����[��Ag�m���k�Yk�.{W|0�]3�f��<>�2�����*�<t�OI�W���6`���\���}�]Y�{�H��X���` F��
Ugsw{f�:����-u��������}�G�v��6�w������?�F������Q0�nW��li�NV��f��{!��w{�\������.�-�4��M�N���������b�9F&I����=����AT8�|�>����W��K��@��;
a����eqX�`��0��i����loZ��]~
���r�0X��t.�v�G�g���Uv�k������h���i�}�O��&��n�o��{�d�9����G���1}{$��TZx<�e�%��� {�o4��
���"��x���g/����J��)�\�*+k/����J��x��lh+<u�`�,�������u���P����TI�&����OoA���0N���0@��s�`��c�`��I�]��8m0q�
&�������S0q��L��2^��c`�Pw�0���`�8,�lf�88ilf�x���*0r���a��o� ���oO�H�m���&�M���t����k�����8�Lv'�~��w:k��Ag�o��9k�Y�
��xj�Y�
�ip�v<�Ag]�i���������8k�Yk�;t�t�������Lv��lna��T��N{�\�T���X���l�R���lo���.?���1�o��B��p�����P2F��U�J�(�"B��]q����
�d�B�@(��x��	L ���C����&�a��&�a��N�`��|0q�$�����6�8��p�[����aL`�x�&����1�Dw���lof��
���m�0�	yM��)�����������HS(��2!��#��P(dS;����#L�n��v����
�3��f`l�Z���.H������e�]&�M
��9+t@�'��wn�Uj]�53�E;4��Y&�����XIV�9B���T�!��������FSS�^��l���I��J��FF�h�Fd3���0��9�!�*��^l5w�nH�x��V������T��@���!zqR��R����B�CW3/�a����C�Uh���������Ck}m�K���v��]�C���vW%Qz�X��J5����g��.28�b���,���fV�@3k�@3k�5���f4��	4����,hfA3�Y���f4����T�l[�;�l�tj����b�� �{H���������������ek�zY���^�<�^��{���,�eA/zY���^������m�{���K��e{��mI�o����T@-��[��rM@9�:YP��r����H9k�Jr=g�OT�&@E�Um[�������9�q��	�Z�
���*������LQk�drQ�),�kM*������������T����W�Za���P5�5x�nS]^���9��4���hAE*�JeP���?���,(g+�A9�V��������T�l[����^��@-�X-����_���/��j|_���'S��.��N��C�:���x���[�P��GO��=�/k4[�U�xL���w4[��WX����<A��	`�$�h�L/HC_f�'T^���|w������f�� �mnFx��?w�s��{�WC�������������\��b��� z����������o��_/>��t@�7���h0V��L�d009�J��h���p�����8/��!������ovyK&f���y�3��Rh��"Y���v��f�:����C\�"����������*vku���Le������^_�&09�#.�LuQ$pB	��.�O~��W�/V1[���*�VW1H����'4����7�/7����2#�����I�e{�Y���2���������QA�

����F�=��tc`��1pZ����B�J�d%.��U�O��i��l�D�]'���(���7lE~��Zt����������C��dZL�t�\�R��a����U�(���i9O�q9�J�U��K���0�p��;[����hGx&�^D����u�ax4�7cp4������P�i<J�*b(4o����</+��r�7���DV�cje���R�Gr.f�.3���>E�5�AS�E-�EY��f���C�#�������y-�K8iV��'�u�NH��,������YaD���W)�)tc^R�R�l+j�9,�������
cY�Cl�*�5q�(�Ul����M�k������w�����
oJg9'P�+��=���[t|J��OeoQ�V��V�
h�@Z3�5#i�:���d0���=��=��p���^������\��_��]���{��9^��)� +�����y��e6+���O�d��-��yU<�JU���m��T{_�0�~�2e~�e����y��Y��s��,e���0��U�����>���h,+0� ������
�Q�K����9�C�
��*4!p��r�TCk]�N�N���{|��Z�-^�S�Ni������W`�����$T��,-�q���B��z]$��ih�V���d��h���4{���)���#��3Q��*�I���H}���t��Jg���Xs���z�R����������.H�\wJ�&�$����K%Nc����0���_��A$	��r
4�|>S��oS?M�n������H�z*�������������������
�z��:-J���V��(r��2oD3N��|>�sa���_g��|y$hg������)�<��+r�|����|>-�W��:^��:A�NHK�|U����yvl������1���>��(wPI2q@M{����9���)�i2q+�d��@j�/5HP��[��{2���O����T�.*�e�sQe.��E�x��U��EvF�����cfd���hv���oj����5��hWwi�����F��N'�;�E��1>u�1��n�!��tj��:��c�Q�J��c�F8�c�8��+�i����~T�?>���x��(1���)�3���c�g�;-s�C!n���@���I��\m9N�Y��!�+p~P5����-J�
�@��w��q�u�F�A|��e_U�
��;��TcH����W��?���S)�V�m�������*���������G��O�7�G���?��x�I�gD
�6AeI|���8��j������b�~���*/���Z�����@,�J���].�/��8W0��"��j�Dn���%7t�;�-E��P-Jt���D��#�5Gak�@��	�IS���WZsb���l�5���~$����UY�+�Z��7��T�����oU�[h2�T����`�\E�P����N�JUqU],��T�U��X�WU0�~����Q-W��"����YR�S�0~�y�����?V-�+��UO��x��������+�k^X�u���;��c�.��*����<&������g3�B��'�<����`'��w��F�UchIK�K���MF�K{��
W����v�T�����8:}�����>�-��s��kRZ�:"	^�0��I����z
�B8���V�9E�U
Dz3]�4f��y�Ds��LY2U�tu!L��e���g�<6�R,����x�,c�I�0�[�U�E��J�R�h�N����t��lTr�1�6D\Z�nM�>B�����-��������I��*�����'��vB���mN(��\�t�P�$b��	%� (���O (s};xA�E��1�[�����%�APAy���d�t�����cS����^e��g[i�m4=��U/��cr,�f6�S^��s�]���Fs�N���L���^]����I(���nh��nh�y�,u��z����Vl�4�5�J��R4�5	4��vic���Rl{�6X�Cw�r�������1�	t��]���f�S��������#i`Ei����KB0� {�h�qe���T^�(���A%e={�A�(�K�XRV��+�FJ��HRR6��v���;����v�%�v�����}�v����v��h;@���FA����o��m�[�v�/��PwT����&���.m�\��e��^�;LV���;@�J�����r�}PrlU��;:	~ (��\�@PA������1�-3���2�vAP�r��"(��;�I���� '��rr�`AN����r�v)��}#��nN�y������7,1ok��<�y.��w���Wh;@�Q�@���6�A�NoX�q�>��mh;�m������}�v��i�[>Og�x3[.�h��\%h��C��kRt����8!-���ND��N���&n�MM9/{��\�����Qr��>&������p��f�Lu�&n�"CY�Y)���mJ�s�
|5���I?�����P�aR�zd�G���t{Z�%�MK%��B���&u��j��T�:��lUK�O�v�Y�U����rhd!��e��MA�KN�r�N6���@�DB��;�{(`���W�{�]d�wc��H��T�R�AJ�m�TR�Ag@J��KH�{�YAJ)�>�LJ��|Bj���vn�b2�%dT�QAFU'�QAFuH �ZV��O ���
2j?.���x"j�D���&��z0DTd(��������r��.���@�|��`,o�����)c���b�M�����Qh_��Z]u+.�a���@����.�j�@�t��nD�j�@s�SG�X���{�K�<�+y��������d��<&C0ZT���(�A�1O`2��r�[��d&��.�n�"��� �6Q��������"]���9��d�~�e"@���	������;z�?'�1~Y�����c��<�����l>O���_�>y�_���Z>�$�e0�d���%w�e�N��Ih�F�%��\�g�9����8A��<.~�L��0��p���KE�wG��x}1��`�
b�������}�WwO�j�~�\�|�������'W����%0�V���~���mr���'�a~�p�>��G4+@y&������&P�>�pB%��N��9���8?]���zy�+|��tq}����p�l�f���0�Y��U��o��L>�o����d���Mr�NV�d= �F���|�+S��I���'4����7�/7���JC��l�����/��eDG6��\��_��Y�o������e��w�M�l7I@E�v�/�/�N0~'R����;�~;�������_l�l/�Z�N�j;��E�K$`y��^F��(�n4���~���p����-FGQ����H���E��]4������)�+��Y�*������0��$C6o��7�zR�!�������
��v��F����������Z^����'��IV�|�S|�FE��a����e�(O��i9O�q9�J�U��K��9�M_Rvi�MA�D�n�����B�n{���)�����$�!C���x��b(4o����</+��r�7���DV�cje���R�GrMf�.3���>E�5�AS�E-�EY��f���C�#�������y-�K8iV��'�u�NH��,������Ya��+}�u"Z'ba��r'L����?'��IYg��sdn�b���5MjO���z��"o�n����p�f���VX��J�L<
p3?�&�
���W�!m�8���J{��
	.t	��}?J���������������%������fDkF��'��	�9��>=�����2��,������j�N�Y<�/O�G�[f?��F����q���Uq�+U��j�O�R�_�����TD���e�A6��#���_3w?9����a�����=YI
�, �� XLU �A����lsHfm08��\�f$X1�u���J�
MEJ���(�4�PCk�S��M��C��9��/�e	��X����qJ�(�1-��H��5�+�iy%TJS�1w14y�s�����$�^�=k�X-�S�9[<�U2��m���?v�4��������k���y"����t���N�d4�	5:��S|��9�f_�:�S�����b���BE�����ud�j�VX�,`e^,�2#�}���A:D�I<y�Z1���W,��T���iH"i78W$��pE=NFh���F�m��>���������a��"�>aQU�U��+���b��u��!K�y��/�KYp���5�(�|�nW�/�������"_��5��b���Z���UZG�S���BN���<����B���\��3�'��*�}!���ylB<"s�3cM��(�������j)/��������S�^��Ne�u;������v%�dV*�4%��D#�hy*�M���1M��9�V�(f����)7G���~���&L)Mh��V����m�+�f4�
�M�iF�(�����	n�{�4��{��&�y����)M���R��A��0�����XpL���B�wz��P���&���U�o�����9e��#~���r�e�t(���,��?>��T{R�������|x�A�
\���R��[;��<���x'^��W�(�|i����[��Q�-����������RL�������S��t|r��'���R��;9=�� ����l��N����*����V��}9����\���	��4r@���y�7���H�~�/�����b(`:��������pZ�Z�_��wW�hW���&�G����$�p*�4(��A��8�D��	H%7<o�N�ay�x^E��{<H�_I
�M���`BS�����<�,�������bkRL��5����u	\U���t-�	�(��J�W�90f���������0CUMQb���������>f���J������+��!d|����Qi����Dxe�XS����d���v3�d9�)L�W�k���N�?o�7�U�f"�������T����L���"������kn�sy�������^����u>O���-���f�
ux�����/��"�
�/�����K/1N��2�����b9��	��&#���������BCl;��*|K��X����5ru���o�����y���B,DIH����M��7
�n�&�uH�Wv����l��R����J�I&,+���\sq�*��.5����U������"7��V��!r�j� �J��b�6;Li{C� d/r	��9S/���y��-F����X	Z��W\
p5��X4\��h05���q���-
�3��Xu��m4����04��T/�>l~�4�����V��>W��,p5[�U�����1I��!	��T=���W�C�X-4�02�9����x��q9�SHjNL���o�#������d�7���=q����S7���=s�����pexw����3�v��y;���F��x��4w��Yq������L��}�i'@��f��>f�T
�1}����Q����E�4�����t6��
�7���6���;�w	�?�c��Hz����xl�8I�"�A�(�#�O +<�g���D�9\�rIY���/A���S,�AV���/O��
(\�BI���5����aX���E���pemY�"B���{�+Qb���S9��@�gZ�����K���O���u��F9t�rP������7��f�������cX2P����g�>^J�>�����am~�a�f����L�@��pk���3�)����>�j�����Y�jl��W�^}������,��?cZ�����K����[04-��C��-
p5��Wc�,p56�n���g,`j:ej�M�4��v�mv��f�`g�K���9dv��i�������E'���F�����O
�cX28:G�J��G�
�d���}���O��4�0�d�yKP�n�C��u����c�	�o��,�7����>�%u��ZP���Y
t�|���f�\���y�J��	#����h������&������P
'�eB
'j�%g����Lj|��5o���ZSH��F+y���m�l����m�:RH���U�n�JD�����+*D���
1��� RU�1�uk�&n��Z�����lPV�
b�\�R(-�}��Zp�Ja-�����[h�Y�	Y���&K�\��}S�� ���3���v
�v��(��W��`#�@*T�^N�����bW��Tj����J�]u�Rib��Ax�X����X���~Y�Lv�������� +#����AV6���6���2���AV�WY���Y��n����?�D���
��3�oB4#�*���2H� )���VI$���\M )����$��\@P�[A�%�����-A1�`�����K�A\q�e��	�ew� .�+����n���f7�,.��������-�B\
�ve[q�� (���M (��l�@Pv���2���&�mv����^
. (oUP����/�f����t�������i�a��tp����!~�o�lMB�=��p�|]������yB�d�F1K������<���l�����z=�����x�������d�,�4��������4�qK������������2���p�oz���'W����%0-+|O��x��6������?�J����#�+`�X?]���zy�+|��tq}�?N#}���������nyK0�Y����l�o����"Y���v��f�zp��#����.��S�[�MM�&���W���]���n*�/�:_�:�E2i?O�A�PK�:_=u~1u�������3�����@�:�������{.�,��
�_������O.z$���EG���G.���Tr�F�*�)/Z����5�m4\�%���$�����/J�����6c(�w�N*��W�B�8�Y�K4O6h���>��2�����r��b���$�.���H
\��;�o�����y�OpW�`���������H��w��Ey���x���(���j�����\���rS.��L9?+�3��������rS���yL������HA$�l��"O�����.�I�B��,���Rad��vS��R1�m_}l�Td(��#]omc2���
� ����e���(�?��|^/�}?-��|^ g�)��
���jxx��E��e�%�����X(Z�#�AV�����}�,]	�'��C��
�W���T����2������������2�~��V������*C���[�[��Jj�V��~��78:�C�����`p
�����b�������:�}�b�+���_(A�w�&���x�LQ�W����l2��z]$���n�V��7���2�F4�@c��u��$�����:��F^}��T>�lF;CV�� [�a����NzS5��B��v.?u��}����n0n�g�#BS�I�b�|���jS�B�n>�z��Vd*�q\E������>��~�}�x�+��y�*��v+j1�2I,����5����l��
��W6��\��`Yn��/���v����]i�pI���{InU��)y�)y��%go3K�.U��%~/�:5������Sdnv���|�v����oQ����(�!����Xj����d�����:?�t>f�t~�u^Tr�@�:��,���l,�~C�����������t��S�������Yb�9Yl(������q�<���F�_�>��c8B�����,wZ�N�2�t��]������'�Kq��'<�g��h�\����B�����A�h��Py'N����kV���u��T{�;�{�����~�������q�gAp~������4���:?z�N�s/�Axz����N�	�/W���=�[�E�rrsq=��������/k���y�7�j�H�~����Z�
�����N�C�^3��z�����
�Z}��6����J���].�/����T������,��h2��v��N��#hJ$#;|Vsn��19o�b���^s���Q���,��H��B��>�T�J���M;��Fxd��]	�/A�����W���1�B5������@
�g��jh>�WC�h���@���2%���COJ_������Y���1q�[�_��<z
��O�=yM7h�]Z�h�{��s��b�33����e0%�
��`5s1���@�2������R������;+�0	6�4���nZ'J�������������YS����8,��Q����"uv�6+c���d�7����s�G:����3u@x��];hM	����b�����#O�������u�����7����>8p�Y
Kz�x�N	H��u��a���B�������*�<����L����u��s�zL*�{4�����0��9r�I�b�m�g�5������*�i���ns|��l���qz����
����W�����
����/��pEWN�m�-����T�Cb$����H��J����n�z�B��M�G
:xtK�A<���Re,d�s����n���i���/W�����KQX�!_�T@a��~�R�x�+�SX��`��%�J�Q�C�|)J<D���bJ+��$�B�-9������``�3��C7��$xT�	��[� 	�$��M�(��0u:$�]|���@K��<1�'`��@
�<A
\'�Muzkx4��u�*h�:{��R�c�Y��T��t�����6����*:8l�y��N�z<���Q�����z0���~I(�:�1
�g�y:t�3H�G��<
�� 	t*	���������^���U�6O��h��a?C�pBZ&#�ND��N�e�	EeSS�jOu�e��5YEF�D�QG��\���l�Iu�U��e�^�^VH7l�&%�e<�M�����!1���A>v#m4��
��&K�N����Vw���_iVe���
f����T%{�����\�q7���@����{�>h)^��|���pP�TM�.U
8(�N�;�;���T;��wa�@�T&��$	�'�
�?i��TMo��Gz���SK���|W���>!+����K`��	L��
0Q�D1���r�d���}��{�D����*:([&
tP&3$�O�*�>�������}f����o�}����/�f�������Ws��I�?�2����K����K|r����s��:��^�	�[�_�y����~�(^�S������z�?'�>y�q����/3�.��]���^q��xs����QX_~G�5zY���)��r������������f
qk����A����_&�7n>���w������x4~/�A�OY��������o��_/>�om�����y��1D7�`<+!{f�	���G�%<�^&���#��f`3���y��l*�?��C)��.�O~����>\}����h��^f8p6J�C@~�h'�E��7��:Y��������|��Q���'h�v�:�+�D������
��jr����(�[�f�OG�l}|�����}_����M��/��Rx�{���})L���������~{?��O)l��o;���{?����}Lf!=���������+����S�v�L�������s�8e����	�������\;��^�����^�B�P�&hG{��h�j46jN*Kgt��m�3�T�������w��6�G
V�N.q8��m�������m�-I��,y�����g*u1S�N�03��&��z�R����
�&�3�3�3�k��k��
�L�4n�f*H�����r�f��|�B��
�L��m�f*L��2S� T;'�Y(�)�a�*��#r$�K�?G�Ax��N=�^�!�����yG[������C��~�$���h�^�3��<�'�|�d���
�_�����C��!Y%����S"kPY��S]g������2�R�J�.�=s4 ��o��#�����3Y�$3�}��7�:���[g������Q�����)���j�4��)��CgI �")�P�(�g<�Ph�G��,�y>��
�)����1�#�xQQ�x.�*K��C�r@�i}6����(�"p".?�G:w�3y�_����N+Y����"w,�\��!ye����m6d�|�	2����5�g����r��p�����QP��<����k>�����f�G��H��{�5���Q�LZ��{d����9��L��e�-��":�Y'\��d\n�<+���	���R�Zr�mj8��a$�7Ru"K_��������^�h���N����0�Z�s�IO@Y>%��J)	g���n�&� ^�����{�8���J�\��B*!8�����j�1mgN�|N������8��~s�T���9�����D1�A>����-��V��v�4��4d�4��4t�S���9�
<b�'QC}o��(~��_��E��c�����7���:>����������0i_��/(���G������(�H�$�PK6��.y�@�����|.j�_��]��������j��=�
��{y�<B���qq�������~ZVp��~Y��%�)���E^f6�&H�=8A��m����/x[�*��tTr��������=�H�j/�7x-�F3��t�{��BJ�o����d���hXi�B�Fa��R��@������'�������W����SK������@������/�������W����SK������@��i��i �i�����W�i��SK�������p�����P����SW���P1��v

��9
�o��I������r�p��;��!H��S�=�| �,>��m�1~�Wr��8��F��s���h���M;�2���RY }$��������
��.�g���?T�N���J�N���33X��e�,��,��~���7�������+A7��j�-�1��N�`�x1l�\��w���&8#��._����_���
���8�*7*���5Y�#_�<����~Q#_�F�w�@ir�;�����iE��Z����%qmt�����=�Q��4�##���0���"9��='E���:iyb=��z�������yC12oU2��8���I���Y��}]aJ�#m%?�a+��-�b�I������!�ccx�Q��������e��`^����������7�oB�&�Ph�
����F|C8�AU��&+,��5 ��uJ��l �x*�3�M6a��Ulj�
F\���7g��
���(��1�e=�b�8����z�O�����������n�T�2JeQ�|F��R�E��S)#d�����?�����@�*'�+g�+;G7S��S��/��tv�0������fu6�
��`maj�]�:���X
<��0���Z�Mm��Y[�Zc�-���x�������q��,:a��W����f%:W�.U�#U���=;�Wgw:��~v�������C	B��m�}��XiY����As8��K���;�0^;�0i'ew��y}a���a�&����[v��;�0^;�0i�ow��y}a���a�&����[v��;�M�a����c4Yl���:������1�$5�7T�~�������t���r>��b�u�K�Y�~�3�����������^m��7]-__&�,��M�$���z7s�5����0���z���q�pX������������Ik2#
6�������������j>'�j����&s�`|��I��%k~����� +����f�I�p7��
�"�S�)�w�J"���l���y6������uy:M��<�&}��Q�?����x5����xn��~���w�� >��x��%���Qg.'7��K�;E�O�7���Sz�@���}d����Q�����
6}B�������F�t?�~dn(���qX������6��j9x���b��f w�(������\�������'�Aw���#���.���hi�����emky[��Y/�T��i3�c��e�Q|�,�kx�RF���A���6�~�e:��L��[.������$���������#�_p��70��*��:A��Q���4I���*�u���kn�\��������R\�i�;�3M�5�J������	�V�r*)����c���i�7�
������,M��Z�#8�j/a��
�%,�Q�	�����ue;��)�#:��v��g�"
�r}/��>dO�xgaI�P�7>�O���~�:��\-R�)���WI�|N��x���)s������9-`���������T�N,���Syy*oohk�����]r����y��b��q�����k�q������p���5�����U��������Xr��eW����F�i����W�A�B�2��>�O�hNv�e�\v�e�a���-S�?f��)����#U+F���bdQlT�����
��.�B}�V�kT�jE�F��V�iT12�O
)��@J������������K����Ol2S ��t��&�BU���`k�m|w��9�
n��v�����f���)6ms��MU�F�������Ul���7�����F�[�q���$����m��=�Q�A�����7h����#�`�����w	�����u� ����H7�9�W�w�Ov��2AJ����S�9�&����S��q
�)yOA�q
�)�@���~�S]2� M�4��1���@�[O <�����$�O <)O��O��S�'���wfy�O`�qJ
��>3� N���Aq
�)EqJ��)Nu���4u�����yg��9�J[��Wre��$zw����5ur]�w	Q�I^���r'V��3fw��n�
x�\����"W���<qa�����[B�m�[OB��QK��D�����EhM��I�]��GN���������v������]�V��v��$[�SI�m����5����v��s9-�k/��������v�)���k!�IB���}����v=+�+wG�Y�][�`p{�]�|��m����nW6
J��U���n�%��I��oC��%������W���eSu�{�������-�_0���_<���o���-K��~�PG�W�=�GTM�^[_eWp]5����1��l���V��V��*�Ze�k[k�XT����u����Nt5��Q9��Yh�"-�z���9�W�Z��A�R��5���Dz�m�|���`����S���:��������+[�: �������8��l�d���������������i������H�$��y�\2���<T�/��~�Dxz1���5����]7!��M�I[/>���/�/iK��/mJ�+%��/`f��g6h��gY?�6��l��[�Y��Y�?�{>�K����������9fv}�x	�X�uuPq[�xi��zv{�x��X�� ^j'�������I��������d]�����0���?�#�^���� ��A��d=������~�z);A,W�	���H�G�M >���~�G��"���I:|?�=�~�G�/	~�G�/y�~�����L��j���	@��&p$0H�Hp�
$YOA�����h����<*x����I��3!$A�	��q�M N�	�)���S���,�8e���8����A�z�jm"N�-
�'�@x2J <���H <�O�
Op���~JOp�����(���T�@�2H N�8�H N��S�
�S{x������-��S������@v�	d'���N���,��d��Jv��
��N�E��N_�n����U�	�'Y��� B|��mhI��� ��4�j� ��
�U�JB|U���o��n���Vge��h��~	�Z$�)�]��`��=�#���G�K�	���N�G`7��@�?m�I����@�#?����L������h�#?}I�#?}�����I*�[>Og�x3[.�h��\%h��IN��kR�Fi��	�o��DyW����a��BZ+c�����mE�V�ODy['���c����M])�T�^���Hg�	~QW~���&r���\� "?�����H��_������~�E�Iu����Vc@+���
ceUC���XJ�'BA��)�����jE���V���2Ck��F�Z��4���Z����Cy�Z.�9-4����S��]wXUt`����VU����u��Uo���v#��p���dLQ�>���l�n7�q�w��l"�����IB��F�
�Nw��iC���Z�,��������p�P��h�8�j-��������cb1�t.,�uM��:�@�l�lvw:�p�:�u���[P�����
�OA}
�S.���	0P�������)��.	���=�)�OA}
���V��
��{�=��`��Vrg�����	��y�)(O��Ay
�S���<m(OAy
�SP������<�)(OAy���7������N���w��bL��v�;�{D�A��l��Q��SQ���G�T=�Eu��*��U
��*0�G5�.�Q{+A����G=*�QA�
z�}���i}�Q�V�*�L]�[14���������2
*���U�i�hAw
�SYE���t��;�)�NAw
�S����t��;�����A��t[����?��n&������j������D:�����s��nV1z�W��}y��=���'��|���u��N��,���t����BRY��/�g�	Z>�U2O/�zw������n�� F��f��h�sG~���������k�[�zr���A<���� rIY��������o��_/>Q�0���U�<�����h0���=3�D
���#���o/���������p���������P�l������Q(��"Y���v��f�z@�u���|��Q<��'���\M.�/fW��,�t�&��/��|�|�rI!�,�/_.)<�rIaR�?]���zy#�����Qu�q�s��W�H/����[S�

�c*�S�CkR������9��r������u�V��"h�l���&+����I�����F��q���8������Yp�:X�g�@�9y_��w�D(�{W2�w��Y��#�p���0���0���0�;Ix�������M�A�<B_f�'�l-��1.����d�,��:dMox,O7�����-�m����v�@���3���Va�09DS�;����P�_��q��&�w'�z��2��w7_
��N:�F�e�;�P_BI���p4}����Y@[�g����?XR��h��d��,����R�3}O�x\R�V@Y~�����3y�$G`�"��J��3yK:��2����o�����-.�B�2��b29
�~��xWx<�\�_OI;[g�JD�
�;��:B�J�I�
���~�v%j���Rc*d���q���������
���00B6k���%R��M�����PAg�k����Gb.:>��"i�B -�e�������\�l^��]���{��9^���� +������}������.�����r�W���T���>S���T0f����D^f���#���p��r�R�r2�1V��{�#td����������{����DG���!Y���4VhJ��g"
H�\��T��^��"���JvI ����*6�`��pd�S�)K�������_	�����4l�s��`��@�^�}i�\����9%a��ca�l@���=c����gN	S�g����Z;������)�Ws������Z���h7��l��!�.e��[T�7�~1�>3�s�<g�o��|� e.kO�� C,���)�5J��o�m���d}x��c������$�����
��*������$�k�{�� �����f���?��u��ty�������!��t�Fn+h@.ZP�\P��}����/['.
)�:q�}E���F����7>�CfL$Q������')��%%�*2e%�0e�#Y?�lD2J�3e%C��9&�S�<;6��[���L��Q��!fw��,]hv��\��Kj.�p<��$������������=}��$����]LTn�NM�6DRg!�h��&��'��Y�a:�E[L��D�����;�p7h����i�q�� ��`vM��A��d�����LGTZ��'����T����Q����z��x���t{h���g�9���o�o�M,���3��������m�{�a�m���`y��O��Sy���J�MR�_�>�i8B�����,wZ��#u2Bz��]����O�� ��C���$�=:@&~��~;|��������u;o�tCy��
�G5n����	7\���u�5+�������4����WR�OQ����F�xn��/��B�rrsq=����|��46}��<���LRIIV ����Q����C�^3����nG�����z4��\���&�G����T���k#M��!M���psm~�|��^��^Z����:���#�i�eQ�0�A`,g���bI�/W�(&�G0j�=�x9��~$��}��!�+������=�������b�������EDR�PP5bD2"3�T�&"���R1�H��bP�g���K��G;�j��[���O�������j����"���?y8j���x$����b��������'���X���1	*q���>��($��[��q��q�G]��q�
R-��)Y��%�e>���86��+�K�9���X �H���'�M��N\�}�[�c��WLR�����`eq����V��}�h�+�I��Q���
r��"��F�
Z<1�`�Afg�����6��%�q���1����0����������8��0��Q|%�Q���{F�$'z��������|�Y�:Bq�Wj����3���V>;[YS���o��0�F�@n�X`1���U}�,F��8��p|`1@���
`.d��B7N`.vO����	w!��
��/����,����b��	,���p
cW�����*m
�������Ck�/����y8z�g_�_��J3������nqa�I>�����8���F�\��Pr>������
������]��f��&�J���m1*�����#"�YI����#
�:�Gt�.u����	��l��:��4Z���e������8w��h&(�����*h7������#D���o�S�Od�����j6cVc���'|��P^� �j�D(g�JX��={����n��1����L�������vi�1)�i�7Y��E�����,�UP�J?���,[�:��C�0M�D�!����0�$���i���1��%=�?��(�� |�:�SK�� ����\%aSx�}��}��A�/P�=K���$QP*j�o�_8xG�r�)�=}�����{�q�ro���{��-v��@��%P��(�(�@���(�@�WI}R�����{��k����^xL�&����@��=���$���Z��v����{;�. v#�>��
`1d�C7N`1vN����	��I��`
s�o�Y��������S"�.v�]@�F�_�a���^��'�;'G�a4�0������6�	���-��P�x�8�o�?B��.x��T�f��*8\����v�]�pU��p�����U$ut��w<�*
����r�8�7	�{LAP������������{;'D���o����P��r�{lOA�g�"(�@�g�V��ww3P��U����������t��7��b���U�6O�#yM���������Z!i+��2.3���x�P�VDke��Fy[���Q�rA�l�"u%)��*�HPQFm�(�[X5�Jd����U�uG�m��&�v�����f��U�a��C4��h��&$��N�+�����WD��c�[���M-��.6{����O�����t���4��Ry3��7�.)���B���nK�w��x.��w����-.��C{����B:-d��l�����6R#�Av���T
b��i��7.6��l'b��k�����WA���n��A�.H� u+�R�aWA�6�H��j u;��$u��B�.��t�=�XH�V�X%��A��dn����s�{o���27��]���� s�#s�i�	D���s�m�@��MEn���Q�$�;l%|>��s��,��-$�A�Vt	o����m�!�%�@�vL�)x�i�	����^��k��U: �l� r��-tDnmA������j r;&����]����MMoP�����/�f����l>����i����L��6�O��������D�C��>�
O��E���	�����U�V��1A����l�^���l:O���d!���,_������-�*������c��q������n�� F��f�wN�sG~��������x���v=���� M��Ka����}���k��������:���U�<�����h0���=3�D
���#���o/����2��+af0s0)L���
�������"p~V7���Q���#D�<&�do��u��%��;9B������"-O�=��\��_�GR�o���X6	�|����/�R�&[��o)<�-�����������7��m�q �q���
bi�I�d����Z�$U����|u�<W�]+:8c	�N��- KfRU< ���p�*���b���W8�b�0`��5���F����
����m	;��A��������r�R�L�$F���a�qLS�����	3W�yr��a��!Y%������z��f���Q�M+�����'m��>�X��@'�U��F�;'�\��2V�u7_
�@N��F�e���n�r{�����nG^[�����6_4W1_����Op4e��1p�4_}���������f�I����H)//�=�����U!%h�e��H{�=�7��w�0�,�p��<����>_!���,�(���������B�_�<X�V�B��3K*1�"2>��;��U���I��R3j��|`D�W.H���2�g�+��&�6���'���k|��]h4[�����Kv�����%5��h���Y��W~^�r��_���'�����K��h���J�-�����,���}?s���,Is�i�(/;�<��?����B�e�%O��=�����\� +������}������.�q�u����*�}���T70�<.����x;��Y���/j�$��g��%)�:OA�%Ec����)�0x�8�+�O��<�+��`@,?����N�a�������3!R��2<�r��B�jx���}����L�#���[�6`�j�}�+��9E������-go��J�j��?���#~�eo��H�K��5zX-�S�8[<^

a3�G�����k��9��������T3�����#`�/]#��N\����|�E������H?���6�������Yv���Kf<�~KF���)��x��G�2$X�-�*k�x/��2�e���Qw��X����}I��9�k�9L�,����7	���^�����"9��='E��|��0]�t�<���ep{:]�Mu�(
�E���R����V�e���!eq�6��`y}�\$�Q�2��[��LOV������]0��LYI)LY�H�O$���Vo��J�"��rL���yvld:����+��
��kC��H�Y���.)�?�>�\H-�5��m��F,1�h����`�1U���
��B�z�Ir�9Y:�.�P��dr2�t4a�!��]%�Mr�.�����Z�������A&3�s�������	4���������������6!��[�hm@N<x�������<���G)��M�������!�&��9�,�9�,��9��UR���T2D�!�5��-�Jo7��!��d���r����1�d�"C*j>s�I��������I������!Xn�,6Tn����q�<b�����~}N��p������Y���%��e���t��n>����������!�l�X��L��3 �d�w-G�#!�i+R_[�����!��%}9o����r�0�}��g�F}�p�����_�J��j^�����;��������p��5g$���M�	�f�����\\O.9b�&��o<��q��?��x��$��"����Oc����1D�5��^���v��l
���2M������d&*�f�T'��NX-�v��&�1H{�����R�!v�����L�@���o���=C�E1i���@���=���g����7��s��N*��W�;�'��I{&����g�����sTC+6QI9^�,��~����dZ����2+j��*)������O&�Rx����/��?��73��������n=*bw�g�C~�r^P�|+v_Y9N�/���/)QxQ��V�����^\���1�Nu���>��],�(}�[0J��a��au
v��7H�d�x��d��� ��A�����
��2,��
����>K�y���h�3����'����N*���$�A�����,�A
�$����op����_�[4r�L��q�y�7���0�	�f�j
<��09y�� ����qd~���-�Ko�a�(��5l`H������T���7����S��dn<�UM����A��!��
06
��!��x?�~�����
+��[lx�Y�N��;�����&+�`�!�`�
�3C�%�~�a��@����V��a�h/4�5:/����
��{$�?�`jF��m����=,�#?��'YC�1�����������"��'E���$�OQl,u����]�(���R�c��^����fK]��t"9�Ir���R������>)�	�F��@J�����i/��vqk����+��j���V��Xm.��i��:�L9�2��i���^�k��"m�_�OH�v�P����[,R��5(�@
��4�T��(�v�	��W�L��:�:��fvp�������1����Hr��z@Nl@��l.���~���SI���|1���
_���V�������a��4�����sh�n������9������ip�|�0�.]mm�Xr�4u�ud����W��"�=���)���u
�<&��0���mm�X�4�GhzQ�^��m��5��i�I��@��"��A��
��
=��@<��>��Mu� ���$!x�X?�z�M��� 	������@���	�=0��Az���
�������2�8M��'�#��X�;����@z�������A�B�����{���N�E0(�A���+�{H7@�^��B�`�!��^�v�����VX`zC7��	/�kA70�C�+��@C~�[�^������N8�-6�N�d8�]pG��a��cp~���=�~����4��1����H��c[����������p~�(��������.b�=0��Az���
����8bu��Az`����j���6I�VV=�u�����*��A�B������[>Og�x3[.�h��\%h��?����h�b"�����������j�����P�VDk��T��R(Q�B��X��`�N-��������fG6���N��i�������j3n��:�]_Q+���jYy}�Y9'&.4
����X�^VQ('��@�2���Fl	-�
v���,;�ZV6U�e������k4�AK�� �%�*0��T��`�J -i�doY�����iu�t���Nr�AGa�����9|0$�DBtr�s�>;4�  �DA@�)HY   ��x���0� ���Z��#�[�k��Y��Cq
A<��
�!��9A<��.�x�a������A:�����}�A:l)�~TL{EF[��2"�� #���dD���� #��	�\>���s��x�x����2"XA:��C�A:l�CH� ��t����7(�t��t3���E6��W��4����]&��|��'��|�^��fF���xq����	z�_�h�@�x�� Z���h�F�%��\�g�y��5� �"I��f���A_f�'TF��J�;�������7gh�V7#�k��;�3D�L�o>�|��o����_/�h:�^
�P�����9�����&��^|����D�����#�Gc%D�"��"�	`<�^&���#�?�������
��q~V7�S�Q���#D�<&�do��u��%k:l��q9�=����aO�&�������m�i:�e�����w_&_�LR���l����c5Dq��U%�Ja�t�~����������VW7H�u�����������_\\e�A��C���w�>f�E�������o�E;V��#�NdFE)�g{5S�R�U���z�Xj�S��KI�RJQ��V=����3��7��.\d\=����zP���y=9�'��������6�����-_7"Q����r��rh�<$�dq���82xC7B��6��o�U���H��@4g���m���
���7��i���3�P��Df��h��d��,����R�3}O�x\R�V@Y~�����3y����;Z��Y��W�d�����f+d�0����((K1��[\8�+��e���8|2�(������*EC%bo�CU�t6����~���Vs����R����O������?M-�0�?4k@ �
�37F��/��/�E����#1��r�4W!����eO����)���o�B�?���"��2���������j���Y��������KtS��1�����W���T���>S��>P�'f�B�
~�e�Z�^7v7]x��iSA�x
�Z�T����*�&�+hF���1��J
t4�_^���4��}�F�~&w�H�F��T���eC
���K��K������1=8��t{�R��fg����+�}����n
|�@�ga{Y�.�����F��sJ�f�G�J����]i�c���[�,�S���.g�!iD=z�m��|�f�l�u���.O�lwesL?�.Q�4h��R5�]}|��8fEFr)��BG!s<2HGy���HI1��>O�QR�|�w�l��^1�*���s����m��nynone�s�J!�NH~��1+��e����')Y���[e���R������H6"	�^\��E�U%���yvl�����
�+�����C��H�Y���.)���'j��#�>�����L�O�����9t�e����IN����� 	J$�V����@� �R Z	D��W�fr�3Q9���x�OnR-�G�}���������
j�yV,�M9V,����������Q���Mm������m�7��z�!����/��������Z��j�r��������`�� I�vN�vR�n���J�MR��]�>�Hc8B�����,wZ��3=2:D��/�������S���|��$�!�%��3V� \K��='��)x�a�}���'�p9��������*j^��T������q%����5�v%^���Z�sSoe��-�#������%������G��C���y�7k�"�@X1�w�)�2g�h�fT-��;��V�����D�j ^_%�Mr�.��tI��+�
�e��,8�����*�t[���8
������`*�Qs����+��bI�/W�(&�GLh�=�p
��~$��}����+������=�1�bN�#35vE�]�)5������f?�o�����Ex�J���������'0(<T�v���O�?�D��i�Q��eJxT�!��3�!������������_V���?��+��/^����@�[�_������F���f07K�����-��yo�EI�y2]�b�\�RU���G+D��t,9�hk
F�aIOG8b!1�^�}�[�*���~WI��=�Q��o������/��tY#�Z
�v�������5�l��a�8o�P�� ����7��t�)���U����`�4��U�X���H-���SK3��NW
������v��>�.�2����Y>�&� ���
�+����Nq0����U����IVA~E.����� ���R:���m~^��<I�����,n��]j�t�1%��gv��@8*A����h=%�-�*�q��������\(��]���������i���P�TM�������6%�gMI���d�����^���"�8x	����8hYA�
��A�`�vAU������oH�*�
6����*�[���?�Aq%� �?�C����|�:�-;�A�<��R}�QSBmj���y�'��r��<���J������W����?�~W�x�nW������]���{`�JTm0/��e�Mt��A':`P�R`'t��:
�>�j	�����	B�#�tv'tl� ���
�+����U0j]}�Vm�_A~5�
��c�@h{����6�����B��N��� ���
�+P�FT�(��.���5��K6�d� [��������.����3�,�����?���S	�]{6�@%*��$w�$���mP	�U������2����t��7��b���U�6O��yM���07c��������0o+�BE[��RQ�VH�D��Tl2�3%Om��b��Y(6B���Tl�Wkh8��N��)6�����Vm�A�[6�K	���^�Tm��?,�(�D��1&@�2F<�����88�5�U0]/6��7�u�r������
��� �zI0�u�Vn0�%����*��B����v�QKa�����|g�!{
J���7N�n�Gb���� ��t�
H7 ��t�$�nZ�9�����ZPr�x���Y�dq
A���
�
�6�	d�m8  ��u��M3� �����m���� ��i;����,�D���p�a�TE��fg��\r������b,���o.��ew[ 
���B��#�t@�a��2; ���!��*;�s��xhn���Y:@a�8`�;��<�q@����2�6 ���x�cN��h����_���/��|~_������v�H�t�yB/�j3�����	z�_�h�@�x���/O��'�0���{4��O�������(�q+Y�`�����4����b��-���tN�&�#N�&&����,��)�/3��2|CQ���w?^_Ln.�9F�wy3��,��#?C�������������\��b�����0b.+||��~���mr���'�i~�p�>��?���#�Gc%d�2������������;x���0�n������r�'����Ja�t�~���
�������
�AX�2C��Q���#D�~L�*�$��d5K��e�N����EHi�#���]}�A������U��d����f/#:�������b>���|�M7��)����o��|���B��zf�/�/_N_����������
�����$�����/�v�n��ml�@���}%�9����e/��eU7dN?�v�=�P��F��������*%�WM�)�\�N`?7��uZHX�R���E"3)��(���Yi@/�7k $
����8t����=��H
�B.S�	�� �r�=)���e'l�24O6h��������_�����C��!Y%�����)1t%��y���1�c�T�����7 ����7���X�8Mb�>c�{O����WD��$k�fy�������{R��������}������T�����X���e�$�L�f�[!���LP�(����p�W����i
�3K*��2a���D�F�����O�
���������G��5���!��r�9]d4��C���W rK�l�d�Qf�t��4���M
��00�7k@�������4�,u���M�sK#@�~$���SY.���!�����I^v�%zx��EI�e�%XZ��������>
�2x�^�>�������O�".�{^w�R�g��Lusd��a���D4��e�Aj�{��y�u��&'��"�)�t���';BGQ0x�8}-�O��<�-�`@��?����N�a����Z�~uXVI�[��"6��T�������u�Ut�Rt�Mt�Ht�C��gV�����8�T�,�Rp)���%������A�}�P�����"�gV���|��h���}�����O��9-���V���m@PQQ�������_}�(U|��<a�R��%a�`C����K�� ����Q\�����m4��4ZhrU
|o�f�&�(f�NF�,E����b!-�A:��I��$s����%�?�n�e|�+A�A��T�
0�$�������A���}:��M��M��M��]��w��+#��H�~���\��ez����d�B�Bz�e�JJa�ZG�~"��d�za^V2a���t>%��c#���U4]�T��|7bvG����fwI�L����fzd
�i�����hrV��xig���_vV��xbgM���a��60�����TFrw�/)�����F;�`�c�h��t��N��hL`���MGT��OX<J9n�-&�G�a����
���$���a�������]�����x��k�uM`��	����&��3����d���5������1�$5���s�B�#~���r�e.9)-������?>���{
J����� M��I����8��W+%8q�N���^�u���������8����}��.>P��Z��~:>K���������g���?<��![��3��C�k_�s��A� S�/'7��K����'���@�!��$^o2�'�D��b>��X�(�T�z����w��"�������@�t��7�=�\._�%9�U������<KG�x*��J"lq�6���D��+iHD�B"�B(���*5�h*�A��$+i�����j{�T���V��g j������;!��n_I����_�'��L�s2�Y�*����Qa����$�x���3�I�
S]�b'���w#��/���?�K��:x�c���\��� :��8���/����������=*����54V���r�8_����_R.���:x����W��?�7-G����U.^��4��r��lv�������^������ ���i[��w�7����x&��t�� �g��79�Qa1X>�e8���	Bmg	8O�9R,���:��������i���`��5�T��p��6������q$h7���j[�!���2��F:�i4�]`�g���-��7���\lj�9e!����U�d�����n��t�^���i� P���A`�J�g���D��E�`���M���T��
���M�`�����*0E��S�����)��>O�O���s���"�;�P^	��.v���]�C��F��������MvtD�S�W�����)�mx�LQ��x"k��2F~r��ds����)��,<[�uk5e��y��pL��A0�&@�x�L���6�*���M�1e�l�"�K�����y�'�l�w{���y�g�l�w{���ic��E�~��	���$=�I��y����7E�st�v�'������I��Aq�P�t�0(NlnGq�?���^rCo����vS�(��I��M�YV���Y`�����G}w=�D|)�L#�HB����+Y��,�_��C����!�������Y��@-�(���C������3%�qx��8|�NU������/~�������J��v�0���
��u7����{�^������Gl%`��h�#��w���a4{��c!��f�Ez���N}�fA��=�]���w��a�]mn�w�!i��X����o�m�z�SL������]2E}�����5�/�;�P��C]4��m��d����
�Co�tD�SL������]2E}���H������V�N���@E�pCy%���h�!������G��%7�v������8�����Q
{d�������	A8!�B���N(8�f4{��V*��E����6�C�|���f���~�fOW4{����=��a`�l�!{�w/0���f���]�����������t��7��b���U�6O��&E�ez��}w�a���ZS�me,a�h+��BZ*��
)���MW�g���MYr�d$�'�6���m��!�c�#;6<�c�������j3n�Z 2���L�*%��
���� @U+1�u+�&~uCA�
���j���YE!v�s�,���5�Xh��kD������i��l�cdLw�Z�u���U��N��-@����"������Vq�f�;��l�Hc�*�K����TX����
��u�R!���Z*P�;:�[P��qq�\�c#��sr�s�_#*�� ��q�}�]����>�� ������>��}�Ei�ki���>��%	�ke�Ya�}	�A�a_����>���@�o�M���(���oZY��,�n�K<��~K�*D��W$���C� �3mU��A��$�V@����?H��v$�}3��O$��k��v�?v�P2K�)��ud}���J ��Yd}��A�WYd}�������0��(��t��t3���E6��W��4����]&����������u�A�5�B������uq�F3���	��y�0���63�����I�-���\�g�y�����/&7��A��mFx��?w�g�~�\�|����#�[����^��t���Ue���s��?^��M.��D�������Q���|D�������/��������7����O�7�s�����������!���d���Mr�NV�d=8�����C\�~Ziq�ma�����o��l�j}�	��b� }���	��#�W2i?O���e���8��n}�	�]"�;�	�	��-<=B�=]�o,?bp{�U�=��c�i�\uT�����;��Z:��	^�A��G�8;3�f\��G�l`���J�{7��������DN!s��������y��A��M�y�0!�D���r��b��<$�dq��}��G����!7P<o��u�}>^&�������
�DhI��w�.��K���_Q����Q��EJ�\���
�R)p�|R��J������H)�+��RW*J{J2P���#s�L��c#p�U��8��A),����~�������0�u�������H�L
p�#:��9CA�K!0��Yh������h��W�0��#�)T�Sa<���"�2���������j��FY�������[:�����q�s���x*�}���T7PZ����	���e�Aj����l�]�����@.d<��]�|����!>�+N���S�=���y0������nHVc08����bU���!/%����f�-�b�+�|p���'%�?C�9B�14/R��.C���Er_j)��a�|N��l�X�2��f�u����L�?ENo�qV��F�r?�G�o-�g#U�Z���R�d�+�c��v)��x��U
��ZU�`����G�2D]L '��5J����n5[|�o��^(e�VY��7�9��k�Zi����8k�k����_si�g��������@��|����}Yb�@,��|��/���R�P��1�b=���J�S�<;6��}[Ut��N��Q�� f������%��s�YM~z���%���)�*�����e�*n�Q(�{;�b9�(�x������c�����!����l��sv��q+?'������<�-���-��a���&/LG%v(w��\����;}������e���j����
�N>�O���bCI5U������1�$5J���s���#~���r�e.�������7���|<�h�vA�'�����c��w?moudt�����G�
7���I�M�_�
�O�k>��?�SW]?<}w>���,����yxvB��� ��W����G�RJi�rrsq=����|���(s��~�'�z�16)����y{g��]8e�z�n��;��VTk��m�Z1����&�G����$nZlr\����(sz�����T�-4Iv���4Q�#�&�W�����d�LZ3]��
<������a�S�Z_I���i�8��q�S������S� �_F���d�(�P���8-����aQ%�K����|=,�j�4N�R����)�8������"�	sj���|O�	�[�_����
������v�3H���ao�`J��*�CJz��3�����f�1�r�* ����O~K]=��;�r?sclj�E������R�8����aKF��[�%�#���73:8��C��,h%4��9�0��eB�!`�W�k�,������r�w������@�m��U[WT����C�s��Zr+��D��O��� ��a�C��bp������T]�U�����t��=��<�rX���N��e2���i�3E��N����'G�����r��T���������m��o�NQ~]�U�[��7g{H�8��rDJ4�k����m��
�=PG��)�����C8��Fl���@�����Y%!�i�>$T=���m5������'����+u�@]+P����|u-O]�S���`_!5)��Di}+���]�b�n���9P���Z?A� m@����@�@{*p��S&�8�xi�)�����"�o=@$}���j5d@�{r�������~���������qK�]>���A���������cjn�}PiR�05������8�w�@q��UN�C��i�>$1hOb����A���)�����!�8���1��}H�5����Jz�|���f�\���y�J��	o���52�G�M7��������B�����P�VDk��T��R(Q+4�MvWE���uQ�4�Bl��)jSl�l�p���d,V�iv��*�[�D��_��Y!�����P`W�zv���1PVQ�1&���@���FD�hreL'�M-���nY.@��Oi���*0�E�Vn03%��<�*�������q�;:����=>����-�;��<����&��,l"��V���&����+6�@���s��Z)�z]"0��$�L"0����Il��~3�o��?��E�myD�
�[:��^��V���S���"p��)j��������2/�Nq���D�[�&v�)�6xDm�<"��@�G��G|�\H<�O�A7��\^d������y��?�2����K����y��M��������2��&�|Y%�5y�����N����h��o���s�����z�?'�>y�_���_f�]���x��������)Y��p���fx�K<��<A���st��"Wo.�����A��:����_�����������I��-��}�����i�����x3[<��S�-��	W����zC�<'h�ao�h�����S������1��A�d�B�3�c�]L�x���#��Gh�L{�2�f�����r�,���w�3[�~�����&����~����\���L� �7#���?w�s��{�WC��������]����������{	���)��=����5������O�HW���c�����O���o-�`"�s��H��t�~���
�������
�y����p�l�f�(<����%�1 ��7B����7�fH��c�HV�&�]'�Y��G(���:�W��]i��#oLz:��\���vR�q�����u�]����������u�����b]|��J�Vf�Y���X3�f(�������8A���C4���w�!��$�r��m"�n-��/_5�}�|��) �}c�U�u���f��t�0	�-0���Z)�(�����W��P8c��C9E�������:���"����@7%�azi����T���M��
8a��GQb"�Jie���P.b	�S�(H��aT)%m�m5����������'X
_!5:M� ��M�_������)���Q�4A�,���Bs�"N�'����|�>&�D�@j��D����/����>��7�~��}8�?�ysvFH^����AD����9��i�|�o���@���&��@������>����>�l�4O6-�xFea��w�x�����������/�����|�������/�	��7�;
 A��L����%0�+r ��|���>��n�
�[�|�d���j��<-��1.�V�C�JwI	��(�G���z\'�����]�Dhr&/49��������YYFv���	�����Kk�j0c
����Y�)���+��)HV�9+��/!_[Dhx��f�>�����44OG�!TCR�y$/�C�tB_���%�W��|H^2����3�����#y��}���c�������m�H�b�E��-�����c��e�h�����-��J����}X#�n��%���\���O�����
��A�%�������?.[$S'���S��yV����[v�4�5�|��Zb������"e�����������=g9������8�D�����%���
����22�a����E�c�p6���Z���,DW
>�����xB\�}r���?��qT��I�!�������#A5�2������g�Wk=������W��rdk?�JP�
���y��-����li_(�E�����4����M�q��JJcH�o�oR<� �f����oq��H�p���I)��u1������
�����!���ww���c�����=u
���kz^-����<���27����3\��A��_)�w�R������%�%KT/iI���,��x�|�%�]���F��D�d��%
$K4^��p��%R�qk�E&|O$�=F�2����)�f2>&{��%G���ir��:y��d�u2O�6%39���<�����)Y%(����dU��L�XrG[�X���&-W����KhJK�M}S��ls"�l���&�G�����x���Ct>4�p���E3Sc�E.�:�NVu���
P*h��J�t0��
���	��O���D0�AA�?A�d=A�$;A��8����D}�)��n��m#Q�WH�o�z=C�~�H�sD�[���b�4^l��K�
���huB&�0yS���+oj3���M;��c��[��,����c��cp��j�61���mb�~2��F��@������A�B��kU�F�$^onI��������zrY�RS��SZ9��C�i��&�Q��`J�H��Q�����}��]�����|��C�^3����	�=\5a��f�,����(�D�D�>y|\%�x����u�|})������O76�N\����o��a��=����-��]��<��% �����Q}R�F=Zm����:^U��(��5���:��e�;
{���j�l�w������(������j����������
oj?%E�O�n���OV�*g^����^+�Nc~hfXz�e:��#����'�'[�������yVz�Q�Y����Ti���<&�������zg���1����O�a�D��v�X�/��9����s�.ai���+|+r���N�U_�AS������xz��3E
�)Z��Q���'��?������}TR��f��e����w��&�lL��x�i�G�1�-�9�?���c$u8���4��$��w4��U��N�r����_B0� ��kB#�m����}_�z���_�9���}c�
������Zj��t�����g&� ,_i��2a�'�m��m�n6V�K7]��Z�32)?�}*m-�\n4;ti���d�[�V�	��v��qL��6B�b��yR���H k�|�f�[�2/?����s������@��;\��Wi�<�9�$q��&y�������g��1��Y$���yt��ip��0���O�w��-�f��\8����'�d����<1[*��X�����w��-����S.�u��%=�u���l�d�G�
pZ��w
8z���R��,�X
8����e�c���,�pq�$����L�@O��S�+$�c���,�����lWH��5��zH= ����h���RO:H= ��RH= �������c�p��e���U� w������z�"W#Co�u|��z
��\}�?�ySD��r}?�� �xZx&"�S�H������r�2����
���#�
S=d��U]�LGK/�X��5s�M�1�~���WN;�6CD�v�B�ABPN���u�M��mv]);�����{�������t�l��-�B����7��������M+�lq��X�|]�V��f����~3V���o� ����P,6v����?�K� u0?���9��G����MS�Q��rjQ4(et���P��%��Xs.ix��'����L�@�h.Y����z*/z��y|YQ�c�K�����'>#~�8��J�$�;���Y�A��'g�y�K�21�SFR��>#��y_�K��d�}�.A�3�_�w@��s�=3�P4*�a�{x�7��x��]�>,��gB��,��n*�5`f@<svMk������3�3m�@<�I!�e7?��g>�g���xvV�����Z��(���bZ��,���bzQ�,���bZ�(���bZ��,���bZ��(���bZQ�,���bz+/�C������v&�Z�����E�ve-6�[�jEqv]���s;��|���(����VE�q[�c�~��>��TB=4wG1u��Z=��m�7;��b���~���^zT���r�9��^`�����)/Su������R�%7�j1����AL-�l�:��YnO�e���<�������h:�e���@'���s{c/�����,fuG�,os��,���:�V��{Y�%��b��$7S��Bn��d���+�Im5%�M_���jJ2"��$'��K�[MIF^����5}IVZ��d�5mINV�)��j5%IM_���%�"��.�?��Xq�u����S���?2�3�B&x���m���R����9
F�(���~-H���24����n��O���)s��(�R7I��MV>a�q�<�!E��G�����`���$������
kb�����)�������o��h���/�M��xj�5������������U���s���ng�]��v����W�s��.�����vv	&��vnq��l����Au;��l�k��f;��\s���v6���b;�k������� �F%&���,`�ka�E�k~����b�Y���XhV,2+vbV�����Y�s}1��Au�j���������7SW�f��b5����,������;���/����[��"&�C�����e��R}s�������9#{*���.�Ro���F�{!�(4%�w�Z\�^u������hCy��x&6
���h�]&{��������U�b�Do��z��n�m�B��YFCU�1�N�������	�����������E��k��K�u�xe�P�X����+UE�B�)$$0�Y�$S9	�H��,/�:� \�2DpZ��TT��@d�C�����6>dzH�u�)G~��FQ�C�b;������\�	�BJfEn�#qs(l���4�2��I����64����.���nO��
�.�z���F�.����n���K�.6��KB���6�s6�����5�.u�\��#'�|Y%��l�X#�
����yMV�B��sd�}Z�9���Z��~J�	z\��(^�F7zvp����%@ws��x1-�����#gS��J}�ww,Br��E���Oa�����+����$L�'R���$z@	�h�z1����5r�F��Q����|����
����]���x�lc�j|6���~�?������V�*g�24�����}�e-�S���O�I��Tf����N���*��K������*)x*5�J
�����Oe�PI�����`(5�J
�?�Ozu��8e`V��o��W�rhSXtL�>�)|jS���0���3��G@OY�"CW�-��|��!.��4�0�4�@}i��v&
�9r��	�C����W�k�Q'I����6����'A�h��O��NJ�?�k"�������M����O@�l�/;5��V
&������j�	M��,�6���][=
����-@�m:k��
��\*?xT5i�!�[�$��`��LC�Cp4Ip�Bh��6�E��	R��rf�LBJ>�S���'B�'��*FN"�S'��\�k�ya4�BE�+�I4v��4J��y�r�h|�\5���M��h�W���{�gB{����.��,�����2fN�}'9*y$;y$;�x$;�x$;�x$;�(��+��8�=<�2<�'<�<�<�<��k����*A���a��+��m���8;�S�T��vU����r�a�q|Kc�DL��VC���&����k!��������'��+L�$9�0�BM��|�������b��9;��FX	����BD=U|G�$.��yC��/�����B~5Baq,]�_W�a)���/x"E~����qE~ji?k���)�	����&�S����K��68�,��A���]*X�"�`��6�`%R�Z��6������������q�|u93iWR��(�9�����z������|Y�l�������|SmD��p��&_yye��C�����v���m5�e�{�ih�z�EB���G�Z�r#���J^X�+�a%;������#V��J�X�+�b%[�����q%p�<�h%��U�����]�g��g��lq�A�3��^�rrsq=�<������PyZ2uP�������!��$^o(:H@z��g�(����8�$��z�5F�F�f�~�[���-�k[p��r�5
�)��I��������E��`X	i��Oib\PU�i�E}4���c����Vc!�&+�4��34��!�+����^S
Q3�9�XsI�2���`{��m��Xm�iq��-9,���?���XgJ�&~E������l�m�u�9,���;���X_K�&V3�A �p��W�'|%}M��'��,���GkA�S�E�O!�Q�?GB�����A�s$u��	��?GB����B�s$�PMo%z�jz+!%T�[�3���J�	��V"R����B5������]@���L}�E��D�&��u��8�;=�+�c�
Y�L�$��,�y��2���L��NFh(�����JE�+�;��f���=��V(�K�L��XR%��V�+~�%agO�
�q�C+����28��\S�����C��>$�AA;�}��E�3��l��n�����z1�]��8��=�g'�bb�������s]{����+����v��&��xDj+�Q=b�R����DK�gZ2{�
V�U�<��WT�������;�u)*�������E�Q)�E�R���o���Y��R���r�\�W7)����4�J-	�1r�N�����W��������b7���K�zo��wC[��&X�Z��n�&���mx�K�dsfKI����{,?�0'��q�;����f|�Ok����%�����M_����X
h�8�7�o9Vc�B���������i�-�O��7�����������^v�_�^~iq�%�[���*W_�Jr7_��d��K��^��dB�kKr�^�K��^��T~JrW^�K�7^�K�^jKr�]��dd5%��${���d�xC���t��cj�8�n���X����L(�Q���w�),1r�V���NF��t�zM�����@z����3��������������~u;��l����s5�����wv��[��fJv;��,��M����\
]���hi�0��n�&s5�����D����r�� /��C�/�g����(/����
����#l\QV�O��E����q7��/�l6f�qEYy^<���B3�����xI�j	8�^<�=�US-'��������j	8�^<q�e�z�"[�����ey���oV,0+������;5+vfV�\_��wP]�������.V3o�o��X���-��	�����Fhy���l�p#4�
7B���p#4���F������~�w���lo�i9�9[,���hNC/_��B���
�n����
U����T���Z>���L���x�s>�Bk�n@2�f�����#�J�Q��wD��n�������*j�8c�5�
��������i���/W������y��	c-������U\#������<�k��W}'9���*b���0w�Tcp������O�������C����� j �:lq3k��������KT���OG��d���=��P���$��~��^g��
z��3@d L�/���e��^��)�?;+3���ATf����e�iJ�O������QJ{�e&��s���$u��Jn��
�Q�
D�� ��#�������IckD���T��@S������=���>��H�-K���@^��ZC�j�k;�u���P�l{����Mq�
��k8@\�L\�G�CUY��]�J!E�
�PT8@Q������CL�:���2�)/N�N)�Xh�;��+�����Z�>��X�7>�g��
��~�<���n!�Z!�+�"Z!���B�W���#��
dvhG�l�0gP3Qs�Lw�����tg�\�4j�>�~�y������S�w��S1/���X�hT���1�}�I~&���������>�����U�����km���;�i.��8b��!�����F���`hu-dC�Wm<0n��Bc��~1f�+N���z��,��d���_�p�[��u���c��u�+�Wq�����UhsgjS�Wq���t�5����q��hy/-�v����"tx�1��y�whpj��kB��o��L����&h��t�?��R��]�[�Y��������2���eW������.�8����/�w�����e~���.�-�,�K-_ven�g�_j�����?�|^�S%_y7wG;�r����v:gM���'!���������m����+���sBr��5_/��������|���U���
h����@��fw���sD{���_�h���f���ll�r�>;�!�k�����3%������]��2��P���]����������4���,�b�����f��R+�I�\���3\kY�E�%MM�o.������7�����g6�6���	�l��,���lG�Yv�%e'M���
��4�	�:)Diz#��m(�1i#d���`��Z��;ma�V4�e�Z����;L�@�JS�0��n������vt=��|SJ�����;������)1w��jL��1u]5�2��m�����;�S�L�n�S���@���|�^���j����[����>!��(/������Z�)��������>���h-��R���s���M�F���:Vz�����s�����46Q���'��/���ho�g����Ph�W^�*�m���v�eD�@KS�P)8aV8a��A���1�B�U��]e����W��2:m����O�<1A�:�}�i�t��t3���E���9�����9i���v�H���?'S�U�^��fFn�A_�fwOh�F��C�:���h�/���/��z6�'���X�����g�	Z>�U2Oo�yw������f�� F��f�7�s�>����x5D�L�o>�|���l.?|��
��� �tY��{
���k�������GC6�7#��������(��f@#
����������O�7���GZ=�����?�������v@�=o��y�3�>�fH��c�HV���1*O���Ec�]f~�������?������'W���>�:<�����i�N�/�v)P��K��?]���zy#����uV��n��j�����y���7�o��]�H?�N��%�~0r�����&���NjyA|������/�����H?'X�Ame�O*��x��m�VJ��Y�Y�4������Og�3��53��
w��N���}x�}�E_���R�n��fQ#��q��m��)�X��'X
_�it��\;��:!��������RR�=����1j�E-����l�H8)�����^���uT.��2�'��d@e����P�Z������������/�	�~3�;$'�o�L�������#�'G|���#�e�yB_�����C��!Y%���1��0�3<�L}����2��N�I��
:����VN-�n�*��yr9����Kk�n0c%�=� �-~�x��f�>��~w�	��K��A�CR�y$/�C�tB_���%�������#y��,}���c�������m�H�blK��-�����c��e�x�����-,\�=����x��-5��r��W"^=�uA�
��3�U���������>�4n���]t
�J������z���l/pb�Z�����<_�jB�M��1������%4T5[����MF�`cvP��$��=���W=[XLY�)����f��)�e�%X�������Z��~��������������^�>��������-��1�)����%���s�~"�1$R�T�xM�g������h�?ES���*H����^��8��3gh�,`������v2LM���.�0�P�@�%�[�y:�_f����v��Z����:/i���v�B����
�v��<�Htu|����HFz����\d�"{��%�OS�49��N�g:Vt����MI������j�\P�/O���b���5dU��LJ�M�]1�t��	B���	M��.C;g`�i�0c���=���MW�c��{��u�|}�s���j�C;��q�s�����a�z����1�`|��N\��<<��*��Km;NH�7�Q�f�H�?8���i��4=G���O|?�&�;���-�s�����4�����P){2sB&\�j��n�5b���P��N|Y��@�!�;����q��qB�1t�
m��*~�)���[������m�����U����a�6�y=�
�L�_*�=2��������k�6.W����I�������W�rrsq=�,4p�����2U���Qv.?�(.}Li��3J�����Ki�>kJ��h�f�K��;��u�j@��
Y������W")=�m��.��#4���|��Yt����M	��OvQ4��#��:0��j��~��O`����d��S�55A����G=W���84���2��?fF:�EK�F��,C��������;�2��j��jJq:�`~hfXzd�yf�g~h�I�g������C3�J��LT_���19,���>��,IMn��-w�@JMk��$�A�*~Q���5.7�)\i�%�]������SZ9�$�����?.����sR|t�|��V�}��;^"��A�)n0�g�*<�������:�\��e�~$_z<�?��h�g�.&>�A�S�=8�3���Q~$#��N
���3�@�<����^��~��$�C���89t���f� ]Y��.M*����&)������OG��Yf5��J����G����%����?v�3�
*f��ng��9��f.�?�'P������<|�f�sOO��M5����?<{��D|9�w'��I�{������ ��z�<1[�U���
p^7<=�O��X���������U��~a A
8x���R���<��[gxZRg����'fK9d?�U���e�S��;O��r�d��R��$�t,��8u�xr�A���lWH��2n�,����lWH��5ps�9��T	�9����n�9�����������~aQ��.*BL��4?9Q����eg/b���u8hX�����*�1�w��[���3u���*�L8:1T��Mb�X���e%������:�3�V�B��q��a��LO
1C�R����.��I?UMl-��u��b�v��%D�e1�;Lk������]v��3��8{��]X��o_j�;x����/��j����X�#�/t?*yD�o�"�?=bX)��<��%k��\�'+��\�������\a1�������w���=����S,z�3�h��3y��`�P�w �
��r^���y
N��y
NQD(��h���k�3�{�w@:�����a����g���h���B��`�+>�=gl3v ��3����������Mc^�2�>0�u���dl���������s����=c��Y�����mYL�����lYL�����lQL������kYL����kYL������jQL����z����"H�A�*M
�T��������k+l�5��-���PcY��|���9'�=c��:�\[e��-�1KO�|���}��7�k�|d�
�`h�'��CJ�{�t�����NV���K�R���K��y���V����]Z��*����]m/y^O����O��+pZ��:����]m�y��"�v<�)�k���\�������]}I���)�p�����/���5%VW[��t�%YFW[��skJ2lnMI����d�\m������i<���S�6����l=��qeHb��LpF��������FG�p�N���F<���e9����d��2.L��F���
���|�\��y����K��^%�����=�d���F������L��/�����J�?������t
�Yg��;<g{yD��Z~��1������&���`�J�����_{51�9c:�`��&�����g���f��b�Y�����Y�S�bgf����|~������[W�f�*B]��y����[^L��A�sQ�>%��c��;��C 2���>��);q�*Z*�~3b�����@�+����(����6�Hm�{�4N!���n����-��R��/������m�-V(�u��-Mt	���,Ty�H�2�R���w�tE0sIx�� ��g�Y�����m��"�[ESx%�DyO$��"'���
e=�/������B�?��y6�\�JY3�����:�m����
q����Z��T�P����4�:�6�;��]lKEt(t�v�������N,q,�,����`����0�,����F�^,e����/�d��-k������^����*��Z�V~��;��Vx��u��h�D��Dr�Q����_���at�0q��8�x2�7([)e�U�T�W�r�8�,I����u��a�@�|�F&6��X���ui������]b�#�Z7�q�x�1����hF��EM�pe D����2�2�R����L�,)X�D��,�]N��2�j$�VK
�J�����=����S��ZR�jbR��?�48b�.eV�����o9�),�4�Ol
��>�)���g?Q�:YFC�-��|���#�'�pb���7�9'��p"��ey�rj_��e�['/�vT�"O���� �7��:�H�%|�����=���C����x�� �om��kL�P���U.�% ���kn�Ga[�������-@gm:W2�����U5<����%G�m�hD7����%���F����5�s��;����H��_=p����������}��:Y��:�r�x
�H'��Zk�g6u��	N�{�������h,3>�G2/�#�?���s�H�#~$��(�!��d��G2��#�����H�s}$��n|���m*����7��$up D�h��GKl*�P5���T�'���U����aL&��a����$�M�Cj*�z;i�m<��c�@G<��CI:$g$�#�������QC�����
!_8~� ���J~q�C�_WA#���TM�_�q�B�oc+��Q��x5G������������\Yj,yeiw���9S�S���8��z�=#=�^W��������V_w���� �\��U�Jj������E�gl����AD���� *��
���}�����Q�?J�G�)9 %����L��R�AJ>H�U���c�,�9Z%�*Y�%z��~���wT�/�>��������w��o��J���Z�	������?��x��y�Y�QY�>I�
vG��h	�u�����nI�?��c�������S�f������������1+�W�K���&h��8�'���6��c8skJkv�w��%�K�k��[\On�[��f�GA������O��m��V����� �l������Is������S�������O�����<��H�#I
N�8�$�9.I���L��#�����H8�$�9CI��R��#���j�*g�T�V9`�����+��U�b���r>K5o�C[�y��I��QH}�
	2}b�NI-����J����q�����,I�F�f|��V�P��l�0
B����3*)v��$�l+���B�2�T90�������O2KR\����h�*9x���i��5Cj������Z'��Lj��]�~�~������O�Z[��=����@jd��3-�=�
��*s�h����JkF�h]�2���� k�oT�o��T�f�[�/e��������!�uH���[����a��!e�[�Z�cp��Q�[���!�un����!I2Q���Cp�P
�z��p�P?��:���Cb1�u�Rn�k��!��A�����A�Z3[pG�|&�����;��
�g|�|���������b��P��Gsr�b��w"7"�^'��nC��TM�������Af
$7��m�y����-����/O	�&&������v�M�\����Q���PS��h3��_�
�9Y=��Q_��;Y��/���H~A��!u+F�j8�_�U<�'s��������]A�f%����;�L�?�#P\[�Z����+�?5����g
�b������g�M��be'��QD�1������8�~�=����c��������aJ��z�)]=;+3��ATf*���e�iJ�N������(%s�2S��K�J��J�-�F����~��������ko�Wghs�	�1�o�~U[���W#�����%_]a�}�^�8��&tAJ�L;/@����w=kO$3^��I�c@����t���w:�>��W�R�v)���/�����+|��t���k"</� �x-�t�HQ���#�}5]�WW�Bd�E�Sz ���I��Q�p!��
a^!P��
!����B�L�gU���6G�>�pTbAFm(�
��HC������
��!�%���4����7�L�h*x�i�i����Ah<2��E/��hr���g��z�#��a�`sV�.�o�����-�,J�66)!r-�d�0�4����IW-2�uS-�>����4�i#?�'���3�������p�y ������/e������2���{�?��R��n=(�,�K�"������/e������2��C���?��R6�n�(�,�yye�bwz�����K!t�,bp�uh"@4�HG����2M ��Db�0H$����I$����8���:�H��XC��N��D���$�"�p{��=+D���U�>�SE6�����V�j��Vpe��)ys�,��8sE�M�HC?�;�g�|���Q����*�k���4m������p���d����b�����m�tHrnc��"O:c�jcQ�+��d���9c;�M�������c;����]i ��8�R]��=W����b����AJ=)�s����-���lKV3�kZt������QRjU<���Z��vC]���m��2O����m��-3�
��-���n[b�aQwf;�N��E�Zz�&T�7k�
����Jz��$�?�"����mq�r[�t-�B`���+����n����T�~�u#G��[��hGm�f��i����%0���N9�&�N��j(�������.����l_4��^��u�
{����A^�������_���/��~�����D���������L�t3?��t�<�XC/�j3#�F�`Y��M����	�z��l�A���k�F����l�^���l:OH��� .B�n�g�	Z>�U2OC��C��?�
h�iv����c
x��A�Z��������&/���������h��b3�H�s�{��{�WC��������-�O��������{	���e?��p��F�\�z�)����������(��f�B
&�?'*h��S
������>\}����En>�z�����o|�k��d����F���?�������d��0�r��|K��G���!�Nq���>����1ixr5���S��TG�7_4_�h�����E�B3Y4_�h>�������q]�]��/�m�����.��.��7�~\���+�~`r�f�X"���M>3\sZJh�������%�����/���(]RQ��f�r����JSN�E#�� ��q����6�87�2����Yz���ir����2b�R�c�:20	�����m�r �����a5|���4��o7~u
�V�����vHz)���A&���;����L���Z���0���\J]>T%R���=��Z�T���<y��y�!����R�e��-�y��KT�2�W+I������8�����E1�=C��s`G$���	�c=����aE$������'��
RaK"���#�e�yB_�����C��!Y%�������cg������b@=l�N��o
����L�M+��%����u�\��"�|i�Z
fLa�t�3+2%�3[b��3�j<g%��%�k�
O������U\������h<"�jHj3���yH�N��������>��K�1}{vF=��|$o1���a��yL�������-�[L�����e���}����#��>�����@����>r�����g�A.���'e�vI���\�jJ���-�����"�:Y�v�\�l�su���F���/�CL����������H;g>;gn{Lq[K�9�I���m��(� ���4�XU�"��4�[F&4�h���Z8�tM��{O�+��`�]�g<!��>9e�����8���$����[CW@�l���|��w��rN.C���zb���O7�����~���n����}[n�!�/���P���\�dX�&��.�O�4�D��j�I�,#�1��F��7=�&��rl�����h�����kRPG�m��!��
�F�:�P�1i��f��:��k���5=��m�%
�4�^����v��p$��r~��iKEkZ�V���,Q��%]"_�D~�%�
��w]�T�-���%�?(]�@�DA�%
�(p]"��&Zd���a��e<�-1R��d|L�D��K�$M�� �x�u�<�������i��K`����?g��
�*^Q&e,}y/l:[^���jSZ�oB��Rt�`�a�`����D&[�9D�Cc	��<hrR�HDu����`�s�T�t����`|sb!.zg�K�w���s�s�Vc�>A�D<A�:A���oNH�7�b��v���6�z�D�����3$���D=G$��/�@�(�K���-��,���N��&oj1�}�Mm�x��i�_y�!�;��������c��cp��j�61���mb�~2��F��@ 1����A�B��kU�F�$^on��\��{9����\�T��SZ��I���(�����	����K����pz���q�5f��C�^3����	�=\5a��f�,����(�D�D�>y|\%�x����u�|})������O76�N\��n)��5{HS�q�s�Q3� ����C��HQ�II�h���D�xU���+ig]l��k�z=��t��Z2[�]G��ppF9"�85#j��Z�d1ix#f��@�D~����OIQ�S�[��~��U����Ws�j��������e��,�c~h�����w�n�H��W��������)�������$J��s�[�������(���"=$e�{����O����	J���(���������Y��7�n8��Y�a��s�n�"m��Xd0~�l�x�/�p�n�s���"0���r�Q�~��������s�����J"+�e���+=�F��8�Iq�y���/��y1o��&���3QDP��-X�8��vv��![�7���/&���p�gG��n���I]R�"3�MP
���REDB�8����v?�{���"HA��7~f���754K��U���[|i'{��s�|��������+1��+S9���a)-o]Fqs��}��?��i^� ��������4�������_g0$���4<�+�=�C������
�HXPa������T�_Fc*_�d;�bM��NO�C��~/���l<'�4=�����'��$V���S2:3eL�MO�	�b�ir��Z�+Ms���(��$V��\2�M�j�� ����$V��R`2NL�Y�Q���XM�J������G���P�<�I�����c���Q�I�$Z�y��c�"�B�<�I����'���Q�I�H�A������ ��RO��)� �����A�	RO�z�OJ����F�����iU_�qc�d�N]S?Zf'��������nqf#nG�f���K���?�0�C��0�X���{�.q(������R��pY�^�vDe���n������in2�!��l��X��r?Q������}<U-��*��+B2�|��.�p) }G�U������(���5�e���l���h�w�i[�-��-��7��)�3}��U���x�T�k|Y�����������,���i��S�I��g���p���a���E��q�%�����������w�s2AT��h�d�h:f��S�
��RI#��N��'\4:�JP��i]�z�3N��Ht�I���	'g�����hN��c&����4�OY���.H'��RZm�S.�_:�$ut�p����U����lG���.%��7��v�����9�W�0tB��xF�?���W&"A<S���P\nW��3��J�.)������.�%A<�V.�g�2�g��w�,	���rA<{V��3��T4+��b��d�*�`vQ��Y�0f�
0��������]�*r`vq��YE-f�
0��E�:(^�'�����m�W��!e�k��u[�jD�l��6?�gv�����d��^-����5,�v0�<��8��)�P���z�>�=*��o~���xK�xKj��NZT�W7��=r^����X�{R�b��;��O�M����nKn��cfs��<��,��zw,5	�%�����uRr�u;fe~c��m�D�����6�T�;����Jn�n���5���0�������&���s�_�e���X/?��Up���D.�&�� ��������V��lVHAb�C�[	$'�Y!q��KkvH^X�B
�Z	$'��@r������;���Q�G6�
�>�QQt�k��D�����5gH�X��r�V��%���A:
��Iu���o��!��X��4�lP��NT��W��[s{��{��u+��GW=���R���>��s-���Ql���Y��n�'6��#_�CDu��y6��]�����������pN�����Y�d3�����<�K��2�]X�}�*����
���pN�����9�gO�X���r�9��s���U����l��]����\W������h�D�
��6`=6`#��8�h�	,qK��Fn`c7����
��
����#����</+��4g��J�F�J�����	z��ct�q�A`u�J<(����-K���y$�7K���%(af�������O�M;:��6����w�{|���Z�����������N�Z�M3rp�Um7Y��wKG��W#�ns;����i;�3�qc���p,oY��<�4w��yng�6�yc}�51�P��mt���:�:e�t��B��S�:�d)Lr��d,�D�8�����RG���N��E�X�T����������bn]e��D���1�!f�+n�U82�BJ�En�+N�l�cac����/_M����*6��|L��o�k���j�unTq�]�56�����T|u�u^
T��n]��pV������G�e�|��K]��� �����|�����P����?���h�,G6/�����9x�=}�o���*��"�G�m`��;;�y��g	�Q��(�L������V@������p���9���P`���������#����4L��`\��Z@)_�Kk��^�kTa7���N�������nt�*�p�;H��	�/����
��d��`�!py�T�����
�I�CnM8�G�?O��Y��T�])^hKo2�mXje��d�;
�r�C���&��v#T8�n�j���IY�S�v�P6t0��[�@���b��=����
8���Sp�^r�y��&X�'>�S�S`xO�x�B�H�"G
�*�[�p%
��#\*��@�+HA(��4�� 
�Mxr�� '�zi�����u2D�$���X���D�U�R����I2.�R�MsL�"�$Vy�Z#Q���4Q����(��H��jd$�V5�-�^u��R"���gF�OFV������h�TF��2�6��iS��2r�K��G������i"{(;����k��M���Me�tJ�T���y#WGH-��Z*�-p�D(���C��H�u�(�e���Fm�\UD�C��V��	.\f�+������FC���ht��k�[IbG����f�����R��U��!���{�q����"{:�������si���5�{�Q=��aO�d����t��=��`O��������#0)�W#�|��:�q������8%\��YG����v���s��	���[:3$���!l4Dh����T����z�-{@��<9_���T���bhz�6���'IE�d+����#Fn����u%]��g��(n�+�J�Cq�[IWb����J���pK7���*8,C�\��'2���/��
�h���>W���$t
�-xp	��0��`'�X"r5��k��x����e(�	��fz���"��+��>5|S.l|�|�=��B���&�ZC�y���/qIy ����*���*����_G�`
��.�]�
#?����o<�\B�s�5�p]�"���m��\r���H��"E��MKMf�+l������6��F~��9b#Kl���L��+6��F�X�K�� +�"Z���:_�r;�|3�l���6�|$�^�����������=���$2Py�3�>��a�g�-&���d�����"����Q7R�����dWQ{�������xK|�����__�����IM�
�&��_�	�)si�^�J��������������//�4��3�5C�U������z�s���_��v���]����\J�����T��N�����:S|Y�W���5�?��T�f�!G������������%V��~��p��/�O���RtC��EU��4���B�(0ho�0RDe��zJ�����o��zJP���Dz��zJ��������zJ�z��&�J!%L���L��+�0�W�HaB����^)v���hQp����PP�'.�7T �^Hq������c�v	Q5)us�l.�W&WFT(e��E�/�	���V�w���$�`=i8V��%���E,�T{���?\�la�����!��1W;8Z��1Iu�R�����G��~��Q�ODl���"�9L._��J'�X���N �
�Jg�X��D�N ����d/H��Rwc�#�����P#ET��~:6��6�Cl*�D��?$y7�^H?���+,����s������R
���ZPb��PTn����V��'��J�1u�r���I<��N�'G��L[	`]���&�k�^�fv������zw�k��K�������T�	uM�la[�Si�����~)��'��i���%hH<6}m[Y���x���k�s�;�Z#����j�+��.a�yz��!��}jl�9����Y�r��{d�������_z�}��V��#��K�p�e	$ ���{Y��<�B
�^�!�C/K 
�
�p���?���xi���,��6�J �=2;$��r\���p�n��w+^
����
�e�I��x��rH�"/8/ 1
N+���J�6�1��|Lz��6�IuMx������c��U��2~8'�p�1�v������\Su�/�lw87��z��pN���;���q��p�C��k����0���:������&��'�;��>���=q7Q�W|�xPA�W=�DP�8�!��&������'�*�
9w74QO6rNp(/��~}"�r�����AS��4S"��(�N�c�L] ��������@�U�C���U?D��#��8�h�	,qK��Fn`c7����
��
����#�����[�`%x��LX	�
K70o�:������e{��D�p"t8:�N�'BG�DhV�vN�FS}����/�-������2_G��&Z@g��]�? ������|��J��@������e8O�	5-C]��VJ����s��>���i*����P���j}����
�|�3��j�c�-�~��p��������M��"xT+)87�!i�i�|@���:[,����\��h�����>�G���L�j�P?7��,����B����"A��Pj�%`��$�����J������%K�{<5?z����G�K�
��AVg���
:N�	��0B�/���X\]�Q���z� N�t�����&,a���	K@�I��h�������u�-�kd*��5U).,�aQ
�jXT����-����G���B#��-�riaM
kjXS��Z-����nMm�r?�%����V5:� ���5,��9���'���6����~?�E�9����P�aq
�kX\��Z#����xqm�p?�U��;�]Vq�aE
+jXQ�s+�O>aEmfE��~4��?d����B'e��I� h��B�����7o.�)����v���Y� ���C#�i?%��)�`D?H�%��#���~0�s�1x62{l�e��y8&J<�l�'>�8f�q�Z�S��a�����������X�*�r]�B��W�Fc�����=��O���y���w^����:T���F��|��V��jF����|�S���N�c�SkQ;������5)�\x�\8�Bm�W�`L�*��'����<z���M���s�!��h��M[���Zc�j�E��)M���D��>g�����L��fh`�Sc��T:W�:A=����}��z!�wHSH+<l���z�<��O���=uM�����������o�f���~�t�m#����,�i�����'KgZ9rt.��������~�t��#����,�i�����'KgZ>rd.�������G�~�tQ�W%;����-�`t|�>]eM_�sOFA�Wv]�@��=m]_��JM*�*���j�N����k��;V�U$~���������.����k��;��U$��;\�G��=)��0P[����]6�������~�)��7x������Y����M�+�*�3�_o�xS|5��dU��Q���f�U��$�����2�V�q�K|����p���Ft��zF�Mh���zz�����uU��:
Q�f���Z
);��9ZC���.Q�J
Q|=��&��)i-d
���P��ZRV��(i#SV������%��I�W�(i%�*��������%��Rb����������lM��K�����jL[���xu�m�����;
��E�`�T+h0�W��N4�O�j2(/�/����ZAyi��DDw�����V�����Z*����-��`z��e0��\��9�|c���o�Kd�5*{���Z�L�Lc,���2�K�
���������
�P�Vq�*�]�m��i���P���K�4a�W0����6��l���<i���]����i���P���K�4Xb�O�i�N��wo�.��}}�����r�+i���������+]��w��l���#p��Oy�Y������6_��m��o���l��o��w��]t;_,�J|�%��o����e�n�<0+x�*�g����|������h���6���"��A��G�>.s��v�`Q�����x������w��\D�,�/�0��m*���e�����w��._��
f��W�/�����,�<�`�����������_�r�>�j��p\?�[���M��M����I5�|���/�/�����xw	2�|�_J;D���`,������8D��	�7|~�i��|���q��D���x(���.�Y���%��tp
+z�����/f�\��~G$��H�;"�u�6�)��������#��;"���H�(����v=���%����#����3��?Gr�,�gU2Or�LY��
x������:���D�MI�nJ��d��di�)�uS��n��%�L��X�#V�v51o��E�s��]��S�F�N�}W���yg������T�pQm�����tS�i��%��vt��������;04�vi���d��4/�X�A��l;��Y�1�4fAF�^�	@���di�,��rG��D���n��h���1[e�|3��X���V 8
���������]�����=�v�z�D�W�D(��I��X)�h�`���<?������"?p�:G��Y^F�P�+��GG�v8�A����<&�.��x"�o�"�5{�������#BP�g#�2A)�#|	h~<;�/�G���?��-{�o���#R�W(%"x���N0|���GT�$��c��{T�;B�>����P;��lLzV�'�']�L�t���(#Y".�r�`�W�C����o�7��7���D�n��(��������F���d�^�����
4�����i]`����l�)��l
&�a�>�U�����,��
H���7v���������l����'xC%�x�i}����8�x�D�����%[��5T4|��B�� <���;N �bc.$.G��w<�W��G��e9��C^P�����Q~Bv`�y2��7�J�/a=(�����-���A3!=
�����Y���>)K[80\"�	��S���<��MVf�5I[I��ai��������[�)H�&�����Y��U�[��%^Y����u]��?��&�����M��g[��_������X�?���<��o���O>��Z�Y�V�-��h��k�����q]�j��)-t��u�����9B�}O��Z�V[ds�V��8��a���&{�������x|[���{�s ����cM�����9Q�#=�z9>^a����+����P�l���nEW����@)���I��2i�R&�RjV���2�G)���rG� (%�0���)�����,!��(�3���!%�Y�6��eY�9}U��4�GW��M*qqmO&�4���R����S?i�����������r5M<�O&���� ;�L��lk��
���v4��T�p��+��T�8,�l����������w���!v�}�>�zA�p= ���!���Mz��@sq��}Q (�R[x�o�)��GG�f���m6'�&�����f!������^����B�N���D����/�|F�4���M4���/w��J�T�d�^�9�s���e�������8��2.Qe�_��9��S���	��2�>a9j�����3(�KM��e���-9x���+W��z!.������-1ag���1�g�Q�����1�� ������3f	Bt)������i���9��V���r5�v&��I�j���mK��)�������d��nl��:���Ur1����h�B�<����YC]��P|�3�xz�O�RIAT�4Q;W��
4m�f_t9C��nt�S[
dRX���Tg����4Dc�g�4��CA����(r�mgw��l����s����5�����h�B�r�3=��r�Rd6YX�Te4�����AQ�`g�-�u��ZB����/?�8 i���<�0��aU%��[6���u�C�������:�q���h����
T���kK	�U^���+=^����q���U(���D���1�8��
`� ��f��b�R���d�"�*��#Y�@�~�&�S9+v���|������`2���������i<M��t8JG��d��d:<I�d���N
O���1�f�2�Ob5Y+�%�3S�B���L��XM�J��81e<<*�2��j�VV��j3iR�5�Zy3�����p����T�<������e<�%k��2���EJ���~4���
�h�G?z��a�G���nEog!
;!����;t�t�g�5�c�1b8vL,�Y�w�2�8b�%���f
�.���g�fY��X��j���@��|wu����I��r\�:��&����q^�K�|O�����l ����#a�dGVUl%0�P�i���ynS��{sb*��h�=�7���7l��7�
���H
��Bl�6����$�?�������qx�0�B����c�H���0e�ELY�X9a�^�a;$�1��Gs���x��L�\h����� c�#L8fN��$�G��+=��K���u�O�S������|:9e�Oi�0�N�)����e�Q���#ZO�G��a~4��<r:e���u��r�M�!�M������6��R�5�b�%���R6�R��}�,eX��R�?	,e��?n��u��i�2�Q���dec%(+��l,$������udP6��A�X���.2(���ll"������=dP6��A�����R���b8�Bf�?(\U��kWa���}�2T�k����1�T���cds��7����i��nO���+�t��P�	�k����I(P�P�'Q�WS�ef�O����qm�4��A�%�IT!+�J�VY����n�����XV
��_�����69.�m��S��S����Os��hY�p)�MF�������T��.d���#���h�k������L�N`1�p�i���K��\��b!<��g,-p[i���J�R��x��'��V8�������
�g$-p;a#��OnBP����\p�%���#=G��!%bu���ZyJ��Y
�t0��j��Z������,+a���?�#]���]8�	^��`bO��p�R������J�^���L]Q1����,c���)�F��Fm�Szv�������
p
�b-�U7#
��y��5���i�0�����C�������Q�W����6��m�n����A�u�^�j�!j�������	���&OH
k�H!i�mR����	���&[HJk�x!i�m����	���&gH�k��!i�m�����	���&sH�k�����QF�M`���
l�6u;u;s��v�D�e`%�M�~.+A�4��J�K�J�����LEo?A�����c����C�r~��
����H|��>L
aNV��i��
i���]����!J{>'8�WMD�t<��\��z(I����0�\���u��
��|�!/�4�`\�1<7��1�8�8������Wb�[��;!\���[\�������Q�L�07���nt���ia�'��>4l[�o�����Ln���!����X�2Q���+Y��%��F����2n4/� �t�^,s��E:e�+� F\���q�c&?Qx�W:��������U:�����Z��Hw�����W�<�_�e/����D���PF�����I��I��]��vc+j2�X	����]�A�x���m@�uA8�v�.��N����@���x�*���w�&��C��%;�$���:�l���&�/����?���h����K��}�k^�A����<������e�E>��7v�h�pOc����E�S�gM�x�iPy7 H��aWD��'�kj�U=�]��
�73m@k��ps������Hr�j��|�/��|�{�������a7��w��`��W��BfA;����%�Mu��1^���I	LI_�5]���]��y\8��6�4���MmK{�F��v�IW�T���i��4�����`�d�J�=	8��\R�QG��X�u�O|��>��>�g�(�j|��S�IT�H�UE|+�D!<~�K%|�?�a�X^����8G�@/-�������,�-��e������Np�
7�X�8�&^
�:�2e��Dn�FF"�[##�����-�i��M��H���������H)���#[����o*�����Me4i*�iS�6��YC��q�,�����
1���x�*� ���k�
�m���v��f���
7���Q�[8���n�g>��L�I&`�xky��L��-�S�f}��V7N����OG��6��1����L~1��������iB0�4��z��J=M��&T����4�i�4!�z��E=M���& QOz��	2T�\aR��&�/�)N[Os5�:�{eRB�ner���5Y3�vO
�Lh]��+yic<W��������S}e��b�������B������a3GI��s4J��D�tQ�O��r��e�'�d��X��������>y�O>�'�i��P����q�4�-xp��v���r������]�������'D�-2`���OR=�H.���|��#B,��#���k4ul�-1jfk���g^m���R��U���h��	�>�9����b
��J�Z�U+�j�W���_���V��J�ZV+�j�X��c�r�V�����]����,J�jc��1���[�����3#{fd��,��I3�iFF���I�����l���|�/g�����o���lI�L�����������g��E�gm��wz�9|H��"�6[L#U@�]��>�e�r�����5�
)�,��fsJ���<#�y�i�rb%��k���������}>	�I`�t~Y��)s��^�`Ao��6��^�V����!pU�����QG�QFk��9?��|b�)��	u�)�9��[Y8`���9����)������|T������-��A��[�~����R%�����n)��z����7����Vp���yq+����N{+8���N{�)�x�[�������������������?���8���y�"�������	��G�	����	�XN("���%�'��*�,��*92�*h���X�K����������R�����h�>>�]B�]J��"AJA��G�!(t.��	M�>�$�������dk+c�����]�+
��
��9q���������a~<D�"f�GK:&����>?���(mu1hL�i]�i]�iR�iE8q�O��@�:��n,f4�q����Et��@���\�'�$O�������E1�,��f��������Gu(,��JLjA�%�CQ��^�?�[�F�>��)��@�=�*��S�)��K���N��)�Z�p
`8������p
`�N��
���
�p
`80�N��`c7�p
�N���_�|�
����p�N8_'���V�p�N�p��u����t2������l��a�|���Q�����m��IO5���&_��-V m���W��x�y�L�����R�6���3J��.�V�������	4$V����,��23�,Lc�������O�*�����|��~^dO`MJ
��x��x���a^%J����u�X��s�d�p������FI08��+d+�����z8���������������SN�����
v�K�b`L��=LJ�q4�(��`����&G���l9V��t��@��`��qjNVxu�W����~6b	p}�G1K@���)K���|V$��5�N2�LR���)���Et���v��*��e4,�5
�������et��hk�����[h)��eT.-��a��iXE���U4���\E�����r�x��F�biXP���T���Z�
jXP�ZP�O�sYmN@u�TdXN�r�\a9
�i�\�r����yb~ �(�G4������P�::�4���o���������Y>�!�F��

P�"�@��q:�Y7��Wo�\�S����A������A��1OC�h?%��)�`D?H�%��#���~0�#�1x6Er�����j�������������c�iia��>w ��K$�@���t�fE"f���_��|���p�	�_B���?�vT3]����Ulb�D�������h������b��&�6�j����+Xk*]���Khy:�E��E�
T�RW�'��qL���i�L3}��kSTt\%@c��*�(�%;�&��*�+}�t*�E����$e�W�a��|ZRO�(���������1YR�RF�XU��[�r>�~Q��}��2����������N����X!�f��f����bS=uYU�>U�����SB�)C�)!���� 'y��,�i@�9^�'KgZr���������~�t�=!�s��,�iT��\�'K�,{U��s�Z�R���;�M5(Q�G.��z�"������/M�ZG�K\�Q����@I=	��
����y�&��Z���E�:�G��v�+H�
�>.�x5��2fS��H_�dce�wT_6�)�����+���Uk�TC����_Ev��SW��T�
�\ZS�+�kJ�>���E9k�v�(p�<�,���O�d����(
,��,�	Z��i:7E�W����ihZ9���F�Z�U�+��i8�m��Jx�/��;f(�5����������_q8��;g�+����tV�n���w�k
F����t>c��m����u���c_mw]�������������L�C
*_Ez#�"�kS�z�[���0��`Pa<�������,g�W�����U+b������G�H��=��m ��W�g[f�;���Ftm;�%�&-LZ�-��������FtKn�n0p����tF8��'a	�����0>;k	�SO�k0�G�`>����c"�BU�|�_Qm�Qe�7|�NIVY��N�������&����������r�M}�������h�����>+�[k�����F<��U���T�Ej�:��
ku����R��a�����������]�����K%�*W�5A��V�����&���h��a	����^=,o6����E�|�"�O>f��>�or�p��FW���z�G���q1��lo����:_���w����~
��@E�Y�)[����Q�����W����C��������KM�3��?�|x�.�����\�Gg�?��������l�lR����M������y}	>x�����K���[����� �G�FS0�WW}�����?������2_g��j�����?�������@���7o/Q����`:8��=s��?��W�{R�3]g$:,�1'Z,&���f3�����Q;#1wF�zg�p�����v �^������}��=?~���v��r��J��-��J�K��1 +��/�4k�V�Y�]���*���d&i���U�l�vUb��d]e�Sd��N��R�b��i��g!	��-V85��������4+2E�U{�wD��J��Dv�H`2���(����g������h���L@jy@�B�U�0��L!��� ��<��O�8��$�f�������*[��Y��R���H����!Ph�wo�__����@F ��� �3�&���)�[k&�Z��O�1N��F�"�(��_>>h}�	���j�8 K#Ix���w
4�����l�9����]� ~��W r���P��_�G�H0�)="$�3�IgG�{DoOO�c|D���)#�
J��(���N0|r�#:������^K�����t��;L�Og���[lx�F_D\2���X���zw!�3f����oFSR����h&��&��T!C��N�������3���F���dyV����
4t�����V�����l�<^���7#��I���L.y��c�^*���E6��������7vb��������l���'x3'�x�i}����8xx�D�l��Z�Z����t���go�Ap��,�Z<H���������I6�_��>�7�J�
�@B2��Z�+��u���[�"\k�����[d���,��^��#ei+��K$8���b��s����r��i+�ZX�
���$�Y� ��(��##:xM�����d��(��T{���m\.>j�W[T��T|�i��|��D}�I\��u��z�CSQ�c�T����m]
iek)is�}t����H����k�'�W-K��u
;�����lk:������#�1�2�.�<,����s�ik�4���� �=N�*�+����[���\���8P���\�4�K��rI�r���"���Q��"���Do���.[n����E�n��]n����9D^`�v��U;��O����2NiR�Sj{�����$�I�z�
�u$�G�V��cI��d�H+,��$%O�_��3�AJ�X�2v����p���������F ;l��l�M��h_�S��vd���*�f�E�m�W�ba2�y}~y���u��������X����<����I�qh��O&r��/��'����dytm6���fsj����Vv�_�,"R@���	,L�:���/��_��$�4i���7��&z�Z}���0�f�I�U3ZHY����B���Fq�M����nA����z�E0�Qf��(���%X��]S?pd������j�AKM��es��-9x#	��+�'G��x��w������(p&EB*'���3fLAT �k��d3���b�AG��j�p_n��������J�������W?I�O�J��N�j���
\%�#�f������!����y��y�ON*?
Qt�-<���{c��C�����k4gZQ���(j=�l4
��~<�Y�u{���N<����g�X�"�������6�zs�9��`��}�W��h�BSm�3=��uo�Rd�ZX�T#5�����D�(Q�`�-������&����/?�8 i���xXI��G�ie},�}H
�o���=Q��j|�8��4^�vPr*HfN�����*^T���/\r]�8��
��*CU�N��N�v���c[��s�"�l?R��L����,C���k����=9w3-B�d;�x�����d</h���L�iz��Q:��&(}%��I�&k�d��'����S]�V`K��)c�B�t�Z�����q
��+�J�W
�R���x��+^����5���d�����n��>�������)����@&�2���l�����,�#��)��Pww~���j��2�RE�	9�U���Z3����&�gv�GG���:�9�f���4`G.w6��6����?���x�-iL+��~e{��p�U�q�J�'�W<��B�
��������1�;����
s�yt�A���������h���F�1cd�x�2�!��I���0��8�
H��	��I��P���	-=�A�IaR��1�2��K��<uXJ�S����L��t�������4Jb��-]��Q2bX�9NK��N�F�t�F���u\fy���J�>:Y���<g��<��Bs��jI`����$�jm���Y����Y��F^��i��EcP6��A�X����1(KV@Y�1ec���
+��,���_��zQ��]bp�BR��%\UCY�kW���}^�2�Q�����1mT���bds��7S���9��n�-Y��+�t��Pwj�k�����-P�P�'Q��m`�ef�O���qm�4W�A��6'mz^e�z
����y��0����������5q��q\@�Ra�&���3JM��+�2��Rz��K��qq��6j]���8G�����n�f��7+����x��'�m68�i��	,��c�lp<�f��5+����xF��6���r��J[���?�t��KYK7���9����t�������h����x0��E��D��U����p��`W������9��������A���b
��Y�@YkS^'�K��:G�4�����7T�������`��v����M�R��[ZX��'���t�������~rV���6]�E
k	w.2��:V/%��M��6N]R���uI�j��%E��e�4�6�]R��wI�j��%e��}���6^R���xI�j��%�������4'Y�B6q�������9��C;X"�2��P��P��P��$��-S"8�%�~~�t��Wb$;��������u���$R�YBe�!���yd���|��9�j"��c�r~�j8�*9���6�\�\�:A������w���W���
�P���hU�u����6��0V��������O�T�r���k��u�tg"w���2�+p���IN9���#�MD)#�+��%{����K���:��p{q!^��
�'�4�8J���@>���
����$�����N�.������l����uAv�P]���pu�52�w�&��p����E����7��j����h�p���|�%Z�c�6/�����9x�(��7y�amW�#�60�|��o�,����F�������Ak�C����c�����Y!��`��	T5r9�*m|���	���@q��VE�'����Y�n{�����d�W������Obe�|*d��.��vX��[�����~Jh2�/�����=2����bL�
3#�X��2�Q����Q�������u��
k48Nt�����5g��������1p�9^�I3�H��E|+�D��~�@%&��
<W���U^������1�@/-O�����S+����e.��x9p�
���%Y,	������5#�C����A��H�0kd$r�up$p�52R@J�X�8�gd�q��h�TF��2:m*���2�������-Oe���t���5�Z��Z��v�0U�)��G�.�1�������k��5v�~���o�'���8���&o<��i��e&:w�����zy���V����ij�
�a��(�8}�6�����=M(��&�EO���	d������z�0=M���&�DOD��	Q��P��oC�����zH����(A�E�yd���!i}�?�c�+�y!�//���
�x9����W�}�
��_XF4�C�ZG!�&��b�O������S}�����|r���O��'���>�T�|�M�;
�5��g%^q�����3�a�~l#x��^,Ov�#����/p�[��i>`���V���������v�8<��t�u�0h���Z<fkfkR[�3���"�`9��J\qvu;>|��w
6�+jw�{�*
j�A�<���V$���Z�P+j�B�\�#���j�`�b0Z,����cds�����12;Fv���X�t��9�-�u~����,�3B7��v��m#���)^^�_^�;�{�������
���N���)xX��f���]((�y��*�n���>����0{�l6'�$��3���,GS"x_�g��__�����IM[R~��6s�^�`Ao��6��^�V����!~aU{�����Q��Q�j�/�<��	r>��Q'���0���14z����(���(�h����_����t�bM�9�5�'���/U�������������F{���hoG����?-�s���c�h,]N�Mc�c�@D����o��1��$�!0�[����$y�����
�[.��hAD'&07�V�\N�&��O��
=q���(�f)x!E�����6B�s�|��Q���J
/���
A���Oh��!&K��&�]]_X[Q�?
�E'Xhi�J7M���9D�wJ.C���F���i1�E2-�����*���n&���\�1	�u��u��I�����f<=�h!���������f\D��$��/�������0q,��3�9f���25\EL.��Q�6P?l�	�P�p��Q���\z8�����p�
�1���m8@���)���6�`[A�TA��	���m��6�k.Dq��m���6!���
���+���]!�������%��/�n���d)������l�Wa�|���Q����?e��%���Enb�/����H�D��zu_t�eB�a�x�:����)����w9����y/���
��V����,����Cc������ �)_����W�_��&�y�<�% )8cPV����+��t�(aY���b�/,0�V�U"36�4~�����W����\�k�J�PG�����j�s������N98m�[����+�u/��_0�/��.��q_�$���+��=n.n�X)����1���{��y&���dc8N����x���I���*�����52���
����z�4,��o���].����������agK�\ZX�
�Q��Z�
+`Xw��E�w�r�k���F2�`X���T��Z�
�aX�Z�O�w�$6':c���*WX
�RX/���x)l���`
���fA�sX����WG������mX�������:������h��_�p�m)j�6` Z�"\�z�����Cw��5v�?�5��i����� ������1�`�?�F�b;<��@ N..^�-������Y���\{�Up+/�Fw}vN��J�bv'<bs��C���M���K.\b������J9�Su��������Ul���&�]V��������Y6��OE[*���6~=c]�k����f�[m�=�/�6����A����������/VR��G=}j�TC/_6�s]��8��O�0m[��
C���7b���>@�����S!�f��D���/�Z�Jl�����^Y��%�,�c�3c�|����O7(/�3�WQ���$�'Kgb29L��d����W����R�K�2�@��P�����G.���`,1���,kO-��(/���C�/�*���c�}�1|�V�5�D�P��K�8���������[�k����f�1���;�X������`�����������`���_��BWm�X��6�b��.Z�������N�+�`�`��r�sMD�i$k"�L�����"�>q��`�j���@����m��c+z�?a8�0���0:���.�7)�;K�e���x�"��YM���mu���7���C�QY��SjHG��rf����Ox9���V#p����g�5����BM\����r	��j�E���\tQAwc�b���#lU����Z�J�
�Nf6���WUl�*SI��^u<U���G�����^�tw�[��}�N3������L��g���UIxc�[ ������Xqa�������=�����jP�`�h���&L��q���`�^u*	���-p��*
�y�������b5���VC()�Z���������t����P�?�<�Q�
?a4����e��{t��&����^�.w��J�
�x���\>	�xa��
�x^�iB�{:L���x�>�����]���a���r�+i���������+�����b}�o��7p�,��m�)[<�'�z���
��e�r�@�w��>��]��7H0+�����s�5��x:�<��8���f3�^���je�E4�6���������Q?����� t��z�gw��(�������W`������������&H��p�?�}����_.��sT��S���-y>�
6����5�y��m���[������y}	>x�����K���[����� �G�FS0�VW}�����?��@O���/�5`k�6���7�� ���Y�?����D5�� =r+�
�R`	(��7����}1�d�$�;=�uzR��]�k�r��D�������u���F:=�N���������c����3�Sq�������m��Z
r4��rT@[x3t���dixP$�A�T�~P�s��A��E��A�/�:A�&��/�7�F�6Yx��8,��\���6-��������).t��p�����Y��(�,1��VZ�RQ�����|bu�W�~Ex5��M��E��CA"���������UB��I\&��L!x�*�
�����_<�� ��������&I�������?L���
��	~�i��%)~����fJDnX6{�p���������Z�hc7�V�"���>���U7��V&�
U��������]��w�����x0�������"�z�������o��T[FS�u�#[����G2�
��w�E~��u~����,/[>PE���"r�"i���a�x�o[�JXBY�|�������jz��;��:n�U�������{�����_c�]��8�Z�48�]�l8�Z$��[����C�2GY|@���i���G��ppD��)g#��#��=��`���gG�%��������e��-X��������?N�[���E?&�-�����q��r���C�x���Ge1c'��v���jw�~&DA/�&�_����_��@}adE��/m��rT�,e��/���+�Z�8\]���(�l�*="�"�JD4o�y�f�Y���������]����I\P�1)��%�`�,W��^�W+���'��������,�
�5����vD[�<�b�u��3B������a����SQ��_~��og�*�=W�x�.Q�z��"��w�<T�S������_��|��`���>������O��+6R����C'
4�,5���P�`UD;_<� 4�	�������$��Y��x���L/�*��|<��u�����y�`OK&��L��(g%���f"�/���q��(�������gd�!Qd����b�D�%�*�M��	N�����Mk�]�h��\��vQ����v%�]�T����Mx�.J5]d;�_�.J5]���������]d����I���Xa���G�(����&����<�_	cT�E��#�����s�8��a��\8L����6�l7Q�|0��,���|m��X>-[.����,�uWt��Ka��[�(������u�!���r��^=|�5�3���-�%6f���kwa�c����)L���\a'����&��j���
����\;BY2�s��Q5�"{*"�"i*�"W*��"Ejm+����q���K���iur��:i��N���yZW��;�,�I���-�Mq���6p��]�s��fw��y�"9������G��4��|p|V�ny��uFf2%����k+�jm��fEwH��L^��L�R�I���(�����(����}b3���-\0
U3�4��<�l�`�������w���2��}�u���z@���GZ�o�k\���	�b�/��'���W��(�l8C��������w����\��n��.��&Q����� ���8���0���m�|u;��6�jH�r�U�h������c�9�	K0)AL�Q�c�����!�d��Z�I����)b�����g���_���m�o�"c��H����3���Q�Z�OUc�'�x�=e
n�\i����r{���3�hf�1b`��p�����1��n8q����
��q7�x���~]���l��|X,2�o�Z<���k
8�j4�
��C��Q?I�O�Jl�)���M�C\�d�
[j��:���h���M������6F��zS�B��l��o�p[7�j�Hmv�}Pa$�����1�
Kn}�Pv�hz3���������y�9�����}��qv4�?�����d��&E)� ������!�V�W�'8|���������/?�8 i���x���[��ik��p���j��lj��"����X<Nx����;V�
Bd]�-�
w��r���\3�X���G������4�A�������1!������j�,��dfX�]��O�\�4&���O�+�����d</����L���qO��8��}@2�Ob5Y+%�3S�����4� �%�&'������ql��}}
j������XM�Jg�81e�>d�F�Ob5Y+�A��4���.Y+%B�3N��5!��I�$�d<4e��<M�P'����g�L�w^�Kj$�R���EaJ���F70���
�n`t�{�a`t����a���������G�)��:����zt2s$��2L��Oh���2
:b$+���f�q-���D�<��(�:qTp����yt��n���_���r�=/�i7��Y)o������k�-����26��tG��X��������T���4����B�2�@�D&�aN�z�c�*&��Jg� �����	QC���a�j}�%t�%�����"%���x�Z�p�Q��5�,�e���l���xs>L��L�������},�|�+�E�6�[d�h�9#�",�oF�E4	��9@�L��c�`�0efP1�(bl�$�8�B��`Tr�bSL�6U��8����`I�Kw	'?�`���h��*���)9%������6��2���@���?^w�	��(���E���Q�I�#Z7�����F�P��S�3u<�$Tl\���hz0�!�t��Gy�Q�0E)��U���xQ�R�KJ��(E����RI������	U�.J%A��V.�R���*�fnrTm!�1
P�Yxb F������,01���@��RRKHb F�����,1�P�@�Qb�(H!���:��mI_5�!|�V*��nKH���"��8?#0�nT�!�%�f8��i�Z�X*;c�&ut�Q��5������hs��W���[�BB�`����H�o	oI���,�46�b�W7%�=r^M�6���j[Ia�H�'B	�q�hY@��t7B��L�*s�S�����RV,�C�<d<O2��sJMBd��u �P���b�>TY�J���l�����R�8Y*{�.TG��X�U?��'�x��=��&N��^��e)��X��HVV	K�G.N��\�����.� p 8YK��YN��B������H������-/k�!x1K!HXN�2@pr����;��3��?�Yb8�N�����}����>���9��U.��S����7H��x0�.[5 ]�S�"d�o
&��;=��;Hv@���<d�	Hd���,�$�U:��
�zt�/���\u}�~�Z����RS�siY��O��/G�S������ld��.��3����U����9�����e�;�	7�y8�<\)>EG�s���m�qUg8��pN���Z������pN���������JE::��5e��up�i��&�-nF�������F8�7���]�`��K	�������1eP�^�j���@m{�2�e_S5����/m�J`�mP��*���CP���j��A-[��e{TaN�`���
l�6q�������9��Iz�DAe`%���xX	z�9SV�^
V��D$~V0>I&�����4����T��O�=.�������6�(^����K"�!��tPY��.&k�IY��8�T���+z`3
�L���>���y�)8���(�x������Azv�#'%^��$5l`��D�qxv|5j����;n��8���]b�+����\����qKG�6�yc}�51�P��mt^�2����X�2Q�/��3R�����L�i'�W���H�]T%�2VK"�bL9����D�F\}0��r�Q�����hI<���6v�k���8��q!n��;�'����(�z���O��&]�T@�Pm>&&�����N|5�:����������Z>)�:�:��XGI���1X;_}V�;>���6����\-�I��o��f3_-7�����>��/���l^����_s�j
z�.����U�]E0�(��,� %vVo�pO#�����WOA��x�iPy? H��a_H�\/�&h��W=T�)�:E<(�;"Ec����
�_�������N���)���������(9oY���3��ds�`"��*1��;H6	F�9���,�����c�h�i/��b�h����.�7��65e��vOS��j�4e�b�~b-q�����d=�H��)C��k�r)l�] g��3�b`��8CN�!O�!�
�
���������B��V��2����Je�]�
�w`����z�;��{c���r+0���\�\�R��[�v�K��##���Ka���"
Us������D�	�F�\Da$1A�\Da�j]�Hn��=sI�e�H.�Fr�4����\N����� ��Q����hs���.o��Q����&��/�ZB��1�41�
IKkYSp�$���~���v���rI�
���}��J�:T�L<Q��[f�*��F�!RE$:��m��x�e����Mh4����F���F��$vtm�iVC���BX�<��5ti�����	J��D �i���4�E{�@���Z�BO6������i��4�>{���=M���w)�S���:A\�����0.%��Y����8��E�����	n�#:����9k4Xd[��"S��BJzp&{@���.����R���)zp/���'IE�c+��GY!��.v��S,�%���&m�I�h����SM����=����D�6���5iM�T�v�I;�� �qZ���
{Q���<��p�j��Z����`��oFF��lM���r~38���I���z�f�(6ns>d�S 9!�Qb�s<�A�<}����<4Z��\���|e5f`n��#�[Cj��|W�@��O��&��c�����7T���o���#�������A��?���fG����v���vG����w�
��G����x�J��G����y�����J&�G����PE�V�B1[9]��(Q�y�r���j^(��`��q"G�y�4N��5/��Q.9��1�"Z���:_�r;�|3�l���6�<��������������=����x�3�>��a�g�-�
���@��>�{�}�����u�
)Kz��fsJ���=�7{�����%Z�k���P��/��_��$���{���hI�/����9�g/_%J@���u~kU�x�������%����������N�O��g
;��e�s`ww���M����T�\�a9:�����]�����%�A��\c_�XJ�n��QS���K����JV|Y"p-�����/��W�����������^{c�����rH�����I�����w~�����ko=��_{�)����"@{�)q��B.�/.
j�x&�JAL��"�P+�+0�V�a`B����Z)����n|B \%2�.���
��d�Rx��zpj�c?�]B�MJ��fK�L	�L=
B d��Kh��!�����H*	:X3N��G��)�E�J�	���}r�![�y��x�i�_���tLR]����>����c�v�
��)�Hs�������/V/5?���m�`���#V�2?Q���s`+�����g���$o���f�H��X��6��l&�D��?$yw�^H?�5���_��y�q��PX
��E?]�I-(�D(*7�K��r+����v���J��:����$
Ch����#�bS��3�|Rw�����.F3;�[{Bw��s;��xh���Q�tw|�~���\5Y�N��D1�|L������u������Xs��il�9la�W��McS�	�,��x
h�X��o��q��b�����*P���c��������^fi��p���^�G���)�����_(������}����A% ���z�XD�Z!�����>4@h<z`��-���?�P!gh��6�����?�P1�q��
��F���V��������n������7��1^p� @"<C��g�6E���c�t�
;HmD[�����,n�;!^a��[7��z[Q�pN���cb�N8-g9b�����Z����pnW�6�����Yw�������,��4�T���0��up�i��Lt%&Z�lf���gz�YqH���Yl��^^�X9�����A�ZugA���x?1��U�7T9<j������z����4@y�[u�A�s��:-�U';L�U�;t�peB� ��nz"�bNjB� ��.}(/���~<(/�G�kq�q8��%n`#7����
l�v�v��'��q����W{N�����)+Ao���V�^�_&��x��
������r�@�t��a����p�o8��+�kiJ8��xYC���~5�f8����_4�g����2�B�6�8_.�u�-o��V^=���B��M��g[�MAj�2�]�����D�P0�cg�4�m�9����]���E��"yU5h���7���/XcQ��93g��e�F?f[����|�G���j�k��D?/2�'@���+3�%�0��q�����"_X��e#G�~�86�����8��f��P3���9�eY�)4��h�|���$�R�*��%9���~~���\��,�X"������S��>z\�a�G2�'���q�M�wuy�V�3�$��4�,����,�0I�,���i��h�������,a���!K�z��_V�
�F&�;[B��������5�Z>a
����>�E��r�l
�KKhXB���j��%twKh[��PWPN,oX��A 
�iXM�s��O>a5mf5�?�~$kjs��s��"�Z�������F>a-��Z�<�>�E��*�]Eq�a
hX@�s�O>amf��>�����n��_��&@_�G�j���h��:���7��^|Qk�k�f,~k�������F��0�������`L?����<�b��9ty�s5���+w���z��N����	c�8?�J��l�A>�8Vn�Y����(��->�O�@�y���w�����_5Y}5*�.���-�^��v�(�����BM�`n�W��{^�pH�P�(vL�m[�L@Q,1]<
��!��{���|��s����ar}��C�|�U���#���������?��&N��U��0�Hd��&W�����@5�T��dN��������>��%�Hd����;�'����F
�F��Jm���H��a��]V�������\!�f�x���?9��8��Eo1�X��KW��	���
9���d�L�C�2e?Y:����L�O���>�S���3M9���d�L;D�0e?Y:��L�O���H�hX�������*���-
Z)���E��=��h�f=n\�M����s���k�\�Rr��.4>|���J�vo�X���*��5��RY��_u�C~�5�k-5�q��������bZ�X���E�:����W���Z5�C|F�nd�����K���i�.V�Z�U��7�����R[���Nh�=���UmFV���h�+]m���*����t�E�v�Iu�����M!Y��U�*5A�*�_�*�'���rUj�P��	���-��}��z�����y�4['�@CM��<\�J��Z��-�[���w���.i�Z9����e�z���6u�>JI}�i5�\-5B��m��
���t��>�v+��S��6�K�JQ�ikhJ���U|��T%N�(AY�R���A�E�W��k�YC�R|���6���BI��%+S�Z
�J�C��5u��:MQ�F���Z
iU��9JZC���.Q�J�W|�����qN�Y}������m
ZvD�^4�Q%%f��sw���k������PQ��*���������U�[|���e�jD����du���QO9��VU����+j�����U����e���_]��:S���3u�V��Z�.I�;��>e+��.U��R�ju������t���
�I���)�I��)�2������)���M�.�m�K������[|��"��;��l�|c��m�i�k����W�x���`������������_	�k7����#�\��	i
},��DJ���|���p�i�����i"��F���vL���09����W��i0�������������]���u�K%�*W�5A��V ��o���6�<��Eo�_�7��S�x�O�j�G�|������O9C�6���B��!��r��x�1[o�0�e�t�y����4��Zm6���v���2�P�v���A��~^��6�����8����"�g�|��y�n3P���.[E?���|u�
LI0�^�z����den����o�E�v������_?�������C^�[^#���&�an����xw	@.������y�G0^�X]����s��?�>>?���C�����_m��7�� ����?����D�� L��3�
���S,'����9���9%�.�����'kO�Q;Q���5������������*������uE7���v��^=�����}�<?~��{�v&�r*��\;�����g���g��
��i������^N������1+�Lw�`m���r4����
��oo<P�t��������a'�^ ,6w���h�X���U�N�[��f��ylzu�����D���`�mu�1qToe~K-����*(��
�~����-�Q^��~F�O��x�,�	�� ��E/^����#)����~�����U��7���u���:-br�P������x��E��@vM���H�h�Zmu�pLT��~`�����"?p�:����r����	t�^@'���*�]����%���a88�_���H����p��%��1>�/�G�P
�8Bo�G�P�����#�H���1z�=*��$^��oD���z#&��e����D& Z��Z������ix�������!��������l�o�����>,
Q"y���'����6���������hh��m���	����l�?&!�����	���7J���*�S$���l�i�$��;5�����Wo�6(�p����>�}D_��r$:Q����*�����'���!�AN~��'	�����PIU�%�&\��,��|�2,M�H6!t�	����p|	������E�F�9IYN? =
�����Y���6)�T@����%�XQ��i�����U:_b�WZ+ci��n�o�g�������<22B�$n5�V j��W��%l�@}�������6Nu�/���-���l�v��/���w�:������&��O�-��cV�a� (Z��*q�E)�6sZ�HZYSJ���k��������C��������lk2L�q�z����#�B�OS=�-c����j:����S6q;2+3�z�c��_a��WayNW��X�h%���j�h%M��2��eiu>xjt0�CcF���;!��L�!�X��6��A;���c��	���s���|�zpT��iy�TaF�/�p*S�X{X�F�k�Z�H���}��G]�%V�q����Y���s��*������������z\T�,�Kf�p�g*4XT�5,�l����������w�����|�>�*)�p= ^��!����Mz��@�m��4Q��R[x���(n�GG�f�m�l6'�&�%��c����&oR1n$h")�1�#����������e�O�H�f�[�������pw9��9��sP��e������.��d\'�BjW�������$@|���<�!���R�~����%���X����Nr�����`�0���n8q����n6�I�nlK��dS���b�A��j�p_n���%�v���A��}��.W?I�O�J6��~�j���\%qQ���K�~�
����R�h��.����}^L��l��E�PP9c������,��=���������?��}q���>}�s7;�����LO2ha��y�������	�����=F�P	s�>�ts��wo�Y�)H�����&%
��������T>#���s�i�'<�����4A�@�# �����fU���/\r]�8��
���K�j�Q�V�-Qj��GBhA�~�&�S9�E���|������I2:=�
&'���d����,N�#��L��XM�2�����1�v4=M&(hQ2MNb5Y�h&���1����'��Nb5Y��&���1��5e<>��d-�q<�f<��N��d�\�m���U������%�/n��%�/�9�e+:	AG����Q��;�s��t�=G1�(�z�$<����6���<��?�1E���\���.&����	Y
	V��<-����>"��)D90���Q��@������������Q�:T�x�3T��5r�U���J��X��Y"l�	Og�_TW����"$��
�
�`���P
��[���y��	�:��)�rxi���7��7���1��=g��#���;'A�1zZ���,]�c,�K=��8�bE����ahn���<�p\�6I������LiNJ+��)ke:Z9J8�qD���r�p������rt�Y��CC+�)gs'�F���N��-����(o��=f�}�(-��}O�`����2T�w�{Jue�P.����Ovdd��U^vc|��a�����37��6�AA��1r���` F���]0#gQ���
b�(
37AA:�I�q�jp�i��h���)��F�o�#��%����^������Q�SMR��f����Ff�D�*�G�o9xe
J00uc�=fI���0R�p�U&���*'U���
��>�����N7W�P��&&���	���+�27�Rzc,Is���"�aC����O�D�uM]��o9E!p'z�1�B<��gG�t�6���-��� F%�-�N6x��\����=C E����]��%3��.G����au���Zy��|,�A:
��I5]O-U��5=z��pp���\��k.���n�<L�	�9�Q
R����%��S���2��Z}����E��%Wj[1~l3qaKc����M�������c�Y[B��������&
k8 L�>��nZ��R�D
�������b���V��Fk"�7��j�d0��K�i�dP��K�i�dP������rL�)�$P��L5+�dP��L�(��^�Z�K��Fn`c7����
��
���O��%��(+A�4o��J�+��2��R��R��J��&���*_y�R�o��'����c�C��c�p�>�b���JD<��zA
�F��)�bdR'�A"O��Y�Z�|\-�u��W�Z���v;��CU�tf��D����D/d5&�D���FEk|es�4�_�R������.�]<�q�� ��H�l���)�]�0qi�����,Y:e�f�F\��sq9cI���q$�;��m��r��8�
q�����'4B�����jG��G������B�v>���n
�/'�_�����nw�)��C+�}h�����mxD�u�+��4W>,�'�v�-�����j*"����uD^1F5�6x����'�����l�op���j��:��}\���|��D��o������Dk�p�}|����Wk���|�GV�v�<�l��g`U�/���{��#�A�r.)��7�+�
�MT���eW����7�@��+���J���b;O�:��m/�8�P�����>Q��D:���P>�N��2]���t�|��w4�i^�i	'S��U�k��2�T�v������
*�8�e�
S���@&��#gHe+�9q�T<6B�G�� �
~�
)�����0���iZ$����M�9�7g�9����n%p+�����<9�(�1��\nF�R��X9�K��#����K����"�OUs������W�\�n����3���\F��2n$�I#�L����\����'u�\�6g�Tom'+?E�M�����Pt����k��
�����wi�m��
/��>��]}~�����O��%n������7���������!����'��o��ad���!s;�i��{�������v�i���4��=�osO����x-�4.��}�������`�V��&��:��^+^�vr\n�U?������n5J�������O��X����v�0�TJ�wT�����M�H�6��M4iSM��&�LN���lMZ�Ii�����&m�I;����ip���_T|]XJk�o���s���T]a*��bW� _��G�/��_�8OL��s��p� Z���f�u�dN������3���b4j^�
�N�&�ti�O��0�z���L��Nf?U���YZ9K+hi%-��������V���[ZyK+pE�n���h^({�"/�yat�S6�E�D�B������(�+�J�D�E�Bi���h^(����q��IEk�����:����r������f;_���d4��{�����p8���o���Ia���u�����Bt�Z<)������e�O�hX������c/j{����^���PW���Z���wrDz�sYu�x'8��ukH���\C�_F���~)�����&�p���tK�����[�n��:����v�Gtq+���@{�)��[O�I��z�����S������O-���L��\!L���#L���&L��<)�6��^Q,��h������*h��R���kC�<�]B	J����H1���!(T ���	M�>�������k�j������h�
}�p�w��+��-�2dkaQs���H�-4��1Iu*~��f��Y�J��	��,�'o8*d�T�������n�V]���/��"H�$�`_H?�5���/��O9U�7�Ca�]WbRJ,��������J4B�s���\�p.����\�p.@8@5��!���\�p.@c���*��s�( (��",���u.�@u5�p.��/mj}8���s$�p.@�&m�B4i��4D�.�0D���C4��MQ����z�����&���|e��h��W�#xOdXy7�"�m���dF$��W�E?�<Q&T��`�����M1 g}����%�!����(���\�o���_�x���F�,�����&�)_��������z�����r���1�!�����r����u�X��#l��Q�R�/i��=s�6[���Y�T���
�^���K�S���{����/���p��
�M�r���~���~��k\�G�����FInx�����vw����S]>���-]��yz� ��t�Fh��d1��,.oc~uc��L�l%S�KYX�L�a)���6�R�Dn����Ft;����}��-eria%+�	2�d^���FX����PV��hn'2N�lX2���A<���vaQ��{�aQ�:���������_TdX���&_aI����5~�KZ�$�k�R7�����:�1}��:�1�R�:�Ej��%��qc��_����)j0���=�T�_�ys�N	h�;��{����0�4���P���F���X�0���#�1��gSl������]B{M�$�$��C�����d���C�i�Q�����������_�3�w+jGN#�8`�Vh��1�b[u3/��p�byAeE��������['&!7������a��
~R����Y��R��O���v���gZ�$����Mb���,���$2;���oN����,���$*;���Ob���,]�����g��1���f�s��rW5��2s��!�eG��p���B�����IIw�@��`�,p�Z.'qK_\����k�W#K��l�����N�
��������?&mpf�����)��6|�T�E��Dwe5�����6���]�Z�9�>��d�aMeU���2�Jku�"�.�<���FBl#�h��d#��F���s���T��H��0#��joQx�KW����;sT�������gM����M���o������{�2�����n�B��1Z_P�a�@P�e|T����mi�x���O�cr��nhx�"�;�g"���U���3�0�����	�����xR����=���=��L�������Z�*qXe
sQ�T^d��+�����0��|����(M�|�A�2��f�_a�v�[�a��*�K�MO�m�������?G�����d�TR�r%-QS���7��`��h�h�m�M�y����>l��������P�|CA�^��f���!��������5��,�/�0H�m}����l}�|�����+0��~w����~v�R�$h��8��������/��9��I|]�?��!��)�9�G�K���km�%�-Jt-��C[t<��8�4KF��ZU�������!(E%e(�>k�AK���r'(���Y�>��_�H�\�(t�n&3�R�|q�� ����pE.�����%����U������X�:������T�j
<*����EP_UW:4�Q�p�m� �48��tj�����E���������\f���0��@�z�h�0Dk���u;���M��d\k����|�CS>�K>�s��������w���@����|������WW3}�����s:��8~���L��
����?D���A�Es���������=�OV��	��FY'$�N��St�x������s�Nx1��p�����D��LJ�B(]+fqz" �Th�0�>b����UY5���F��>���/V7�!"H�x+W�g���J%#�����)N��U���W�:�m�R��#���aoNMlR���D�[]N�
��]� �hs���F�U]u	E��h�JX(���C�{��'^uT��ON������Z0���*�c��C|S���CrSBM�&e��?{�t���}*��ja�1����.���X�O��7��a�+tX��
�V�=����D���F�;U�5��jS�D�������	��2n�0+�%<\b�Ky��uH�5~�
/�� �����]@���p�.��R')��������#� _D�G_'G���kI(����i���\�kI�.�2e4��#�2T�9��I1oaD���W��U��U*��_��fT5�&'w���UD?! �E���k�s6@��Aa�Au���BZ|�o���h�.�K�'J���Y<K�GB�&��zn�7��n�Y*_�w#�J�pdI�r���HF|�w��� ���6+/����x�����5��9>6�
��V�J�	�8.�q�h�tUl\PJakBIs|��z�RM����Ip�z^��-{��l�x�_�
��iT�C�a����8�F^��^���_�!���L��0-M�����9&�� �;5�q�y
����iH*12�x�q&#C�
@As"<|�8��>����~���!�}1����}�`c�����b���G.�t��}��������C�^��2+��y�����=l��z�H��sq�������Xk��N���s��U��E�V9;-��UW�;a��2����)����~�:�r6Y�����~��[�������6��N���w��)���I�����A��W�./��'����o��.�t:�?M��dO��s(-f��&3M=�e5���,<r�1��l��w��@B���4~W��6\>������ w���;a]!������DcO�U�?��5[E���n��uig�Bv�!�p����1I--��Q;����f��$�8���"h4���M�H\�,���UcX'��N�qV����:���8�
��>�nX�k��
�
l{{��;��T��0�J,��N��:5kCq����i�a��������ijoX{a�k)��e�/��E~uj�hz�6Z���^.��v�K��O���]~�o���t��;�LW��Pn55�Zg_Rf`��$}��E��������M��^��������X2h�i���ML����IK�)����^_;�H�]����H�j���B%��P��[�:�w�H)���v=���X)�o��BU[�
W�q����w[S���)
�x~s�9���]g��l��o�`BmWx���i����&��=�DW���8Xc�����r�����&��5��X�>�7Zw��������L�~!���@��h]E��q6�"�m��&�Xo��{B���]�\m"���&����2��%vv��7�����`���R������y���q����IH�(�H�C�
2�'�K%�O&S)����B���Nu)�;��AR�!QI� =k�ir������7��!�q�_j�Q��&�bj�����h��	���9�)���>S��6������#����Jsj�g���)�.��{��y�j�n���y�F�h^���ZkX�Pd�U�@�a���|L��g�B�5�s����,~�W��:&<p�3��^����e�&F�|,}��o86���o�*_��+~4��L�o��!��D@�6���7���
@������
�7��������&1�9CP|�����k���5�u���f��1���.
�=<2��Y[�:!��J^�(*0��T�#��	pL>�T�������|�q
q�$�I�\�y�q�!�	�|�qjrR��w\�yO0�D�'��T��k�����)!b�T#1�q�Aq�K���s�1��ho��w��9��)���!J���0���f�a���q�)�9e0D�qV�����������������y}Y"�X8������Ar�I FL'��.c�]"4����Q�/t�Gfb��C�;�|������B�#�]�A�!r$��#����G
Ftq�����/�@���2�U:��}@�E��Ev��[Y�R���f��]y��=��'��-��+�lp6"�3>\�"�}�p���sz�h?�2��PH|�mL���H�o��(�E��06�/����i�_�~�~�7B���Kr����tT���l�,�/JGe������b�
q��_����"��%�E��,�/L?Ee�a�����(z'�E{��,�/J?E���0=�eIaz�����(}�~az����tT����PY���Z���tT���OPY�_������)*K��OQY�_������3T�������=���������Q������%�E��������HC�/L����(�%���cT�������0}�����)����tT��������(�%����H�o��
HQ����tD���(����tD���0�(�/JGe�a:�Q�_�����tD���0�(�/JGe�a��j)�E��,�/LG4J������tT����/`Y�_�~�~a:�Q�_��h���O�o�/LG4J���QY�_��h���G�B��(�%����FIQ:*K������tD���(�%����FIQ:*K���6,�,�/LG4J���'����#%�E������#%����FIQ:*K������tT���#%����FIQ:*K������tT���#%�E���
�4(��+���h�?s
�lC�c�l�r��F��!F4�px���t.:��
�^e9��o����AM~[��)�����P��q�uRUs��q]U��F^f��x�:��b+�3�o��{F�5��O�C#�K]]< '��� �u~��x���A4�'(�<7ol#8D(���$V��Q��{�n��G��P�$��I����K,N��:n����"��KP���zR��/��@G�8:J����D]����C���K�}}e�z6����LV���v����V���O��CL(�Q��7���7o/�o/.�~q��l�
�-�q7�'e7� �V��V�K
��*��w���.��UM�R���Y*�T�}���B�i���0 
�7�Gu�:Mk�]��>Uq��A���?�D/L9������=_/�����^�IJqb	�e��jld���N5���I����������NF�6Up�~�IKK����K���S����?bd�"�$xIQ�;�{�#��S��E�\�l�{������b�������I��^4�_
V����������R0$�_
�q�3X�q���<f�����/Hu�G����d1�P��>��a�����T6�aD&����/q9,��:����4��/Q�zlS���_��xcA�F���Np�r�`A�N���Np�R���8�
R5pb�v�����'VA�N���'z����/������.5J�����P?��Cs����!K�	\����7L�	�|s_����#�S OpD����1~����1��H������\P�M��I��=JJI9)�>sy��7#���q�����������7c>�	��p�AIR�	�f�3��L���n��%MQ}��
^��Df����W�/�3��\����(�@��9�����|K�r��hCn#���&��dnZ���_^�F�����Ly���fC�M<|���E$M9^<�S'�18\xn:���1X�AHR��)�7���x�X�Y���K����M4����)��������4�������cdk:�Yn4�U����"���A����R��g�m��|�j?Cn`���'����>V�8����j����hk�;Z��D�s,4V����6�m����z~�����?���.�6��������j�f�y�H^cr7�Z���1�S�����Zb��KOgE���F����nsc����5�O�&�����`�'g"���8���D�!c���6P�lS�z�03:CJ<�`xb�����/?�T,��Y�7eQ��}N\{�u/�Q����>��`�p-�q�z�(|���K��a�������3;���!�A&�q��v6�k�rK���"�W��\P���c���)*9.��9B*�6��W���W;�z�`��WqS��l�����z�=�L9|��	� ��R�� H>���AR~8A�A����<�
�T;�Y|���
�%��b����#�d����34��M��R��fb��v`��CQN�C��`����D��0urzr�&�o���T�q|�J����TM���xC�j��x2�n=�"��8�g�2�7�ql��T����$��x,�P��!�D�
2�����D��Q1�d�����i2���Yi�i��'��
e<5d��F��t|:I����	��1�%�O�� '�����W�-<�j���DI�\5�O���J.�-X����8���
{�V}�;�Y�.�$���������7`	/7E>�����:��i�;T��i��P�%
�
n�>:X����Tu�f��o�P�;�CR�]������@������W�^�)��������+�w<$w�#��!~kzD�(Lx�w�#F���O�)vq0=b��<&�����	�n~���	�n~D�C�|7?B`��xH��G����5="�S�r7?b`�����G<�M w�#��&����&����I�������H�xD����G��&����&���'�	�n~���	�n~D�1i����4����4�������&qDZ�#��!~kzD���	C������	C`���������G<!M��~���4�6?"�1i:l~D�#������Ip�c�#NIpe�#&M�����8!M�����8&M�!����4��6?b��4a��S1�a��8�h�(#����`c�7����a�&��K�#�M���K�#�Mq�S��#���OIcL�v�k?"�1=bX\�i����q���1�G;������1,���4���`���S��#��������)��)i����������1,��i���bR�;|i|�������4<"X�����4="�{�&�����a'�iB^l�k����1,�}JczD�#\�i�����5����A�~ixD�\�	i�����OHcL�v�k?%�1="�S\�S��#���?%�1="�3\�3��#���?#�1=BX��#�4>"���qc��v��&����"�	3�������S����`S\��4���`G��#��#��������~Lcz����c��#����OHcL�v�k?%�1=bX\�)i����������1,��)i����������!�x����N����4<"X�y��1>b�S�tJ^,�<R��,�<R��1,�=aD��s)aD����0"�G�9��0"�G�9��0"�G�kO�#���GJ�#���'����b�#%����b�#%������F��a���aD���?����G�9�aD��s#��1�?M�K�#�����0"�G�kO�#�����0"�Gk�e"=bX\{��,�<F�1>bX\{��,�<F�1>"X�y�#b|�����1>"X�y�#b|�����1>���c�A�Z1���N;�H7�n���
���jT��-��Y���� v�'���b�;:�l���'��Z0\X�BaCH0X�N�B����94��C>�����|��O1������}B��#���~0��1x�<"^��3�8�����x�o��M�zH������_q�����������Y�H,h�aR��i������	c���Z3X�O�����b���S�KG5��$��X�C\���j�}`�������+�e`���(_����������V��$Nv�9�vU&srac/?2������;%v�V�*�Yt��Uw���P����H=+��b��jz:����3���r���y��b�mtZ�������-��'����;�+o���8���,�/�	B��Ez��K�Sb��<�����:�H��f�����KO����������CRiz�5J<h��d�hJ��2 t�f���s3�����?��Q��f=�zV�����u��|�1[g�E����������Po���������lvr2�e����2�z�;����x����������t0�^��4	��?�Ff�Y�����������?�l�������fu��d�-?����&���Ay�y��[=l�V��h�����|�����������hUPi����
��|��o�@`p���W�������a�G�|���7� vW�}�09_~��3���|��������^��5������z���B�����^|A����T�h{�!�M����i[PNW$�������j'�4���Yo��n�`4��nr:���#f''7�y�Mo�h��&����a�0
)�B� �`�
���!c�t��%D�������&�P__���y��9���,��<�Jr�K��*��ce����]� �l1�q�w��}�7�&�~��"��"��x����
c�(��G�*�&
;��i%���?s��f�W��|]����;|��{]��x�������J�)�6{Xl��G�<�m&���|q�my���|>�#���l�p��]�#@�Pd*�$�8i���[�J�����N��ou�4��B�U(�;qt	
N��a���}��e�0-+���������|	r@k|����cwN&	��K�Ykg�v/�V������|�[-�c ~f�6l�,�}��5�" ���r�#������>�o�� ���N#�!L�n\@@\��`��d@�7���'0�a���� ��k�"�I�pQ.T&���X�4a���i����'��� �K�Uyj�P�#�?�k(��0�t��W�h4@a��J�a��� k�B��Wt|���n�j��(�("�9i�
OD�{d&
�Q������>��;���D2�����tVQ6R�i	/w]�G���R%2��\%��J�'�]��^i����[��D_F�q%tj�p/D:�?I�5K�>���P���W0W��X�w�~���1�w��+;���!{
d�w����<�9->��a�g�-j�1a(�����R��dx���(�l8�����2�$���WuT��}��rXP

����
Bd�9��� N�wc�2!F��e��z��:����M�6U��#W?������{�rf]�nSUl�c�E��x�"�K�<j���Q���Z�[���]Vq����������^�vY��#w���k����m"w�u�=z�LrwY��#w�Dw�u�=z�LvwY�f�+���dw�U�9r�LtwX���v�$w�U�9r�LpwXC�#G[l����|�����j�C��e�b��Q7��J>��
�g�����A����k�\[;��w��Bt:��]kj�_
����}�-���v�������eK;�c�Y���l��{YG���0���]g����|v]�O*�����|�q� ��y3���K5n
.U>y.�=�����N7}u$t�U@�|o*���J8W�'x�4V���<p�N���Hu��	8��s8���J��"w��g��
8�V��9��+x����^��N��	8�Co��9��C�c��9�<p������}���b��	8�JyJm��6?x��.��li'�u��	8�������Qg��F1v������8�����������-���}���5��9E��'������\z�K��lZp�	.=��'�������G����x��KOp����~G�E�K����o�����o���U+����\z���}�[��'������7�����!�1���
.=�Enp�����T1������<��vR�\z�KO�f�����:�������l��p�A�<�������'�y��-s��6�j�s,��&��"f.�XXr������y�������$1��4��3���T���v��sd������/.���1$V����#�68��	
�yG�Gp�	�;�y��j�����w��G���Np�	�;��`p�	�;���w��6	�;��np��P�����,]�yg������w��Np��z���!8�B���������_����{�@��Np���C�Sjk'���y'8�tif`K;9���Np����>������
��2CT�������.��/�Mt�E����������5��g����|�c����y����w�Ayh\~���}:w�OJ���t��&�G%6��4�k4���|T��0����Bm���M��<.hw�J�������jNO�����������%)6�GT��
Q�#*xD�����Ne�GT��
Q���G��T?xD��n�=t��&xD�����	Q��n���P��G��,]�#j�����<��GT���z�G�!xB�G������_�����@�GT��6xDu@����vR�<��GT�f�����:xD�����l��xD����4�@��~���W���\��E*>�H�v���|���T�|o���j�-w��3O��<��� ��p�PL\~:�w�Dw����\b�;���8�o^��(�%x�o����Q���;�
�(�%x�X��(S����Q�es�QS����g����!x�����C�
�(�to�}�v��V�F	�(��{�
�(���qu�(Cw�7�~��Q���qU�(��O�����o������}�-����(��;��[z0�(�{	>'����G� ����W�Q��A8?��:TH=6Y�����-�������.����>���>4��u�-�,� ������nz�������u�Yg�.��9��C�t9�,��@��@�Z�]~4z���p��SsT�.O����2�����5����D�%�JD��"��:�e�=2�1������V^u����#����^���3��Z��D?����?��~m�.�?��~m~��?��~m���?����6��&F����}G�S�d��~���nk��z��
��`���������g�k�Y���Y9W��Z4���5�/U]��p$sT�E�&'���+8��u��=��:i~��W��\��dqD��w�<C��8##�����K���V�v@��k� ���������B�����W��n���9�o�W2����kG:��#��w*Cu<�.=��Q�*HJ��]��er��+�4y=jg���F@<@�;}���v�#����d�o"������g�����V��;��|��/g��8\���p�Z��j�{�W$x�A�;�@Ja�uP��R�b�>k�M"����[��ZW�����\[������h�C��{R^adU=��{�]��@�6M��v��\!��%i�E�t���[8�6��T�:�y���c���Z�����UC�����u�h������&Sq�O�yIe�.,��������y����[]�e�c
�V�u���Z���u]�.:���:��Qz�]�������R��u��:��u�I��./w]kQk]'���mx|-�V�uy��X���q�^����nu]���5�!?|��1n�����1�������,�1��\F��W9���2�;���%5-���BX
��}���q���a�;5�F�:\j���v���:U^���	���A�C-���.%|�hS"Q �V)J�H|�3�!�m���hw��:�T	������._j`v��o�hu[8���8��!��0
Q
��`7q�y��'�q�hEh�	?��y��U����gQ�m�-Jkg3j��j��Z����+�������:�<�>o�y�������{9��:e������5z(�Uw\������}wn�{*��p]�Rm;����]��7�~�#=�y o+��yH���6v�-n������[��>�Fx{�3�{��AV��s}w��_�	9��t��Z7<��t���7�{-�FW�n�w�
iZ���;x�k��\�����iH':��|@����n�"���t��:�t�MwS�<�OK��U\����ntT����]��e�k
i������h�����"��vt����Du�!���.Pk�N-���Z�<~��w��2��o@Z���
8���{��g�~O��7����C�^�����9K�����pH�����C��Wtt��o�5:n�5Z���i��S8��`��������n����up�����Ep���.M����Zp��h��up����Op�>�Fw��U:�[jK��U4�ZC��Q]4�\K��U4�ZC��up�>(��b:���X���n�"���t��:�t���������t��:�Du�!���..R�kI7����T��	��..S]kH':���T����n�����t��:�@u���B�����Z7����H��|���g�~O�u8�l���	��=iwk�g�!������zX/W[�e�m��2�2X��M�]E���	kd'N�F������X?����:��
zQI�
�5�8-��=j�z�u�F��?G�^\����
IC��oZ�;2���g�������B]���:�P�49�P��j����
.���:�Pw�>��� \��W��B}�-�FWu���k
�FGu���s-�FWu���k
	.�����\	�����ciG'���\@����� ��vt����T�Z��������t����Hu�%���.S]kH'<	��Lu�!���.RkG'���KT��������vju0.4Ok����8 ��	����=aw�����'�p��]��W���y�h��?��gw��|��o��/w��m�g���Q�?�w�����Q�N��hk�L�E����5�C�v� ;�r}8�[��� �'Q�@�#K�5O������<�>������D��&��#L��FT7a�	E<�;�@��y�{v$�����Vs\>�v����z����1y9���s��B��z�Z���������������6 x�����������lD�\�^�������]�AK��5��E[����]�Ak��5$x������< F�c������.r]kH':��<@���
3�..P�kI7���KT������"���t��:�Lu�!�p���2���t��:�Hu����..Q]kH':��T��<�!��x.=i�uc>'����'��p���������?���v��c�\_-�}�����j]��7�{��S|�w�����y��.{��
��<}�������Q<<�c��� &>�?��;�&�������0�r+W���W���������[���7x��49x���j����
^���7x�v�>��� �z�W���{�-�FWu�
�k
�FGu��s-�FWu��k
	^����������7�ciG'���\@����� ��vt����T�Z��������t����Hu�%���.S]kH'����Lu�!���.RkG'���KT��������v�^u0^Ok����A5 ��	{����=a�����'��p���zE7]|��b��F��s����o��f��
�����h���>.��CU_����������=�m�m�st�mgw��������8:�v����"�e
�������o��������@p{�z�p���������M���|7�F�����@p�Il����o���:�>p�|��<�����X�]���0c�M��7q}��m����x�1����+T�����>���6��$zC��/a�69��M�����C�*������.[K���_����T�_��zF2���*����p8���-����!7�����g3���)7�dd���+��|��?F�)����G>�� z��O����#����Q9���9r��>��q%5N�p����D�3I��It=����g]�$��I��3��g����o������������4��!PN�Y�k��>G���k���<���}���\?����D�oI�~K��f��m�i�-��[��~KH�U����6��Br.�N��>,3���x��sj3��Y�,NC3��r�j�V�"���>�@_�DK�7��*����4.xE3���m�d��I��#=�E
��a	��hI��[G�nT@��j�X������
���%���EU�ak�v��^:��[��"g1T8���� t�b&���1������l��M~?�M�M��g[6@��~	E��C�3��s �����'��������4"Z��������#������nq��I%�x������L��1�nE��+k�v9���H��
���z6@�������ib"�pL�n��H����X������"X�l�4�������z�a�a������P������f�zj��5B��s�8������J���b���������
���*�<�j�I��7�n���~q�L7o�.X���`1p'��n������P=��,0�"��)�-�9`���~|�����v��4�o^>�F��x#�o�������i1��b��_�����t���zn'7�Qd
��B����GC;���E�HF�#�������t5Fz����[K��j�v�
<�������lg��~����A�����'����>����5g�\V��\����U��t0���(���bhZ2�8�k�6����2�h�I�R���-�!���A�ceZB���0.:�)�W�@�Jw\]4f��vX���
E3�D'�l������D �a�� ��>�5���Z���+hn���� �|��o����&r���������9����N��dM��������I�����"���(��C���=������OS�z��]"�.T��f���6�ZI*h���hRx)_���~��
Q���p������cv@�������6��V��|��/g�Hb������g��E��?~�O�����G
3o��h�V|����[L$��S	=qr'����H����j�m����?o���J���oII��~�!(#�X���$H�q��iJ������L���jjP��n����{�N�9,L�Q��Q�u��?��GC\�NeNV�\-��U)
�b@�W%�[����6��2�{�������}m�Jq�BxK�1.9���p��N����AtsS��Q���]���������+��:����1�wf�(g�����>�������z�k��
qy?�5�7*�i?H��D��� 5�P$B��#�F&��n�5.�*�D���&i)m\*~�VZ3���h��Z=���F�s`�������A���7�59.�aQ���1�����RP�&1-QSTGbK&8�d��v���$���;�Q	3�
&	���&"%��%�n�u�@��I��0p0�z��(d�6��S���,�D�E���-Z�.�t�:e|b/zL��D��M����h�L>a�&T���+��Vc�Bt�]Q��!��C%Ct�]1DW�H`�NT"DW��9�Bt�]1DW��P�]���;��=�Ft%[���rC;����k���-7�+�+����78�������]q�	��!�"%;�3���jm��Xi0<5<�����0DWl�~!��#k\���+���>������]��O��A����C_�N���!�b��������;�\���+���!�b��x���W���D����z*�z��H����]�+�����1GWT##B�a����i��|��>e��|5j1:������e5��|�+�u�_?l���:Zg�y4G���<�9�\��z�G8�"��!75��6�"*���+��!y�����BX���:����6��Q
9�L���<����#B�M�
2������j�P�!��C(H�J�P�!d��(���DBAvs\�P�!dBAv��!���4x�m{��J��
���v��=����[nhW::��� ��p��as���}!��*��C(HJvgp����
��`xjx� [�a�J�B(�G��
2��� }�Go0tMg�&(w�PS�r7(w;����� C(�
r�� wZ�
2��� C(�
�#=��j]	u���a�Tb�<&Z��X�=!dW�SA��y��!I������x3�E[!������.��/�
�x�x���(��8_�2~���?�79��h8G!x�j	�#�!��A��qpy�={&?�'�	Y/d����~4�.k?��tp������:1�#�c:T4��aK��1~�y�� ��f�t<Bc���%3��15�)�d!CH��q���!$c�5
!��1�dT+B2����W!$c�B2���]�f��d"�v�E#��-�dl����hO���������!CH����vm����i@_�������������i�6�d�4�BH��jB2�R����5.�d!CHF��Q��]���	��'����
����/h'CH��1�d�A�BH��V.�d!CH��1�d<���A�+�YWB��rX=���=��Vu$�`@O���A�T���B2��Wq���=�}�W���b6���a�'��X��+f�"G�d[�w�Aq<�:Eq��8yZQ�S����H�����h�%���GS4��Q1DTCD������M�(DTLBD�QQ�D��"*vs\���!�b��"*v��!���	�w�m{��J��Q���v��=}���[nhW::DT��mp��as�^�}!���*|�CDEJvg�����Q��`xjx[�a���J�BD�G��Q1DT}�G�/tMg�&(w�PS�r7(w;����CD�Qq�wZ�Q1DTCD�Q�=�(e]����a�TB�<&Z����=!�bW�SA����HCb�R�e�S�x�O�R1��>����6�}��#��`�y>����E0���J�,6��'�	�8�M��q���=�^(�� �#�����)��v/�cy���zt����|4v���F����$-��'��
�fhI3%��-�f�J�����Ft3���R�7,B��"3��!2w[~Bd6Y�"3	!2C�L�!Df���qBd��!Df���j��O&�a��Q4�+�B�������tB<��n��]��"3��^�������������LpB!2)�y�A#�VkC��J����!��l��!Df+�!2Y�B��"3���Y�e��5�����}BM
�������v2��!2C���.���i�B��"3��!2C�������u%d��.��S����hUGb@���]TO=O1D�j��#dB	�f��@L�P����G�L��^D��6�|-��F,~��R��������i�)<�`:8�=� �!>f���b|����4����2���.C���2���.C���2���.C����L�X���H�uyb]�X���f�u�dB)v�E#�/��l����hOo���������.C�����m���?o@_�u���o��������i�6���4�B���jb]�R����5.���.C�K��QT]���	��'����
����/h'C���2���A�B���V.���.C���2��<�P�A�+~\Wb9��rX=�`�=��Vu$�c@O�u��A�T���b]2G���~���:_�W��g�ls�o(���;�l��X� ��:G�|w����~�F![��Z����fxo�8������G<4�#GG��>G��41��[����M���h�;������������x�����1QY�+�Z�w)BP2��S��:
��e�"
X+�c���d� O�;�I7&�ZC~��������a�����y0�	�G����%�����l%G�������>����w���,�M�Um>E7���n`�PI�������1��Jg$�\]&�-��|�X�i.�{���)�f��U���hP�,�y6Q����#�w^O��a����Moj�ism u|������eW�. f���K�p)�b�N�I���()�1wuMz
>�.1��<\����P�2������>���>�:��^����@�nA/��S9���m���ic8S�P���\'���	�/dK_%�W��W��U*~�SQ�Q[ZG��H�PC�@�m����P��>�W?�L�{+����6���Tw��[Z�K�f��k�F����nkf���|���Vkm����Eu���yN��HFv��o�_\F���\�e�;d��������r�o����#{��uf`���z����on�l��m�/q���D?G+a;�������r�&�E�%n�7y�����b��vix���:���T����QA��h����}v�u�>�A"�n���g��k�>��B3G_J�tz:��Lq�'g���C��Wo�3�����&uvh�)f�i�|�y'P*��9���=(��A��0-�������M�O���]�9�����
:3R����w13����3��gU��*{���
Wp���C��e-�����H2��HH��N�~J��)���A���$��xz���\�,q�8�����mj�|�/h���A��j-880�L��=������L���|$�g���^-�iMhMuZ�-,��8m��������]�jd��^�n��t�M����l5��O%r���TOa�+-HUX~���������M����4�	�6�����/.�D�( �k
-��R�8,��'��I���������]����A?k+B�]HZ)������ ���_u�G���]��b�����[R{��
$�:���p'���yZ��[��A�y����;I~���e������.D�Q����P���>7�&�Cn#���&��������/�_�LNq^g���9a��e�+�a������
�9�:��p���!�"�d�1'�_
����� �8Vz(���%���h
C�.#��(_���
���J�
��
�����|�]�L���UC�"e��h��m��e�j��!������!����@�c?Nt�5i}���:$#���H;\u���8nw���q"p+��o����8���=��Z���-d]7�#�����y�H^cr7�[�����)��Dns-�������j���;��p���(az+������$n�A[��V�^��W�-
aO�&1%QR����������#��D�vD�V�t����j��M�U��*w�
��Z�4m��=E�<��x9��K=)����]E����q�kg�x�=I���r������$9H5
���O���'*���>��8��'����q��/�:�����������+h�w�1���������nW(���j}���������:����OPb��M&#x���!��L����t�L��0��a:%����@��f����_����v��3���?��h�9�>_���4]_�iz;<;�gq�^__�>�����~Z-���G������;mqf�����(�6� ����5|���r��?��X��>�������{����������d'N�NG���8��?\�#�m��������k�~=�/�K �Q�g�+�3t]�s�|{��'��E�^��u�)_����-�=�?.�{Pi��p!��^���y����W�:_�p��-1�-�m_�Mv�s��^=,o6'���|���������.�(#�>~������w����I���
v}�U��%.	6��Y
�k��avc�o�5,���>�?���<��9v��a��%���X|������B���a�V������'\��H_b���z�k�}�%n�V5�K��H_ ��$zu���!��=������xL�G���%�1�>��S�p2����b;���c���>����/^�Fm@I���Po����'��Zd779.���5�,:���z5�G.?���_}�����BW�������6�a,W�O����f5��"��z�k���-:/sZ2_��fP���,g���'D.�hX�,�������j�t�������O�	��(t�
k�V`�o��k�"��z�������r	�X�|�����g�L����������j���A���l��D�/�,%"�Pc��`N�h�sm)�o4y���������<�GI�������OPD��>BJ��8��32Q&��qqE)'����@98wZeR
�������R�m���
������L�����VK��3d\Q�#���U�o�oE|�<d8MQc���~�S8tk����b�k>�^������u~��w8,���/o4���}Fh�����R�.��k��N+���8�_�������������cNWh
x)�u�w�}t�i�O��p� z���Z0�-?�nB��St�5 ��l��/-0'~L�1Q'��h�������)Mi�|
z��
���9�^���������#��[8`t����8]E�=86�;���e�{26d�����.�3��%�[�����wn Rl?lr�X)�#����Qj/�`$���H��$������cLp'<!G���
�B�*�j@8y��+�L�������
�*{����#�ao�G"!�������|9[<`2q������#�����|q����UQ��|y�������<��JB������,[����zN��/,��=0�����Q+��'
�����f��Z�/@����&�e����Wr�Dj���=@�(����o?n_��3T��q�=]���l�a���n����t������`t�:�f�'k��&����-��|��,(����{��+���d��EH����9W�����C�����PRZ4 �����z����L�D��jH��\����9Z��P��K�hY��-��47�,��|Da�N!��`��	������V�P��)G�X��m��z�1p��(rL����iJ�6�W�<+A#�P~�D.��T��9����Hs"���ac6�����OXY(8���I��������,�+���#�N��"X0�@�i(��J���8,�f����<�/��MO�!�4����Tpq�!nE��>�Q��p���C�-]��x�`�}��Lm	b�����0�����(6����Z��~"�@�~F��Yu��,� '$�g�0_(V��z���F���
�D�G��#|�$�QZ�J+��DEK��@������U���`^�� aIv�U�P~Ds?�zp�D1Y-�`�,��r�(5(�B�R��~)�%���=�\�����M�o�z�g��z�5fUp,b��h ~�:|��{�,����]�I3M��7��=�C3T��F�����Lh�=Y�hxu�u�g/��XRi���'���nS�@<*�L� �^�D�����v�u@���O�P�|�!���@����1�p�]e����R_���X�/(��i�R�BU���wQ9�1������ �b-/&/�6(�Z`��(p�����8�!�1
�z��#D`z�d��A��3�!���C�����������q/��g__g�_������[ ��w��|y�5�c7'���(��&�����k�\��h���4U>�����l���i2�^�]8?�������
D���h(:����`���0>�GP_��Es�����E���z=�����?�M����t��M�[4O�8=9��]O��8����h{�����s���G�q'�$�^��HT*L���"��5�Z�V`$q���@����z�3x���H-���9 ��$�o�q����j���_C���tm���$����;~6_n��{�|��g�$�xV(_�a�l^m�o���p�����{4�`m ���W(�XMJh�w9�[ u1��$�J~�U��C���[9����^C��-`���4������<I��kh#��$��k�h���+���M�q{�?*Jy��;[ ��"pe�F
2����x���0��������>���#�<C����!6����q{r�o�KZ)�s�U�����F���u����C��� ����8���6�-S��a��?�f�����'�����6N��i u��w�=G��(��L���A9@
�������p�E�S��#�{
�����e����+@.iU������:�}x����?�g�1�GU"���N:������d�����T�44��b���G*���O!a[�~���I�Hh8Q��w:��b������7�����P6���h%w2~��L���59�O�n�]aX���5T��-���`��S� "����UF\�f2��{?���n�D��x#�R����n��@Fv+�
����J�����j@�[h�n�(��������>d����l��@��������;�� W���Uv@&e�i��7'������Y��4�>_�����x�.f����a9+�����D���O�sV
h�i�T`���
���@�L� �?��-x|Q���U�����#�XmV�f'�4Kh�9q@@N���LW���|��������s4WH��,��t�D���+�WG`���g6�o�����E_q�#�������J�3L���E�YDf"&�|���Y_�v
{a�B/vc�R����������8�-Xr7����~����Q����<p�N#l\'���9�uY}u��:c
������8+��8A�N@A-�PvY�]��c�QZe 5b��a:�3"t����y9�E��/!,��8��/@�j��@nD� ��<$P���3~���@$���TB��gQ��4~�Dg�|�K�0�@��+�kz$A$2D�!�u6��r��g����cb� 
�~��s�^����D��
&@
��x���* o�+B��F��
)���K�Zc��>a�Z�}�_"���Q����-(�9�P�%�T��X��������
�ESiu��Ck�������������l�MC��-�$���zX��p����h�b(��F�A����[���*/
:����>.�?o'kX� �m�uHN�Q�a��I���XSQ^�R����X��%~�X��=�xA���D/Bk�D�,`r9����V	!0�5����K����/e�u�EZ�$��N�M�����Fp+l���PU�FB��|b�e�/n6�r�[Ip=IN(s��b�e�Y�}�/�Z6�������Gd�V��A��
-l}<2�$�?�X�K����M&�41�`���xa�0n�Rx]�����������&��F4�yz;����Iz��e����Z�����@5���`
�	��$���V��'��sXe8�7�Oz���2����:GCmS`0�~����,�8V�w���tA����|��+&�\Q!��m�P"�_i��0��+��l����s����%B���8����gXfqS��#a�O�l�@?&�v9�����c�v5(�q�������� TmX��k�r�"lmUb!%V�q�T������r����(*��t
�.�����5~I�1�4"��#�$����O������k\t4\]9���,'���O�)�b��*�P�cN�B.JP%�,?��N�bHR���:
��_�[l�>'"e	W���u.n�@�n>�M2��� F44M���E[�,�9������N|����1��T��z����&/�����%�F�U�o!���4cCAef�X��������L��Jb�i��(���#�}���0��qT"Z� ���1�A�
���e����h�0'�����3���e2����L��X�t������#�b�����K��o���>��V5h8��`er$��|X,�i�\vp%���X�@T�����}5�:��u#���i����
��EL���Y|r2�f������4����fXH$�#���SH7�_y}Z�o�a���������#4�7��i��o�R[��l�0�e$y��d����H���|s�Z-8�t',��w�n�:����5�YY��A�/�l,2�k���"��
@V��}U�4���Z�ic�-�������(�>���%`/#�g��F�{��3f��1/��9cFM�TALKx���5�R�F��o�F���E~�a�����D���9*_�T��=(wU|O^�,(�|!�
�����
�?��B���>����~H�UMu�i��w1fQ�7������1z[��W6)�|v�m��Z�DD����#hlX�
i1 ������w��\[�;��+��������g�x�#P~�#"��d���I"}M�u�K=�����&Od�4�BUXG������B�s-�@��BZ�pU��,VV�c����RQ=0�np�rN|r�]�p
��o8Fx�}��o�p�G!�k��:�{8��� 3��Bl�!G^|�+xR'�~��T��f�JSZ�n7���,����r4�'SM����Q/,�+��d����1�TyB��M�k<"_�-7�8����N�#C�9j���;@���C����G��	�������{�5�E!T\i
zq_ ��z1@�y|��6�R��"7��FT�"��I:��E��M�d�8���-�B0�����W?���;���:���#$���w���{s����������1@���nl�7�n�5�Px�{��x�����W��%t���.b���+#�MR h�=��>:�[��D�c'b��bs���
�m�b�@��`EG(���7��c�-����P^�J2���8���M-4�r�}�Z����C�@����l��RdTs�����]W�+�gqI\rh	��&_�r�ji��m>p��r�A`����e&"'��@{�8 �PaM�TD0+n[�h�m	�E��@&���������r���r3��"�<��>x��T���H�9�2����Aw8?���|7�D��'�E0��p�c�k<�XB�I��"N�sD���0eG]���w�g]����mr/��8���]
z&���>R���5p���2��\�XU	�6`l�Ilf���K������.�A�'�U�;<���t��
��{/	��K�b�{a�����7�^���U�O������7 ��R��Y&,�O^����4PS��a\��n(?>{FHl���W��t�i��%�I�!������v����������4[�1��rf��6�f����;��
��%f��#�O������?��~u��������k�X}�_�{��� �_��������F��?���A�9����(�&z�
6��9,�@�7��3�'�D�1�l>��/6���_�;����?�`��?��!t�`��*����U	�K;?��M	�t(.~��������y)�B�R�y���I�<�2,i�]��h��xS.��C�d%�1}�7g�Y���������2T��K3���$�F�������?c�S�b���DL��{���?�.�T	%2F�����L�dSha����j%��\:!�`Q�k*���3��Pg���K�^�b;A�`�D���sA@ ��53�`o@�k&������]�'
eE`���0m%�����nF&cq-K�6���b����Qp����������5�! ����t���G����$�?�����K1J-�	Z8��l1�f)��Z��"����E�/"lH�%�7��D��*�H>�g�	����&`(����M�&����4�������V�K��2|.L%�k��C8 �Z�W�Nt6Q�L�
�A�<�C�N����?��m�!���F��A�e�R�>rG��h@���$Si�FR���@��E�J�9�Lb��������5��o���n�������'���k�v�o��������"\M+]X��(?��hpip �o��,5������
Q�����
8���9����7rT*C�=���`����R�H��S���A��LK�h�C3����:s�m�	|_#[oxNrD�T��� 2ER]C����x�g�	T<�k0�
m��/�PAo�����P�cP�+F���
9]t�,������������tc�w0B�8"�h�Rd��g�����;����(�L����(P�Nh�#J���V��\�2>Sa�q�"^�N���b��wd�Z?�E|1������e-���73���:%���~�9o�:o�:o�V���6v���H�F����q����vt�Xo*�y���O�Y~��P�.iXI#\X�z/��
�!&��9G���|� zp(Es�TO2�93=�����PE�02"�����+VN��q������G����D�����C�E�|�c��k�,/yRV�[s�o��H�"������=�����hmFt���*)ds����;�?�'��������;v0L{�����Bj�������B���j�/�h@��_�{\����"���9��TBI�yp�g#���M]?�7C�
-Se�P�/�Xoq��P���J3�`���M���(��u!�a2|N�qa�c��F6��B��%<���&�
E�g����t5,�dJ%���B����-�O��B���r0�H.J%���H��3����a�0�����X��ZE�3�=���Q���YL��5XI�z:����Rx�(�,�
Zj�o$���p9��%��i��8�	�GL�����D�	ie{�_����.�VkW��_��������o��T��C���q�����
|����h&6=e�.�DA���Ha�[����uhuDe�-�_���x8����HnXFD����9G�S}���N������d{�K�A�A(
�V�Ac�J��%��f]��MH�tR(�P���*��%:xUS����������������p��?�
r��r6�f��:a�sA1���"�2`A��K8d�������Zsh<���l�����H�D�B����Q5�	�
CD�3-�����(����,���������c�n�1�_`��cv�#� ���oF���J��V�rm=r�a�M�J����8-N8=���M_&$�p%��0��o1�F��!5�,>E��dG�]U%";��������m���7�gn���
���b��ls�h+��V���~�M��u�p">!�7D�X�8K��D�N��	n7�D������k����/�"��R����7#��0����o6�X'��8��z,���;�����@EQG���sx���A��O�{�kWb�1Q\�Z	�.[n��f[:55� �������$��n�FTUOn*&;ij���9�F|�0�L�3��Q�>$�|I�����]p�3���+�66���I8����jp��|h	L06T@�&W],"����:g����H���ta����<�g_��~(��(6��j��v6$K��Ta�E^���:��#��H��!$b> ������V����^K[�`�0kk-^�Yc5��[��~��hR��##�K��G�����<O����9������\�D}����7�] <��[����DF�W������t��$�,3@��E���Xg!6Q��EY��8B���������q��\���B��0����_|#��W���(c�m$m%�p��A�9���\�q!_^����.���������yN��P(�:�PH�#�'�=#l2�_����$=�����@w�	\Y����D�{M�Td?�z��J;�=|($��.�`z�`9GK�GP��&�	8Z�gj�s������X�A��B�Wx�1>�I�����b��'�B"�����.�J��8`p��\1Pa�����9�,}��4}��Z�Zg	�kj�v�>s�����V?*�;2 q�w����o:|QZU%����4K���n���*b�*�EN>��T�"k�
�{�����`�a� �RM������&Y��!8c�)aO��,���i$�o#����v������\��.��&�-���E�������v�@�&�5_O��0,�Vr���'I�#�(2S�����i
�X��,h�D���7:�3�^����,�����a�/D���3hY����$�l�R
=����F�����
���X����$�S�?���t7���xt(����`�`>��/C���Ne��^�S8���	�YA"$��Y�,cRk�:�	�pu���pX#����>��^���)��d,L�6x�U�����zn�6���j�*r�`E��l=�~��b�q��
���9B9Kc�c��02���ro"��)����v3+?�y��Z�g�����	)
N�&����\�������M����:$���Bsh�]�z��=�lM����(gR�J�:ou�|r�%��.�%�"�a�sC�#���W��g+5w��LC��� �y
D��j��)[��>�9�S�R�O{/�#)��4�0�|�p">�[������b�k�������$�e�W	'r%1����$�f���E�����13'��9�t�p�U��2��hst�/��%�	;g��n��|��3N��m����
?����0�k���a���`W��e�� �]�7���4AT?w����]�3����3|2u���y��\4��%�]�N�����o��k��
?t����nTv����������1D�d[�
!L�����"r�4U�P
X�9���\�cD�P�skk)�2Kv������^���%��*X!H�(��q:M������1��Kx4�_Ct�Qa�B��|�B1D��G'T#��\�#��9��O?.7�/�_.�/_����/I��)�������B�K
��(�X��xrAtP\de�o�/_�o����<��%r#���<��V{�e�tS��[Ps:L5�#��9���ZOC?;*F&QN�e�@]�V�-���4���B�o�}��-c����d�!
���q�x��leB�H:��0NTi��VAi�TT0�k�0 ��XW#���Q���sI�)�w�?�����`�P0������D�b��X3�8��Hu���������u��G�FX�P���%�?��/�L���X@eo����;�S!W��v��_P���a������Hq0��G���3�w���;��.�����G����"��d�`E�-h���2��U,9��@��T�'!�eA7qu��-��q:���{d�P�����}��`�p:�I&�h��a�l �I�9��$�t����n�o	��s�
���/H�G#iQ��"�{�R�0�
�����|�X��.;�y+�)]���#�2��W������F�C��)�����O�b�n��w�-���l��g6���s
��l��]�'?��]��9g"�%�A&������@�2X����)��<�h<r�?{P�=8bu&\�%29 �$z���k�^"3�B�2x�R�@f������$VU�T:#�9�c��9-O�L��+�H.U���g� }��h:�e�f%��[�>�D?��@Gs�}�����^�HeB�
o�\ �2�<��$��
�� �-����`)�B4�a�:Ww����3>�P���#��y#�C<�H�c��>�x�'������W\����ZQJ�=���Qf�\���	�GUJ�a��:���&�>��o�mYY8CBo�G�!E�Y�_t/�	�	
�����P�mb[��W�'/9y��c�3Uq����g��4�$;�l+�����p[�2���zbHUR�Ty�x���I��e]�Y��t���N����5����g���O���x���c�M��V�A�-zv���@�E��~����;��!��"����CA !
��]�4{8�"!�es�i0+"ee�D�r^�����>����V��.bL~���,H�
jnD��KM�T7�/�H�Of�)���/
����\��A���9�S��jAD�Azql��NoBX��b��1�\��+����N�5���\R�����D����IXU�uF���*i����Nx&h�tT���QAu-�QNk]NG���&��>�"��>�����LG�D���:��tNr��1<�����fz����(2�Q��FG��}:�a�0�������1��&�d��tZEH��B���
TK�C������c�$�g�G���)&����0�b�:I���^��	q��� ��b,Z@�&��p%��5���F�^�,P���H[^L;�����6)���T������Q�,���~<}M���B��`��j=;	��ll[M��OSr�,��6VT9���bC�����3�u�����)S
Y+_��aK��0�|$oZ)sTl�	$��j_���&����������*�vN�����=�e���6L)�1P���G���<���v�z�H��?z�Q�����u����- kDm�Q4���`�)E���V����y�N6J�t�����=�P�=�tF)�C��8����6�P������}~�Z��F�u>';����t��-�F$H(c�O(��,"�G���9��f�y����=�p��c���pbpc�qe(;:�$�������Y��������X�����&���j��/��W��s������q_�+_�
}(V�G����P�=A��`�i(�8���O���C�V%F'I�K��Y���>����]82��a��Z )������0�QT�����Z��)V(�����:R��\ �Dp�.�bo.��1a�s�X] VF���b^�4��@��.s����q�8:�R� �5�Cc'~�8@`�}:�qn��BN���z�CE��_{���e����<E��
�a0��b��P�A��D��s.��]�/:a���"���B��s�FK|�
��
��_3~�����sT�6���,�]l�IV��3�5z�e���v����a��-�8�>�X/C�����l����_�Q�;�32y����'�DB������%gY����|�"�r)�L��pIh�������$y�)�i��������''9����\� �q ��������f("���-�rK<��t>�4�=%�C������X�K1K����d�t�Qx}C!�>�������3i;�
���f�d��o�]��6�D��j���=�n��b�Mv����u������s��)H��"5e�f=�����h��L9Db�����uu���ZY@�d��G:���~Kl�����C�~��/���9^����|��7��������Q��q`��G�5�������;���hG���)Php��I_Z���z����O����1�D�|�2�����$�)���Z*p��p����J�zoQ�����iS�%��`��h�AF:������,�#�LmDhR�1,E
�����"�������������	��/�����3�]��t�@{��9�_:[!>�nq�?������p����y����G��R��R��p[>��.����{}�{�;���=^l�5��s�k�pUoA�48���?����%���Y0a���
��
}�6 ��L������g����f�;\�7\�3\�/��nS�@Rt���M��jDu^�wC���h�����f5��1�����$=�l�J�i�p3����L�[�
��2���D���O������lr�t7B�������[��N��/��JO�B<i8����e����oWbc���b�/����?��)�p�RVa����
1UR{)s��d���N�"Vm��j���.PWi��=����Y�<��&���D�~���'�%'��}a���GL��+p�h�)Y..�K2��&�k�7Wz�4�*��)��0�������O	�(_%`.��(��q�<����~��� ����g��JW��;�!��q
i�K)�����K&|������cu_�=?����p����\d�&�C��a�i������������t�/��]�c|�����v�r�].,���2	_
�9���:�7�ZUc���U�(�m�C#E�;%E��7����tl�0EN��[�$��"���m���(� ���7`E���"t4"�Bfy
B����J�s����7�\��Iw�6���)@,�n �
�7Hq3P�����Bi���I�����<�o������g�6�,��/��I�L���1�9�5�y���k=����T��������H�>w�����[�F���;�7�}{L7T��	�J������r�9���������b�z��P�_��e��7�W��E���	�xZ�;;������).�����jA���(f[��IF��MK��"XQ��y��Kp�U+���iE�3�\�<��%�sU�l�Ri���<i���������5@���,X��i�H]S*�@:�yO�p��J�K�d�S��^���wc��^�9��t�,�Y�}#

�;��Xy%�����UX��+%>�����.j�����#�������<C-�1��_y��%z?I%��^z�:�Ty*_eO�]�,�Eb��jo�	Ik����IXPIg���;���F�4#���3!����6��x��T;�����=���1��g�z���[���;���#����Om�EK
�a��R���2y�Md,@����v15���&E�C�(��
�8�����	��X�u��+GG=D��"�3 �\z��#]�ck���*��,>��sW{���m���00:���s�]�����Q��H}��]z)h��?:��n<UF�t���9�y)l�S�S�G�:K������^\���R���3�F�!V���xD�20WK�!���,�9���	�]�b��
����1�A� -F�����lGN�����Z������
�c��*��&A���?J�E����#�O����aU+2��"6=P�WJ"y�z��Gq�z;�O������O��^�B<���1�|�Nz�7�/��i/���Wz�M�T��'.�Z��i�5bI����4,�Y�V4:1?P1���Q�DK����r�I��;+G�h
,��a��N\wZ����K����j{��-��T��������������a$	����EAu��4��Tl'��X����y�-�u��������������{C�M���E�q�)�
��W�
%�:����T/6���%��J��e�U��)�8p�g��5�����_��@7
����o��8��:�g��F1�B�����k�y!����nT�]��2��fQ� �E������������j�������]���R3'�ZY���AZ�!�Y-��:��~/�J���B�;��l;�����e�)n,/U��H6���[��B�� ��#�F�T*N��
���y~#����w
��]X��}c��r�Uw�l���j��M�(#����R-aK:'vR��Gd���j/��+��x
Pa���N�,��e�?X��4�����Uz�1�e��y
���`��5��V�<�2(� �2dOx�r�	�;�����$]82N?����k�*���S�.]w��e[��e�f�8��(f@��
~��)����/C{�	�"��$O���]2�%�T�Nx�j`l�{~��U�������B}��h����c���r�$4��#.�O���;����/<ryv=��-�L&��4!��M
~�a�8���
�@��|�w�>�>���g�����WD
���f�?{���M���������9\�������&�XI]��oT.�OtX�
�����o2��|y�|qz������v������`���	���i#��J��]4�B�B���..����
1����1-+a��Z��s�����4�#d�H�f�X�v��A�Q,!ZO:P��P�K�I�
	7"&+z|���X�iw�xK������]����{P>�@��3�|��:�
���"��T����GF�<���)(��Q�N#`����;%��`�[C���iw��W������h�T�
1��R�,���D��������pE��h-����`�
���r�xlk��G�k���*i
�������5��|*jiQ�����/d#��91k�7������lp��iE�f2^���;w�J�;�n���f�}B��QT�O���c��}s�Q���9X9��l����r���B��%�Y"G�yQG����j�����/�&.�G�2�A�����<�=���$�
��h���YC��#�#� "�h��
���f���!�MM6l{�#�a�F�r�������e��!h0Is�e��XsavM���������$>|'L2j�TW�,���g�iG4�lX.�)
��rq�M�##���4K�oe�X�������X�)�g�T����w��Z
�g8(�^�0��%}K��_�q���0M���_��a��-��g �l4% ���a���l�N�fS��(B�>�Dsy�����A ���w��>_��Ss>���l
	�����+��d]��� &��A�
��=<+��
�\���s����sK�����q��=K���7K�6T�%/���d(����|���ENOM`g&�/� 
���=&���ZR����-��qOHTX#���H4�f]�T����sZ��X�i�<"�����b^��^j^�.����o��>��p�)�`Q	����"M/+�i^DRj��tTd���� ���UI�����uzG�Q"�
�]���[vG��0����G��1l*A�HP��t`����YX���x���
��w;%���5��v,1&7U@	R�,s�l��L����x���<����%����V�aT&����B�3��G��}��W��7j�!�Z`�A���&�I��y�:"$�%B�MDH�!M[�S�4%>x�!�yN��� �������&���(��&3M�u���\.�����I�E�����N7w2ie��r�,�~Q�B�_m�fAi6�Iy�����	;Qu�b4i4�0
2_4��K���(Rs�r�s��#L�1%��c����L���EQ�TV�f�\����j�%�V�n82�\�4L�lc��N����yMWs�������5�T����R�4��O{�dh�P�8F�� -@�T�A�*�f�T($�M9

��q�R�m��T����f�R�E�Yj�]�L]j��O�y�,m�=p�][�I��v�$g����b
	er4��5�@�P�s	j��|5}�����y�Xt����X��$�\��iM{�4�v��^��S�z�~���tBkl�Z��Z��',/��j�G!|�[�Z�����r����I�5��U��)��F��m��,�O`ZX3c�������#X��te���z��������F�a���V�,�Y��Bh�����B�S�k�*
�SC$�*�{fG�
Y���L�����XL2�O����������@0~6���{�4�����,=�pv~s�{J�
=��zq�}�g}
������v��f�ufy���L�:k�*r��E�@����z�a>����~B=�a4�C���a<����$���@����|���^^��e~�+W���^/�{6������M��ltq�����t0�M��$gI4E?a��.�A��zO��(���0O�	#�]��}���;��"�oV�<;�t1����Ia�����q�^f,����?~G��$�����_���	���'�������_��M��F�&��j�g���
�j�
>�����^�����N8����f�M����l}E"-5��]�_�fp
I�L�y���!����?$&`0<��8��]���Tz��n�Q�%?����Gm��}~��7��&G�8?f�:�l����@�E$92I��|��G���
�-HJPR�AB��x�wa8��:�M@�`��w-�
���} �#��Lz<��$�,�A&�,o������j����Q]�@��S��,��r��A��A��_u�n�Zr
�g�d^�x�QU�LS�c|����\mlN����dh��7<��/��b�m4�K������%�z�n\�����<��$�������b4�����(����w���2��a��(��pFjl)$�s(W�XjY#�@�O�~/���h���&���GJ�h�qd
'�@����sw�V�m$��
�������Ph����KB��^��e/�M#��-����faP�A�p�q���'C'�m(;�>R��=~�+�/1��t�L<I���N=^���P�#[a�x}��!|��)����������p����./��a���Z�����x�����.��Q����DajaBp<�*�<�7:G
[	)��$�SB�����b��&q4�H�m%��TB���r<�x��sJ����-���r�g\���w�9Q-����qgWY�������f�d�j�^�[�7�:l>C�/^�������?�8=���O��|���W���(��X���������G'x��!�����Y,�6:�������G�������eEh��
�O� ,m��1q�h��?f��u���pA�U�	�������!�S��r0*y&i{r��?g\+����H9���e��|~{q1��CF��E}f�9������
<&��0��,d�1�=E��7�~�w�>�1t���5��7;@Z����#u�Y�-P�W�+���]����h�X*����XhO����o�a>r���#��F�v�
��#��G���T�m~���O�A�MZ���������t�����3��M������8�^,;�
~�_��rl�#�fv���W������@jL�\��Kbx.���X�Y-IXaD�����<����y�R����>)����|��S�����"���J�1��*#�TJD������x���Q��1K��n�y�X�a������1s���Yz��b�a�����|����I�1��hE�D�B�B������O Q�)��d�[��c Q�5�[L��Qv}��gH$����Z~LO��Y$�I�Lz�m��a!��|�R/���8������f��(�!��T������G��a�����Si)�����x4��fr���@����pt�#e�G �gD2�h� /ey�qn\&_�%e����XMV�K
��������Z9������i�a�
���'��tF�r.���g�;�9�^U�]�{���]���
��M�j��Xo��[
5�[A���X��J�6H�ZM���!��g�=
6��`3`�%X�9��cS`L�F�(������6���B����U��!�&�`���w���3�,��ow��#���~kR��J��z�s�7�"���M��P��9'O����X�m�,Y���g
m���K��J�EP�Vj���uA�0L��w���w�Eb�MQ�36���vy�����CH�x0&j<"�
�G%Lr�f���U�>�-0�"4��@��=%@3�c$a���� �
�D���bD��YX��ld�$xq�]��V������:���36��m"�m�������+��`5�����F!�[�����h����\���9X��X���f��<x`p���D�@bz�U�'���1�V�����/%��8�I�JX$�C��{����|���j��(��
T�.z�	>z��\:Yc�����G�'|�{�������"@�Q�,���+K ��m�^���Z4�R�N������|��H�p�����&b�r������Pe��YY��I����(�6���x����q1c7�3�l�P�-$C���N�cnz��~���#�8���m��
�b�����K�b@5��I�M4��e�=�S�5�A����JA�h�bO���=(�PH�z���6�4�*W�L��4��VNe�9��'qo��GT���e���k<������;x�a2�D�4g�n�WZ�4�L��[�����xKB�0)k�7`s��� ���Bb
9�/2�A��]���3��sn�a���J��
��Qe�U��������3���D���u�+_�/����oDW�� ;$�SA7:*L��P���\����K~Du����V��}B��r�����Yb~U���[�#��&"E��!ba����`zB�����]A�*8�P!�c=���
f@`|9�j�4k�jU�n��m&��y�_2��-�����gb���]��"q_$q���0,G6�����i�%^���VK�{���@d �""��(a(�����E��)���>L:"�����@�q�� y�i^�_�=J"���W�E��qv��b�������T����?���C~��A�=&�y^����X��{i���).�~���Q44��gx�����������
��l�N�����xp������a�3�D�.�0�~��� z�������&Y��'QEOh���I����h�����O�izyu���W4�������a�W�%����IR�\����J�c�[��v���67A�[�\�JF	�������0��v��;3�T�Y�'����x������^|a[��J�|�+DmG=PC���x��{�e����5��g�x
=_��HS'X�����y�v��H=?}q��z����"�5Sp6m�	�����fG2p������j� �X0�nzJ��/)��oO������;����OXC{���>B���&��vs�n�P=
#��$!P=g����C�@3���y��������|��������#��E���x��� 7���^����#
MD@�+v@z���7����/(A���~H����c���V����F��6�z����F3�>g)"����e�U.R�E�r�Z.��K�r���Z��Z�O���9y������_�����O�C&�G}X�0>,>&��M�����#��xb��O'�M��q3��p'���nl�KG �s���E�S�hc�bs@1����p��o.�X��f�.���hc��u4X,Z����Y�=+7��\���	[������A�r��\������\���V����Sx������Y�����K����I|!&�?��V����I��bJ�=P��0�����
���}���<�X�IH�6sq;��iX_n���}s��e��z��?�>y�5��}��F��X�f���i�N���;!�t�	�_C�W@�?�lxN���x����bty{���S &(.�Wr\��//N��p����6\.�%W���R����Y��[��
_�dwO�P�[�o:!����n�,���L�q�'����'�IF�wd�)�y��
���lAm��5,��=(jV���B$"��#�\�_���iA��l��(<��j��Y��������������Z���V���r#�1	�'cZx+#���V���Re$;�8"��������a=��ry�y����`�^�WDE%@�>����j�����N`���I�1������x���:���0��*�'uq{`�I�&�+�L�=d���D�`,��~�d9�pA���	�?����?������?��b�C��H��8��*-?Z+�FN��-��v�?�,����C���#x��	?T:E�J�[���/x��?��C���	G����)b<I4�bi,\��&��c'�L*��0�F��bX
P�K`�A+���P�d4�A2�^�D���-��AJ�%���@��R���������8�5�~7R�ny)G�O�gL?M�i04�v������^o���l@�?�;Q����<�CEr�On�Ox���,���@�Ih���!�	kCB�'�|���K�}Z�/�@n����@����������T��C(�n`(�>�kwDG������"VN �Eo���Hy��$���O�1�-}*��d�{�t"sB[���w�)���IE����i���+�)S���m����Q����rC�=����gH�����;#���)�!�F�=��SI�&����[�&a��Q;�8H�~{����P}�����RR�EaSb�>����W���$���r�z	Z�n,��"H������
����\�aj��?��$��G��;����M��-�@jf�\X]�;�Ek��N�{�5qQ�j,�,��N�A�*�9�n&��������G���>=B�zY���y1����J���3�!c�1��zF�I�'��|��w��}g�4�{�i�D�pA'� w�([O��8y
�c?U��~����P�@��2���!}so6��&���m�5�e��� i�V���(PPl�2x�o�>�!�X�t����e�q\A�S��M/n=��#��L���2��;!3`��"� S�K��RZ/�Cv�����6��*<��:���
�4�L����!V��\�W�Q�_��cj�l�i��)����c9�4"c�M�@���=��M�ea�9q��Gx�$abqj�47VF��2=N!�3=}���+��O��	�W���I%����E"�Y���y��j,_e"��4]]��l�f?d7@vs
k����j�p�hH8���
+.��)�s�4��1>�m��JyP�	�9~C���k���)F��"���NIb����v�u���3Q=�v��IrHL>��`>:�n\��$O�v�C�$��������:{#H{������h��Z�-��
=���}��5����64K����b�hw#��DzYMwcZ6����l���k�	-�����H_��<.�)?i�|4Z���(]j�p����������g�#�<���=�<�4U~_N�v_�
�������t����h�u�WW�I��Q�E����u����-6l��J�9~��k�s]]-,��yA&��`�p�|���g�����B��t���+>5/�j�����6�����������o����Q�"_��2���
�[�_)��J���u��"��{��}a~������U��� �?��,�D[��-	�Y�_��0�)�}����-�2���]v=����Z�rS�ZBh�$'��_X����7��Cq0e{�C/������w�	�{��)��6��������T����6�co�nJ@�cN��[���I�? cp&��H�?����d������~D
�����y]���;��\e����V��Z��0
Jg*����������l���T���*T�8����Pt����G�I�����Xj2=���($u��,�9~8Z����9.o�)d��HqY���HEP�2���V���9[�y��y��������D�'[M�
��Jd����H|��o�]E�o��L����Sz��o!D�����_�3j�U��A���
��Ay�CR��y��A��%q�1��J�W��-J��,Q2�����N�I���qJ�~�d�����l���P���#o
��g��"�����p�e�2�
�;��+�������D�h�"r�����bM
A��3�Bw��g���j
�Q{+���5�?�Q���ew'����%��y�W�
W`�f��1���9*�g3]|6�D�!��}�t<�tJ�)���L�v�E�h����[������ji�����}99#wQ���-�*�]�TqzITY\��Q@P�������>���ZEOq�C��]�g\�vLwc��z#^��Z��3-U��=��G���t�j����Q���I��|�i���g
'9�&Y��$V"�h<J1�B6H6���+��)���q����������1O������/�4����������,����s�Q"7�
=N�[ct��������am��N���\�I6���um�=��[{su��[�y�^)^��F0�G��3~��>����o�k�d�I�4��I������tk~}�����k������c�������l�o��U���}v�^w��
(��]��z\�b
�"{q��^.V�ie�mGT��K����TC�d
0��.m�zU����n�o�l^V��VC��=y!��Iz�q����*P�&��M�i�<��	�lq�6Ur�Y����-r��'3�[�����u�����Y��P������!!�:v��C��B����Mgdc��-�H��!M��=��H����$�{��	y�����w�<��#�?�Y�����
�����y�j<��=f�\(z���U&]����k*<��������	�%
sx{��������S��si�v,�*n��a�SHs�@<*
�����R��KH��M�������i�����
�/!��m��Q��5��:�$���@B���p���7)L�p���@(N�������o:���������C�;�/��cN����]��p$v8U�r:�{�x�/��:��*�����^x�{�8��n���-d������r:�#�'��?�t���3������E
��2?���7,�7��FE��������_v�!�r��r�9
����9�7�
��v��j��T�0�����'���k�������!����B����~����&=*�;:r��3�M�L%%|���_��W��6�����z1��K�=�,�OJ�^F8~!�x4������^+5b��q��Pl�*��PUw��CE�v�(iS�����S���]h�)���S{�9f��3JZUf)z1����3�G��9�!�e��"�2?��_pT���0kC�Ey��
�{��=wz���<@����\}o&c�Q���I�rN<%�1?��!;�t#�}��)S���3]��,�fM�T
>>E�i��~(�<���vE�3�����y�\��2%�c������@{���2����PGUE�C�r���������X�*�,8L������Bf���:�����p������&�����f��	��0��m-��+]y��H��{2��ev�c��Lw��Zhr�8>rg,��9�q���g���.pe��3g�q��D��#����=`���.X�<���0�_Tr��4�E���(>B3G��*�h���V`A�H�,�92�gdK|�	�������K��-�o��[q�[��V����W���j~�����������t�^�q���\YVs��vB*�l�\�NXe:v/h'���Y��sV%�`�M�i��	
o�o��g���"��"^��cfd'%�;�3�G(��������>xG���	m�6r]7����@9�Qc����
�k�U���y^���
��K�2��Sn���M~��E}Z&�����-�,�.�������^w��diuV	�����T�!�w�
	F��F���W��VD�*��X/Ng km��Ch�Y�*�����^{b�������I�Ss����)����6��VA�zl�V�}^{����k�W�BfJr
w�)��u4slp��9�j��fR-���en��*>��,�������i^���
��i�
}^���A[a{�l����g_�=y�j�]��m�[�G��=�v��W}���xplY:�t>�w4W!��Q;���68n�I\*�1�d�X�On����M|�K�U�&^��Q���|��a�-u7�������]��'�
bg�OTWYR,�����Yg��4�7�B"yyF9�0I���N�72Y��o�-�?'>�u�k��g����KV�*Y����Z��E��%������������k3T�Mj�
�b(���h�dI���*�f,Q3��o5�s�K��a��^)���5����9zm������?^�}�/CM�nT��^��RQ���,���L9s[�x��=U���tDaRpD���pt��#�k�I-LQ��`�h\�L��tx����
�2�8��^!g3BO=�]3���cA�������$v�5.�&�I��V��O���������MM���P?�����oz[:��9�J��������)������Lxk�Bq�[s$
���&uKC���"s�������fxS���CK��S������7//����N�+
x��x��F������n�����l�*���y�ALk�'N>����t6�JHqX>�-D�g�����c�+�G{qJ�u���V� �����\��u%����f"o���EE���k<��S�r��^���u��a)0b����p�E���]}�2�x�]f�)zq�n�af�����7��2
�F�cE��O��bl�&��6@�3���e���w~� �<S�z���^��&��Qrn?�d����B�q���7�[(�#�����Q�U��wECCEC�CMY{}mg{�e6o���Vlv�����^v��A���b���i�p�)T?"a�pd.T?bY0�8��o�~�eA��Up 
�[I������of�1�l���5�9p�@O�@�������<���	��:1��0l��3����pk�#��	Y1���������Xv�H�Hi��I�,�t�����9iB�+������]��f��4!�da;��Yx��b�SV��X����#��� ��Y�	;��(�� ;��(��`KdnS���~��j�w\�-2���A���d��[�1���:�c8]x*x�V��.2��w<��TC^,
\J/���>4�e������Bgg��=N��pU���N��6���+ FEK�}m�af�K����yZ��o<+����g�=��
a�Ya���W���k<�-[�h�������8u��}n3b�m�=�
���:��/����Cf��
��9S��=t��P��V����qJ�Bn��I�!����=p��`&<�	M���Lx0��3!�f�N=O�,�YX�Egq;���y��x��l�����n���7o���)y�:�-��V�Y^ey�.�h�D�9���X�"')���6�����vbG��o�>(\��2�:��8;��HJ~�^;�W���y� ���-}�����Y!`�&Mm~�@FQ��0i����z:������0����h=��m���2�hq����tD��h���V�a�������v{�<*������=o�=���05�$F���6@�R�@t�����	�I��Q�:��U,'�?�O��'M����!�����|i
�,_S;�U�w{JF�7���\��T�;i�;[P�KT�����&*U�|��� ���=GuM4��h{��SQ%���(�GX.��X@���\�I��|V������
�����)�k#_`�3������w�=S� ����hQEk���Z�5����FX�>�z�5��!�9�����:'8�i�R�#0�����kj�������|��#x5V���Xz��h�Y���,����`�K�e���?����g||�-���?Q3,�����%��a;%@ G�v<�t
idS��
�5��<WKG|�!���;f����(�����*�L��G��a����]�z=�D/��J�P$�[,S�Ks������r��Z"W���j<��FNp����H+�����V��b�>�9`{{�������PU��X�N��/>��"�.��1����s���9��^p#�u��=��	;�Q@��������<Z�!���/j������5v[��E���� pa�
vyiY�yv�Fu�������������3Q�
-���p�����t����*;Q��Q���NZ��%�2g�E���e�G�
�����"'�u����uZ�<��g����D�UU]F!�g�y�|s_77�%�
�����!�Q��Q�x{(v\�b���5y���"�L���f�����d�[������}f�u����>{|���9��Q�1iNO
��O`��
s�>���k�Y�4=s�L�0w�Br����$�
;���(�[�BfSZf�f���7�e�8V�;�S��Q�Q�z���V��������%j�+�����o�V�ui�!�������{i���
� ����v�w��������Xz�!Oo0,;�!+�e
3�0�M�z"GG��Gy#lj�Y	d��a=~)R���V7T=����+/2�W3rg�����|����j-��\#�0F�,�������8e�h��`���dV������j����8]v����
5\���@�rc*[���/���n}+�Lw��]*/',�l�TX�����1_���t�g�{=p�x���lkF=�xT��1�@�F
1�����h%=����f���r+�,H����<v����4;��l�Qw��,������'!V@4z�6V�M�o�J
VV���+���>�����	��	Ks���Y�skgG�H����ZX�{b-�y� �Hd���d��~[��}��{����u��Qr�D�o���(��
�`a,�0v:h����V�������G����]�Uw!�������p����y{��K4X�7�PS�0[�6l��4�D�A1-z����K/�O��A���S�0����������~�JQ��5H��8�-�����L��8��()PT5)6���5���������w�;�UOW�T�����7���iUZ�^gDo�a��z�X|�*�����+��a�,_�Cu����-�~�[hp� +.X����
7+���Z��	y�����d�-7��`����T+X�>���.X�A:,��v�.X�Qo��7���� 9��.+.�p+sEK���*f�2���%v9�!��Bf������ke4d�4p�����F����4VW���9`���ZS+0��,�HE����"�e��Q>��0��cT��i�u�b�� �F�1�m�U1��es{�}Du����f��
oS/�T��'����ZQ�O�8Bl��B���#W�F����W��/�X�R�*������������TXup�&���� kcI�v���iI���%������w��9_���c����#��X�eZ!2�Ua������������������haZ�������+]ke�����/�Q�N lc�G{�tq����Yl������^��b�1��uK�e�����VY�U9.?�+SV4��Gk]��L�M�*����j�.;Qx��ml�������e�tfw��n��R.�WX��
�X8e����1&.8�S�\�
����%��.3Vx��m���-J>[���x+��Q-,G����SV�!�-�
����x����rs��X�X(�$�$�C�C�C���X)���I,(q�~�uB�}�Z��|�����g��+;�j����;�9�(�W��u7�/Q�*�J4�����$w�$� )L���z���l�i��F�/�Ac���S�B��]oY���:
�H��z����O�;2�fs���?l�x����q��qi�����C�C��l�{W�8G]����v����	�����{D����5A�(%E%��8�~D��Hy�D������D{G���z��$b�}���|������pGyj��'Q�����s�q!��z���t���g1�K:�m�3���8���4���2�g�[ e�d���f�E�B��<C�
������F����?��'��~��D����/N���o��ON��z�
O����I/����<y���w�0�>�c5M,R��B��9� ��7�^��G�������3��x��h��jy6�JWM����?� ��w������,�V�,�V�����~��*�IYqU'oN^��hs�����+a+.��rk�������7o�b(P�W��"v�g��6��������1h�5���ig)YLug���cW�����b����I,��g��Y�^F�D���Y�b�=��I7��z�	�Y���'�CzZvyk~���SO�[�f��g�i..�6:
W��B�M��6��'��>�&�T|w�l�<n��m[	�n�/�w�����J>�n�����
���G_���9�����e��/�Q�����ep?����/�u�d��&*|�
g���J7'�u9����I1#��O�=��u���O��A��|R�
.mn]|R�r���b�pqss���b��[�k�������!`w�R�i���M?���(L�=-o���]��G����	��#���t-�wBq]���o?^e+�6=��9j�W"Q&��I���tq.qy�:�VhrgUU(������	��Qhc��(=D��@P ]�0�j����*w��>�d�P��nZ"Q�����a-�&v�Q�C�*����S���b~wo�}��8�m�<���Js*\�T�dOh��:��� ������d)2S1���� '!�6����_in�q���>��I'��_�n����[\����0X��fXZ�,WW��v5�Q����4�Q�C�m��0X��f�L�!��f<g��C�Sd8V6��J����BHe���T����;R��w�j��H5�2l����4��r}�.����/Y�����.�;��h���~9�e���>��]��P
3V4�x�&���0nwCX��u�!�������kf�nH�43O7�Y��������
itf�g���L��I����13+cf46��0�N:�;�+��iAg�C��C7�l�����U6O��]>�t�#�vZ��6�I�=�8��8����SlGV#����[��%�}K�����I}�xG�<�l��q�
K7���t(���xm���+�����C9����U�%��WO
���������'%M/��������z?K) W�����{���p��_��dq�����~f$ ���3b�g���F�$#b�a�J���dn�B���\q��[vc2�L[�,�Ub���LF�y�K�^na���q��D�+���3t=�����W��U�?��3�=a�"{*��#�*�����a7fq�IL�}�)'{��O��,�_*:��������
��$�M�oU����P��o~�}��=��q�d{��z����4j�L�Y>�����;B-���;�����r�M�I�������g���=y�����p�{l��\��R����bJY����h5�T
�SLM����h
��;q��7�v��n�_������yf�*;p�����k����j�5zV���%���Y�'���W�g�g�&�`u�.�P�@3���!�Y.�=J��U�0��t=��rg]�����yf��K���z�����6b�6�90���m����3p�����x��d�$�+�t&/X��$��C;��uOG����e9���:��/�����u�/N����Gy��s�R�@
����,���������T�=�������0�k �q2�Jq����C,
T�������
�_��������
��V[����F�X�`"�!4�I�U�q�b���^��o(�t���9��Pr0�;��9H�=/w�|�:�vo~�
q���A������b8��]�!U�xX��?��]�9���y��s�����=����[F
p18�H�Ad���t�o:�7��<�I�@���r�~$%;:!��h����3=���k���b�F�1��#�M��?���7?�hb���Z���t9ap0z���b9���2x���� �aunF�#���ApT��4tzE���/��DQ�(����r��`��[&��Sv
��K�=������=QN���'�	5x��:��U#Uagf���8�O�fQ1���i_���G*��Xy-����r�qL�1
Z�EX9�;��R��\���R��
/�$$���Xb�	7l'�Yz�"SS����T��O�Jybk�-O��������H��z&/��e���q������E��UZw�cc�cm�c�0��0��a�+
s����at���0'�0'�0'�aN�aN��T�Riz�����+�$�/�����SQI+-���������y ��p���������7|/v����Ir�E7q��;��3�}Q��Qn����������^��G�/��������6q�\��,_k���3�]Z���}zv�<G�����5M�������T�H�*]��c��d��
����W��7��d8�j�.\E*t����E_��#��Wh������`�c��f��hc
���<���#`��W��/(�.t���R�l�m>���7�D��O��7R�@o��'��Q�1���}�o��
��,��v�����bt�~���5�"��F��9��t70t^KV�T�c;�z�("���M������xs��]h���c>:���c��G'�S����Hh-��bv��;3{%��N$�'�:���(_^���(����l5OoR,K�W�d�ty}3�����`���;@��I�6�Ih����lS�!��|�,�����}zj�zN��vjyVLA�h}#���i�����0GO������������pb��vh���om,����H�nC�8���r#��g��$�\p����n���������J��e���P�Z��{�� 1� �4	���x�Vn�o���Y�D$����)?MHS�4}��ST"��O����~D�R �F�CP$�x2�1Q1�<n�����['�V�3qG�y���	�6?�X�?f���]F���CV�N���YTu0��K#E5V��R�x�A��aY��Q�o9X�:I���'��*��,���H���7���������F�?�9�On��+h�C�(����8_���5Y�+���d���6�S��L�=����#6��2^p�WCY�^*r7��H��>u@u@���e:�G��?#�LY"lT�h"������L�['���� �\��)[tA����P����=���b��l�M>m1[�j�n��0T����*�����{��{�^w�Z�������[!0!07�����J&���)�-��a�7��b+0Gzm!�<>��\e��:+��~X-oo���(|'�Ie�q����G��R����,��?��W�t>���	�������ov�CV���xtY��F�4�h�.V���8�&]��84���Q�T��4X��Q-Fj�OK+�8�s/�C+M17�\��Z%\��U'�5�od1Y���y/�vCx+�7'��ay���������������-��y+uQ��a8o�'���e��"#.8��;W��2�o�w��L��2�o��o#2Je�P)/�0�U����}�3����]��~��m��&Mm��}���P�����P��mD�����S;[���<P��>��g�����Z��@�����s��!w5"��z$�:��k��~����v������#|d��������9m��N�rD�!���Lq%��|���i�<b���g���w�4s4u�3X����������).�y���y�����;�T}V���np���{����x��.�9��6�
>b�+�S�N��:���Sa����6�ub�������*���):�g���o��m���[���j�^jv�}�bg���������z�lD/MBR�J�\���wH�q��Q�sl^��J��i��k�
���-��eJ�f������	���������9��,����c,���.��b����mZn6�!�EkO2���]U���9����9��k�\$������\Osi����4�=��Ls���]�ir����c�T�����Z��0&�����P!�h��f��������Jh�r\>9o��*t��O2s�..����?�z�� �Q�?�������m���A����nb\�qwl�f>x�|�i��(�6>$��p��x��8��x��Q�;�x���p������=��|0��9@$x��h��dt�D=>y������A�<��~��a��1���\�{�'T��0O�!�w�*z�zIiE��|��G�%��iH���7�/?f��p<fRow8�(�qx�{����DI7��m�?�9�)��!��!��!��!��!��=��C �C k@�l���!��!������R�h�0g�Cp�.�������{���-F5zF+�($�������;�B
�X#�M�Nf&�GT��/�)��������{U5�=*������v�;*:DJ�r���f�(1����'qB��Q��(��*�D}?{�� O��=K��M�A�]1#�N��u��>���X�{@��50�����]����9D���>D�0�C4����y|�Sv�h�h�h�hJ�C4��V��!��!��!��^w�]���y�y<D=����c����� �d�K�j����+l�c������{����+����O5+z���Kz/ER����`;r��m�M+DF����c%��0�OS��FOMNLz���0H"d9
|p};_�n��q(��,������O���Eq'�;��R���AP�a�U�O��3���Q�c'�-��l��o�G�����[��kK�a���%��2��%����>���A&�<x�N�a�~1K�K��w�=��������-s�}[Z��#���������M)��qs9���y��e!�2��X�r!�*�����^H��_�-N.f���b�h����h'��;���������;X�/�f��^��U��s���{:���z�.���y��r]:Ho;���\�����z���r��5s~���={��qG�U�!C)zV��B��T�aJ^%�&��ty��� c����W3LKf��K�<#ExS�J��W�4q]�g�9�dh��R`�i���s�f����vq�w=^)����U�N��tU�}���G���������L��(6x�+uj����cK�E�(~�n-����w!����7����'y�au1�X�nB�B�Vu�?�`=�`�Kw.r�7+
?�����^�!�1 ��u�����ug1X��$�>���}����,Z���|�cP��)�p�����g��abE �S�#V��*nx�~���b��%���O�D����"�z�$���xh�_��1���!o1r�.WWt�$�kG\q�����.�|�b���V���ah����U��%���p�;
W,���p `�<���X�r����^���!=��=��z�A�;�>�>p5����W��B��5~N�r=|=|=���9�>�~�z��z5:�e6��5�����{+L�(�J�7�?���"vyN�&�[����R�7Q�I�����F���&��u��	?A����{�S�+�c�~��O�}��+���|S�f{���w�����Q�H'���
	`�O���&�xA<m�IP������*��N����EN���50:�]�f�f�uw�|L��H9��Q����/���M\��&G�����Np@������E�1-���	�9}�0��:��_��jU�!P��kH�j�U���yzd%xO���qs����FG��*�l�_X
��K� vOT\�b�D9!�OT��('�����LTbMT�\�j��Q3Q7j�7�R"��P#J��n�������k��7�'�9�I�����;L�"-B�e�`�+|S��jl&=o���3��#��6B��R|Z+hMw�WZ�-��"Q��#l��&�+Ub7��1��_�z�I�YM�����	-�a)����FX���Y6Br�k�	������E��UZw�cc�cm�c�0��0��a�+
s|����nW����h��x�9��9qsRi���a�D3���P-���&6�F�llJ�b[�7Q��q�>
���}����Q�A�K����?ev���B��/l"�w���Ho$i#1��r������b�D�c��-E/�w���r}Q��Qn���������W��c�^�(7����0���9R�LM+OCM5�b��d0��:�V],Wh����j�`���.���R��j�������oNm��Y3G�	���3a�}�����5u}������}����������1ko����M�A�!�<��b�)�2Y��H���|vq��r������Uw���R��=Lis��6�<@��@3NLT'N���������1n?R��D�Uo��c��������c���e����c*8�VZ��V�i�#Q}8U�h9���oVY��C��������~����W'_9�����J�@�3'�w��J�3��w��F�1m����O�6!���c�OKW('����%%�o:�;�����*4�.�,z��_���Re~��;B���������Zm>�D��#b/�����@�(����o����z����=	�W�o����H\���4Z��3{E�5��=���5y�}	�����������qZ��#� 7b��c�n,�:�`^�k��d<U?�U&�t{���V`�*��y���B�vN���i�'�L���h�x����g�x�����LdLMn;�����v0�r��\c:c}:�]Og��S]����Xg:j^��t&�����L`:1�f"�����^�O�8~�+{(\���W[-�l�9���]���4��/]��������u�}P�.���|�T��A��q��3.��s������	����9�	�s;�����������6��v,?6��h����6J����A��cg�=�Q��/?v`�����F/����0���s
����`�!<���{$?�6�����^�c����������X~�c��p#��v���������a��s;���(�'$�}�	�0�~>}5 Ul����l��j���r���f.O�^q��(3�a���r����=�4�[X����X��I�����d��u�"���`3�n�3y���~��<~��<lz�����G;�*����e��/%Cq��D�]�!=�GAnU����j��2��D�������>�A�����y���F
��60��-Q8F�� �#V�<��% ��j�![���f}�v���P'����e�Vr���Y�P����p��o1���&W^���FIs��{��p����/=�ND������N�U��n�UZ}�,�����x�Uj���W��1o}�q�b�h}"�d�z�aI=ZO���	+��
+��	+L�1�}(��
&3��@�W���Uu�j�v��_V��l�~e�D|2��;L�5��]d����Lp��t�g����v4�>�aWZl�jz�	���gz���b��E(��(��h��N�-���6�O�K��b�7����He1����O>�,?zya��@�f�S�7"����|�yV����x��Q�����Q/p��K�fW���B��K�6T�]1���G����\��	��*�D'����9y����.+��e�UZ���wF�=H����px*
-����8w�oF�@�(����#��#
���qs|��;)
W�;Q�0���Z�%���H���
�R���CC<���|�CV����bN���
WGl��^������!�z�XR�nc����7�)�������|A�E*���6T;�#�d;��h��lQ�
��m��m�(c}s��E}+<��A��q����(�6�>�k��P������_)�Jry��g��M�B�]�E���*���'�MR�vD�I�������3b�\�dwe��uha�e4�7���l��>�-��7@��+��Iq�j��@��$v�c2
��&a4�`p�?�~�]�=�m������{�9��1�t6�ZW�E�T	�7hb����U����U��x���*P������kL�,���T����U2.V2+V��T���=�3�@H%�%!ctZK^
��	}E������-��;0Px�;5O��113����*�'@�$)�F
F��f2~�(����zt��}�i����y���n�W�x���}�_�;�.��K.,����(�G�=���b������'{�0�U<f�����*^��hTC�CbJ�r���
S�*]\f:,ned���q�x��~�E��jYQ�s[�<��|����I�����2�0CFd����iH���Nvo����r�I�yh z%���F���j����W�%��{��m�X��`w�B�������|<�&��q:J���}`-��F���P3YX�v<@-���E%2�h�����K�^�fO��:��KD�g��"X�f�����s'�9��e2L�2��iP~z����iP ���g�G������B���E�d��q$�qf�	c\�w� ��@	��O��V�{&�!
���I�[[mp�$�������_�7�^}�R�*,C�r�Z.����r��\��K4�����r}
OM�����` �w�g9Q��5,J��#?,Y!�a���71�Z�</���Se:�o�����X�EKw�Z��k��6�t�;�#��L��uquB1���#1S����1�U���l�e�x�m�Z���e����v7k�g�&����Zz���fYX��tS���n���[+�W��)<5W_�z|j���O�/d�}�*�/�3;f��C���7)���+wbe��u6�#r�����L#��!J��hh���mz0^g.�F������U7X��"#��U�d���E�5��<�YJ��n^UjU�(�T�F��I���mO����C�j�K����X����<�\�B�|WE)?�Ubt�o���n�+�;�w'G��{��=��
���D���x|����]��<�c�
���SL�^��
�{z�����5��UF�L�bl0��my�i���kBb��v��jQ4h'�CS�w�1����px]I���V�\���I.�T�!=s*Lz��
*�#��\������T&�9��{`r�{�z��KDBV:��n:TKS��
W���"IDGJ�jKF�I,T@���M�R�5��[��V�Vb��hoy��/i����#.-���yk�j�sI�5�t�^�Npe*�sI;!��U�%��U��c��v�
Y�q�%�8�a%PbIo�\����0$���&�!f����qd��7O��@�������*U'���0�����K*{��Y�L�K1���}�D��o��6�f��j�6����o!p�N�s�6S�	���v]�m��?��([�9"����Hj��l�>��Z��"����}����j���
�����F����;a��Ft/��B<m*b�/N��!d����f�#U~�Vw���Vp>���?�����b�$L�@&e��$&eQC��N��:��Z�%� ��u��+�$������3)���-)�,i�A)�l�\��w�|�g�<�q4?
�e�G��������:�w	,�hk�lC�m&���E�������&vn/*�6H�h�6'W#]k��������R����-����n��%�����a^��v<F���o����wP��GJZ��qT��V��?���=#t����N�;:�;<�5��;<�4�z��f���$&���;p�#��;��Wh�S��`�c��8�Z�pSj���@�
i�Ks���+��4�8�"�/������aq/q����.�{��^t0�m����(��$tj�r�����z���������_b��cy�_�1���bdxV���e>��Nl��f#�/�;�S�#v�g�9�p�W���[	�_/1��e�!k
G�j�5����p|��Z�-���b��`��X����3T���}��FR��W�]����b��=�<9{��}N���xX���!��}_"c&Z��Ro�����'+�d�,S����������u�R�>/���)���v5�����}�����a9�2��v���0��a����X�*7,����b.�>%�A	1=�g+B�%��C2K��l�MU0��p� ��8��d2��������r>�|�������q�m�y0���*J��|����������7�,��q�~8}���LB�+Ve7���/�SeN��jN�a��C��#�64D	�����F�Yh���?���`�8X$jY$�!>T���/��lEU����~?��}����`%9XIV���d��${��:w��V�������\�}H"�C������z8��>K���;�m6�e,Eo`A,�'U���W	�#��+']�`��Cg��j�;1�f]R�)(LC�zc��K~I��b�`.��R����)���zX�u�>�\�'�V�#K
:o{���:�l���r���F�����v?$^z`��|����~����-z8��^4b����h���,��!�.��MC�e����a��} B;��[�^����Z3�lI��C3��}4����1���=~1}t�R��=n2z��2�C����@%��#���h����}��9���~�$�E���X�V���g�7�����#�G���������qw�h��p��usy��8;�!�c�n����V;'����s�Mn/;�Y���Yg��`��]�u0������L��^
�a>����~����$
��(�E�%Q�7������Yb����W��e~�+W���^/�M����ae��p|<H�i?M��$9�����|4���'���E�w�
�F��{B�G1���y�N���R��>f�	������
����I���6������F���&Z4FQ�I2|2�Vo���n'���	���'��������R'OD�1�O���!��2�7/�0�����WN�)�����{N����'��.1qo��+,y�*�WD�Xt���k����f1�Z��AiN��������E�����b�^��4Pg"T�]e��F������]�>���$�\�)�C<�s�@,W0��F���}�w�7f��x28ac�m��
�<��3��cK�����X�t�V0/o����?��c�����s�O�,,A������ ��������a���L�u���������c��F��Q:t�Qop���vf�rH2/XyY�%�I�_�$�[4L���3��M�Z\,����r�F����������u'n7��G'9c�U����o�p��O��~������_|��I���9Pb���D��W"�`}4�K7�@����1�Ghr��_��j���	��|��������(���P������='+��l~N���VQ�R�����d}�e����������P��W������<R#y"+�c�����d�S�1�t2��u}�Ye�t��Ip1}�*��Q�:��G�Ar���q��F��d<���7f�^|q1��t4J��x4�������#��_@���Y~�?B���>����?I�iXGN�"g����������]���7����5B(sn��N`���%J9�tq��B�m�/����/`^��Iq>�C+���t��]tJ�3�'7f���$/��J��h�H���On�4�
��Q�:#��q��.����y�`�m<0f�Gz��{��W�c9�\~�����A�(���������7��0��3��},L�3z�S�������c��w<M���
���i���~H6���~=��[���Wc���M����]f�������a�Z���dyS���#����m�A����&.���7+�������\�~�Gs���9|u�!������z�8k�k8��l��v>�j0X��d��+�z�!�x�;�K$�ID�:�\b���7����v+�L�)����x�&0����S*�b"����#<\�d��pu'x��rQ��r�":`�gK������	M��/1��������B��:\�0>�<]�^�Y�����W@v�t�1��a���l���VY~�� �<&:d�$
;�"]eX �5����H����	��+��BI�$}J�~���@��:<\���.f�*��k2�]�(.%"d�!����r���l��
2��2�+�*�VO�����p�(=�6���K���v�c
��j����h�/^x����$�jN6I�<#h
���xf��$�e�`������,4�k�W�w��3��t������?�����w?��7"q#����h-�n@6~u�X7"�#��;��tE���T9M9f�X��"8�z�"�K��j$��[�vP�VT\`���d��E"V�l��/.�:��B�a�/P���?�Y|tC������H�y���e�"@JX
s�O���S��Z�)9�c�
JP�����+4��2<TX��>OW�����`�R�2F9��V	%o��b������%��KL�������h�b
��P�)��������,:r�X}�*zX7�:#�Z���6�HPJ�R�R=���Wx�4�M�lx+(H�F
�P5Q,���7�������RP�����
t.���-���������M���<E3�
2�m��`��2�p���C
���f��������w�}��������4L~�)�)�9^�����%DO)�2HP#^�d��F5
����N�� b�a���B������K�WRL��*�	I~;/^x��9��7������Q
�����p�+�r�X�d��������=&@q���0�}�02b�[��>���R���4��K����BHV��Y��?�GG]j��{�~�Z������Qsf�Ad���9�w�!��`�e�ae�����[���0/\��G[�������
R !|�}L��&0���f@t!�E��o��f��j+2,��-T�e1@6#����sUfZ����s-��������N�U���k�K�u ��<
e`P�F��#�f<��mm&���z3.�f����Uedd��9Y�q2�����(}�[���:\��p��u���:\��p��u���:\��p��u���:\��p�����W��0%
#33Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#32)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Hi Ashutosh,

On 2018/05/14 20:14, Ashutosh Bapat wrote:

0001-Hash-partition-bound-equality-refactoring.patch
0002-Targetlist-of-a-child-join-is-produced-by-translatin.patch
0003-Partition-wise-join-for-1-1-1-0-0-1-partition-matchi.patch
0004-Add-a-debug-message-to-notify-whether-partition-wise.patch
0005-Tests-for-0-1-1-1-and-1-0-partition-matching.patch
0006-Extra-extensive-tests-for-advanced-partition-matchin.patch

I noticed after *cleanly* applying 0001-0004 to today's HEAD that while
0005's test all pass, there are many failures in 0006's tests. Maybe, you
need to adjust something in one of the patches or adjust test outputs.

Thanks,
Amit

#34Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Langote (#33)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Tue, Jun 26, 2018 at 2:27 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

Hi Ashutosh,

On 2018/05/14 20:14, Ashutosh Bapat wrote:

0001-Hash-partition-bound-equality-refactoring.patch
0002-Targetlist-of-a-child-join-is-produced-by-translatin.patch
0003-Partition-wise-join-for-1-1-1-0-0-1-partition-matchi.patch
0004-Add-a-debug-message-to-notify-whether-partition-wise.patch
0005-Tests-for-0-1-1-1-and-1-0-partition-matching.patch
0006-Extra-extensive-tests-for-advanced-partition-matchin.patch

I noticed after *cleanly* applying 0001-0004 to today's HEAD that while
0005's test all pass, there are many failures in 0006's tests. Maybe, you
need to adjust something in one of the patches or adjust test outputs.

If the failures are because of plan changes, it's expected. If those
are because of crashes or changed output, those need to be fixed. I
have kept that patch to notice any crashes or output changes, in which
case, I pull that test into 0005 test set after fixing the code. Once
we are near commit, I will remove that patch from the patchset.

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

#35Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#34)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On 2018/06/26 18:02, Ashutosh Bapat wrote:

On Tue, Jun 26, 2018 at 2:27 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

Hi Ashutosh,

On 2018/05/14 20:14, Ashutosh Bapat wrote:

0001-Hash-partition-bound-equality-refactoring.patch
0002-Targetlist-of-a-child-join-is-produced-by-translatin.patch
0003-Partition-wise-join-for-1-1-1-0-0-1-partition-matchi.patch
0004-Add-a-debug-message-to-notify-whether-partition-wise.patch
0005-Tests-for-0-1-1-1-and-1-0-partition-matching.patch
0006-Extra-extensive-tests-for-advanced-partition-matchin.patch

I noticed after *cleanly* applying 0001-0004 to today's HEAD that while
0005's test all pass, there are many failures in 0006's tests. Maybe, you
need to adjust something in one of the patches or adjust test outputs.

If the failures are because of plan changes, it's expected. If those
are because of crashes or changed output, those need to be fixed. I
have kept that patch to notice any crashes or output changes, in which
case, I pull that test into 0005 test set after fixing the code. Once
we are near commit, I will remove that patch from the patchset.

Ah, okay. I thought of reporting this because I felt the errors may have
to do with changes to the related code in HEAD between May 14 when you
last posted the patches and today that you may need to account for in you
patches. For instance, there are many diffs like this:

***************
*** 90,132 ****
-- left outer join, with whole-row reference
EXPLAIN (COSTS OFF)
SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b =
0 ORDER BY t1.a, t1.b, t1.c, t2.a, t2.b, t2.c;
! QUERY PLAN
! --------------------------------------------------------
Sort
Sort Key: t1.a, t1.c, t2.a, t2.b, t2.c
! -> Result
! -> Append
! -> Hash Right Join
! Hash Cond: (t2.b = t1.a)
! -> Seq Scan on prt2_p0 t2
! -> Hash
! -> Seq Scan on prt1_p0 t1
<snip>

--- 90,131 ----
  -- left outer join, with whole-row reference
  EXPLAIN (COSTS OFF)
  SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b =
0 ORDER BY t1.a, t1.b, t1.c, t2.a, t2.b, t2.c;
!                     QUERY PLAN
! --------------------------------------------------
   Sort
     Sort Key: t1.a, t1.c, t2.a, t2.b, t2.c
!    ->  Append
!          ->  Hash Right Join
!                Hash Cond: (t2.b = t1.a)
!                ->  Seq Scan on prt2_p0 t2
!                ->  Hash
!                      ->  Seq Scan on prt1_p0 t1
!                            Filter: (b = 0)

Looks like the Result node on top of Append is no longer there after
applying your patch.

Thanks,
Amit

#36Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Langote (#35)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Wed, Jun 27, 2018 at 12:28 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2018/06/26 18:02, Ashutosh Bapat wrote:

On Tue, Jun 26, 2018 at 2:27 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

Hi Ashutosh,

On 2018/05/14 20:14, Ashutosh Bapat wrote:

0001-Hash-partition-bound-equality-refactoring.patch
0002-Targetlist-of-a-child-join-is-produced-by-translatin.patch
0003-Partition-wise-join-for-1-1-1-0-0-1-partition-matchi.patch
0004-Add-a-debug-message-to-notify-whether-partition-wise.patch
0005-Tests-for-0-1-1-1-and-1-0-partition-matching.patch
0006-Extra-extensive-tests-for-advanced-partition-matchin.patch

I noticed after *cleanly* applying 0001-0004 to today's HEAD that while
0005's test all pass, there are many failures in 0006's tests. Maybe, you
need to adjust something in one of the patches or adjust test outputs.

If the failures are because of plan changes, it's expected. If those
are because of crashes or changed output, those need to be fixed. I
have kept that patch to notice any crashes or output changes, in which
case, I pull that test into 0005 test set after fixing the code. Once
we are near commit, I will remove that patch from the patchset.

Ah, okay. I thought of reporting this because I felt the errors may have
to do with changes to the related code in HEAD between May 14 when you
last posted the patches and today that you may need to account for in you
patches. For instance, there are many diffs like this:

***************
*** 90,132 ****
-- left outer join, with whole-row reference
EXPLAIN (COSTS OFF)
SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b =
0 ORDER BY t1.a, t1.b, t1.c, t2.a, t2.b, t2.c;
! QUERY PLAN
! --------------------------------------------------------
Sort
Sort Key: t1.a, t1.c, t2.a, t2.b, t2.c
! -> Result
! -> Append
! -> Hash Right Join
! Hash Cond: (t2.b = t1.a)
! -> Seq Scan on prt2_p0 t2
! -> Hash
! -> Seq Scan on prt1_p0 t1
<snip>

--- 90,131 ----
-- left outer join, with whole-row reference
EXPLAIN (COSTS OFF)
SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b =
0 ORDER BY t1.a, t1.b, t1.c, t2.a, t2.b, t2.c;
!                     QUERY PLAN
! --------------------------------------------------
Sort
Sort Key: t1.a, t1.c, t2.a, t2.b, t2.c
!    ->  Append
!          ->  Hash Right Join
!                Hash Cond: (t2.b = t1.a)
!                ->  Seq Scan on prt2_p0 t2
!                ->  Hash
!                      ->  Seq Scan on prt1_p0 t1
!                            Filter: (b = 0)

Looks like the Result node on top of Append is no longer there after
applying your patch.

Yes. They are coming because of a commit which removed Result node on
top of an Append node. I don't remember exactly which.

I wouldn't worry about those diffs at this time. As I have mentioned
in earlier mails, the expected output from 0006 is quite large and is
not supposed to be committed. So, I don't see much value in fixing the
plans in that output.

Do you see that as a hindrance in reviewing the code changes and tests in 0005?

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

#37Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#36)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On 2018/06/27 22:21, Ashutosh Bapat wrote:

On Wed, Jun 27, 2018 at 12:28 PM, Amit Langote

Ah, okay. I thought of reporting this because I felt the errors may have
to do with changes to the related code in HEAD between May 14 when you
last posted the patches and today that you may need to account for in you
patches. For instance, there are many diffs like this:

Looks like the Result node on top of Append is no longer there after
applying your patch.

Yes. They are coming because of a commit which removed Result node on
top of an Append node. I don't remember exactly which.

I wouldn't worry about those diffs at this time. As I have mentioned
in earlier mails, the expected output from 0006 is quite large and is
not supposed to be committed. So, I don't see much value in fixing the
plans in that output.

Do you see that as a hindrance in reviewing the code changes and tests in 0005?

I think not. I'll ignore 0006 for now and focus on other patches.

Thanks,
Amit

#38Dmitry Dolgov
9erthalion6@gmail.com
In reply to: Amit Langote (#37)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Thu, 28 Jun 2018 at 07:54, Amit Langote <Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2018/06/27 22:21, Ashutosh Bapat wrote:

On Wed, Jun 27, 2018 at 12:28 PM, Amit Langote

Ah, okay. I thought of reporting this because I felt the errors may have
to do with changes to the related code in HEAD between May 14 when you
last posted the patches and today that you may need to account for in you
patches. For instance, there are many diffs like this:

Looks like the Result node on top of Append is no longer there after
applying your patch.

Yes. They are coming because of a commit which removed Result node on
top of an Append node. I don't remember exactly which.

I wouldn't worry about those diffs at this time. As I have mentioned
in earlier mails, the expected output from 0006 is quite large and is
not supposed to be committed. So, I don't see much value in fixing the
plans in that output.

Do you see that as a hindrance in reviewing the code changes and tests in 0005?

I think not. I'll ignore 0006 for now and focus on other patches.

Hi,

Sorry for my irregular reviews. Unfortunately, the patch need to be rebased
again. In the meantime I have few more commentaries about range_bounds_merge:

* From what I see partition_range_bound_cmp is executed on the same bounds few
times to find whether they are overlapping and during the range merging, is
it necessary? Probably it would be enough just to compare current ranges once
per iteration.

* Just to clarify - the iterating through all the partitions, is it the best
way of finding matching ranges? Correct me if I'm wrong, but from what I see
in the comments for PartitionBoundInfoData, all the ranges are arranged in
increasing order - maybe then it's possible to use binary search for finding
a matching range?

* I'm not sure why in this case partition wise join is not applied? Maybe I'm
wrong, but I was under the impression that it should work here

=# \d+ test1;
Table "public.test1"
Column | Type | Collation | Nullable | Default | Storage |
--------+---------+-----------+----------+---------+----------+
data | jsonb | | | | extended |
id | integer | | | | plain |
Partition key: RANGE (id)
Indexes:
"test1_idx" btree (id)
Partitions: test11 FOR VALUES FROM (0) TO (100),
test12 FOR VALUES FROM (100) TO (200),
test13 FOR VALUES FROM (200) TO (300)

=# \d+ test2;
Table "public.test2"
Column | Type | Collation | Nullable | Default | Storage |
--------+---------+-----------+----------+---------+----------+
data | jsonb | | | | extended |
id | integer | | | | plain |
Partition key: RANGE (id)
Indexes:
"test2_idx" btree (id)
Partitions: test21 FOR VALUES FROM (10) TO (110),
test22 FOR VALUES FROM (110) TO (210),
test23 FOR VALUES FROM (210) TO (310)

=# set enable_partitionwise_join to true;

=# explain analyze select * from test1 t1 inner join test2 t2 using (id);
QUERY PLAN
-------------------------------------------------------------------------------
Hash Join (cost=3.25..6.56 rows=9 width=54)
(actual time=0.082..0.105 rows=3 loops=1)
Hash Cond: (t1.id = t2.id)
-> Append (cost=0.00..3.18 rows=12 width=29)
(actual time=0.026..0.047 rows=12 loops=1)
-> Seq Scan on test11 t1 (cost=0.00..1.05 rows=5 width=29)
(actual time=0.025..0.028 rows=5 loops=1)
-> Seq Scan on test12 t1_1 (cost=0.00..1.04 rows=4 width=29)
(actual time=0.006..0.0 07 rows=4 loops=1)
-> Seq Scan on test13 t1_2 (cost=0.00..1.03 rows=3 width=29)
(actual time=0.005..0.0 06 rows=3 loops=1)
-> Hash (cost=3.13..3.13 rows=9 width=29)
(actual time=0.033..0.033 rows=9 loops=1)
Buckets: 1024 Batches: 1 Memory Usage: 9kB
-> Append (cost=0.00..3.13 rows=9 width=29)
(actual time=0.006..0.022 rows=9 loops=1)
-> Seq Scan on test21 t2
(cost=0.00..1.03 rows=3 width=29)
(actual time=0.005. .0.008 rows=3 loops=1)
-> Seq Scan on test22 t2_1
(cost=0.00..1.03 rows=3 width=29)
(actual time=0.00 4..0.005 rows=3 loops=1)
-> Seq Scan on test23 t2_2
(cost=0.00..1.03 rows=3 width=29)
(actual time=0.00 3..0.004 rows=3 loops=1)
Planning Time: 0.921 ms
Execution Time: 0.261 ms
(14 rows)

=# set enable_partitionwise_join to false;
=# explain analyze select * from test1 t1 inner join test2 t2 using (id);
QUERY PLAN
-------------------------------------------------------------------------------
Hash Join (cost=3.25..6.56 rows=9 width=54)
(actual time=0.073..0.095 rows=3 loops=1)
Hash Cond: (t1.id = t2.id)
-> Append (cost=0.00..3.18 rows=12 width=29)
(actual time=0.022..0.041 rows=12 loops=1)
-> Seq Scan on test11 t1 (cost=0.00..1.05 rows=5 width=29)
(actual time=0.021..0.024 rows=5 loops=1)
-> Seq Scan on test12 t1_1 (cost=0.00..1.04 rows=4 width=29)
(actual time=0.006..0.0 07 rows=4 loops=1)
-> Seq Scan on test13 t1_2 (cost=0.00..1.03 rows=3 width=29)
(actual time=0.004..0.0 05 rows=3 loops=1)
-> Hash (cost=3.13..3.13 rows=9 width=29)
(actual time=0.031..0.031 rows=9 loops=1)
Buckets: 1024 Batches: 1 Memory Usage: 9kB
-> Append (cost=0.00..3.13 rows=9 width=29)
(actual time=0.006..0.021 rows=9 loops=1)
-> Seq Scan on test21 t2
(cost=0.00..1.03 rows=3 width=29)
(actual time=0.005. .0.008 rows=3 loops=1)
-> Seq Scan on test22 t2_1
(cost=0.00..1.03 rows=3 width=29)
(actual time=0.00 3..0.004 rows=3 loops=1)
-> Seq Scan on test23 t2_2
(cost=0.00..1.03 rows=3 width=29)
(actual time=0.00 3..0.004 rows=3 loops=1)
Planning Time: 1.154 ms
Execution Time: 0.201 ms
(14 rows)

My investigation shows that the merge function stops on the second iteration
because of this condition:

/*
* Multiple partitions from one relation map to one partition from the
* other relation. Partition-wise join does not handle this case right
* now, since it requires ganging multiple partitions together (into
* one RelOptInfo).
*/
merged_index = -1;

I'm confused, since there is only one-to-one mapping of partitions.

#39Robert Haas
robertmhaas@gmail.com
In reply to: Dmitry Dolgov (#38)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Sun, Jul 15, 2018 at 1:43 PM, Dmitry Dolgov <9erthalion6@gmail.com> wrote:

Partitions: test11 FOR VALUES FROM (0) TO (100),
test12 FOR VALUES FROM (100) TO (200),
test13 FOR VALUES FROM (200) TO (300)

Partitions: test21 FOR VALUES FROM (10) TO (110),
test22 FOR VALUES FROM (110) TO (210),
test23 FOR VALUES FROM (210) TO (310)

I'm confused, since there is only one-to-one mapping of partitions.

No, test21 would have to be joined to both test11 and test12, since
either could contain matching rows. Also, test22 would have to be
joined to both test12 and test13.

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

#40Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Dmitry Dolgov (#38)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Sun, Jul 15, 2018 at 11:13 PM, Dmitry Dolgov <9erthalion6@gmail.com> wrote:

On Thu, 28 Jun 2018 at 07:54, Amit Langote <Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2018/06/27 22:21, Ashutosh Bapat wrote:

On Wed, Jun 27, 2018 at 12:28 PM, Amit Langote

Ah, okay. I thought of reporting this because I felt the errors may have
to do with changes to the related code in HEAD between May 14 when you
last posted the patches and today that you may need to account for in you
patches. For instance, there are many diffs like this:

Looks like the Result node on top of Append is no longer there after
applying your patch.

Yes. They are coming because of a commit which removed Result node on
top of an Append node. I don't remember exactly which.

I wouldn't worry about those diffs at this time. As I have mentioned
in earlier mails, the expected output from 0006 is quite large and is
not supposed to be committed. So, I don't see much value in fixing the
plans in that output.

Do you see that as a hindrance in reviewing the code changes and tests in 0005?

I think not. I'll ignore 0006 for now and focus on other patches.

Hi,

Sorry for my irregular reviews. Unfortunately, the patch need to be rebased
again.

I rebased the patches without any problem on top of commit
commit f7cb2842bf47715133b40e4a503f35dbe60d1b72
Author: Peter Eisentraut <peter_e@gmx.net>
Date: Mon Jul 16 13:35:41 2018 +0200

Add plan_cache_mode setting

This allows overriding the choice of custom or generic plan.

Author: Pavel Stehule <pavel.stehule@gmail.com>
Discussion:
/messages/by-id/CAFj8pRAGLaiEm8ur5DWEBo7qHRWTk9HxkuUAz00CZZtJj-LkCA@mail.gmail.com

I didn't get any problem rebasing the patches. They compiled cleanly.
The tests added by commit 4513d3a4be0bb7d0141f8b7eaf669a55c08e41b0
failed since this patch-set changes the partitions of the tables used
in the test. Adjusted those tests accordingly.

In the meantime I have few more commentaries about range_bounds_merge:

* From what I see partition_range_bound_cmp is executed on the same bounds few
times to find whether they are overlapping and during the range merging, is
it necessary? Probably it would be enough just to compare current ranges once
per iteration.

The bounds that are compared in those cases are different. Any two
bounds are compared only once per iteration. Remember each range has
two bounds, thus there are total four comparisons possible. In case of
overlap, we do all four comparisons.

* Just to clarify - the iterating through all the partitions, is it the best
way of finding matching ranges? Correct me if I'm wrong, but from what I see
in the comments for PartitionBoundInfoData, all the ranges are arranged in
increasing order - maybe then it's possible to use binary search for finding
a matching range?

The loop in partition_range_bounds_merge() runs as many times as the
maximum of number of datums from the given partition bounds. So the
complexity is O(n) where n is the maximum of number of datums. With
binary search the complexity will increase to O(nlogn). I might be
missing something here.

* I'm not sure why in this case partition wise join is not applied? Maybe I'm
wrong, but I was under the impression that it should work here

=# \d+ test1;
Table "public.test1"
Column | Type | Collation | Nullable | Default | Storage |
--------+---------+-----------+----------+---------+----------+
data | jsonb | | | | extended |
id | integer | | | | plain |
Partition key: RANGE (id)
Indexes:
"test1_idx" btree (id)
Partitions: test11 FOR VALUES FROM (0) TO (100),
test12 FOR VALUES FROM (100) TO (200),
test13 FOR VALUES FROM (200) TO (300)

=# \d+ test2;
Table "public.test2"
Column | Type | Collation | Nullable | Default | Storage |
--------+---------+-----------+----------+---------+----------+
data | jsonb | | | | extended |
id | integer | | | | plain |
Partition key: RANGE (id)
Indexes:
"test2_idx" btree (id)
Partitions: test21 FOR VALUES FROM (10) TO (110),
test22 FOR VALUES FROM (110) TO (210),
test23 FOR VALUES FROM (210) TO (310)

I'm confused, since there is only one-to-one mapping of partitions.

In this case, test21 overlaps test11 (10-100) and test12 (100-110),
test22 overlaps test12 (110-200) and test13(200-210). So, there is no
one-to-one mapping.

PFA rebased patches with tests fixed as described above.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company

Attachments:

pg_adv_dp_join_patches_v10.tar.gzapplication/gzip; name=pg_adv_dp_join_patches_v10.tar.gzDownload
#41Dmitry Dolgov
9erthalion6@gmail.com
In reply to: Ashutosh Bapat (#40)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Tue, 17 Jul 2018 at 11:58, Ashutosh Bapat <ashutosh.bapat@enterprisedb.com> wrote:

On Sun, Jul 15, 2018 at 11:13 PM, Dmitry Dolgov <9erthalion6@gmail.com> wrote:

On Thu, 28 Jun 2018 at 07:54, Amit Langote <Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2018/06/27 22:21, Ashutosh Bapat wrote:

On Wed, Jun 27, 2018 at 12:28 PM, Amit Langote

Ah, okay. I thought of reporting this because I felt the errors may have
to do with changes to the related code in HEAD between May 14 when you
last posted the patches and today that you may need to account for in you
patches. For instance, there are many diffs like this:

Looks like the Result node on top of Append is no longer there after
applying your patch.

Yes. They are coming because of a commit which removed Result node on
top of an Append node. I don't remember exactly which.

I wouldn't worry about those diffs at this time. As I have mentioned
in earlier mails, the expected output from 0006 is quite large and is
not supposed to be committed. So, I don't see much value in fixing the
plans in that output.

Do you see that as a hindrance in reviewing the code changes and tests in 0005?

I think not. I'll ignore 0006 for now and focus on other patches.

Hi,

Sorry for my irregular reviews. Unfortunately, the patch need to be rebased
again.

I rebased the patches without any problem on top of commit

Thanks!

In the meantime I have few more commentaries about range_bounds_merge:

* From what I see partition_range_bound_cmp is executed on the same bounds few
times to find whether they are overlapping and during the range merging, is
it necessary? Probably it would be enough just to compare current ranges once
per iteration.

The bounds that are compared in those cases are different. Any two
bounds are compared only once per iteration. Remember each range has
two bounds, thus there are total four comparisons possible. In case of
overlap, we do all four comparisons.

Yep, indeed, now I see.

* Just to clarify - the iterating through all the partitions, is it the best
way of finding matching ranges? Correct me if I'm wrong, but from what I see
in the comments for PartitionBoundInfoData, all the ranges are arranged in
increasing order - maybe then it's possible to use binary search for finding
a matching range?

The loop in partition_range_bounds_merge() runs as many times as the
maximum of number of datums from the given partition bounds. So the
complexity is O(n) where n is the maximum of number of datums. With
binary search the complexity will increase to O(nlogn). I might be
missing something here.

Now I'm confused even more. Correct me if I'm wrong, but here is what I see
right now:

* We're trying to solve the standard problem of finding overlapping intervals
from two sets of intervals

* The current implementation implicitly compares every range from one side of a
join with every range from another side, which is O(n^2).

Let's imagine we have two tables:

=# \d+ test1_overlap
Table "public.test1_overlap"
Column | Type | Collation | Nullable | Default | Storage |
--------+---------+-----------+----------+---------+----------+
id | integer | | | | plain |
data | jsonb | | | | extended |
Partition key: RANGE (id)
Partitions: test11_overlap FOR VALUES FROM (200) TO (210),
test12_overlap FOR VALUES FROM (220) TO (230),
test13_overlap FOR VALUES FROM (240) TO (250)

=# \d+ test2_overlap
Table "public.test2_overlap"
Column | Type | Collation | Nullable | Default | Storage |
--------+---------+-----------+----------+---------+----------+
id | integer | | | | plain |
data | jsonb | | | | extended |
Partition key: RANGE (id)
Partitions: test210_overlap FOR VALUES FROM (160) TO (170),
test211_overlap FOR VALUES FROM (180) TO (190),
test21_overlap FOR VALUES FROM (0) TO (10),
test22_overlap FOR VALUES FROM (20) TO (30),
test23_overlap FOR VALUES FROM (200) TO (210),
test24_overlap FOR VALUES FROM (40) TO (50),
test25_overlap FOR VALUES FROM (60) TO (70),
test26_overlap FOR VALUES FROM (80) TO (90),
test27_overlap FOR VALUES FROM (100) TO (110),
test28_overlap FOR VALUES FROM (120) TO (130),
test29_overlap FOR VALUES FROM (140) TO (150)

And the join:

select * from test1_overlap inner join test2_overlap using (id);

Partition wise join will work fine, but what will happen (I see this following
the code under gdb) is that inside the function partition_range_bounds_merge we
start from two partitions:

test11_overlap (200, 210) and test21_overlap (0, 10)

In the comparison loop we figure out that there is no overlap and go to the
ub_cmpval > 0 branch, because test11_overlap is higher that test21_overlap.
Inside this branch we think that we need to handle a missing partition case
(apparently, by mistake, but it works - in case if it's not a missing
partition, there are no any records to join with from a default one). Since in
this case there isn't any default partition, the result is merged = true. After
that partition_range_get_next_lb_index will move us to another partition pair:

test11_overlap (200, 210) and test22_overlap (20, 30)

And so on and so forth until we will reach the partition test23_overlap (200,
210), which would be actually what we're looking for. So basically as I said
above we will iterate over the all partitions, and we could skip some of them
using binary search.

* I'm not sure why in this case partition wise join is not applied? Maybe I'm
wrong, but I was under the impression that it should work here

=# \d+ test1;
Table "public.test1"
Column | Type | Collation | Nullable | Default | Storage |
--------+---------+-----------+----------+---------+----------+
data | jsonb | | | | extended |
id | integer | | | | plain |
Partition key: RANGE (id)
Indexes:
"test1_idx" btree (id)
Partitions: test11 FOR VALUES FROM (0) TO (100),
test12 FOR VALUES FROM (100) TO (200),
test13 FOR VALUES FROM (200) TO (300)

=# \d+ test2;
Table "public.test2"
Column | Type | Collation | Nullable | Default | Storage |
--------+---------+-----------+----------+---------+----------+
data | jsonb | | | | extended |
id | integer | | | | plain |
Partition key: RANGE (id)
Indexes:
"test2_idx" btree (id)
Partitions: test21 FOR VALUES FROM (10) TO (110),
test22 FOR VALUES FROM (110) TO (210),
test23 FOR VALUES FROM (210) TO (310)

I'm confused, since there is only one-to-one mapping of partitions.

In this case, test21 overlaps test11 (10-100) and test12 (100-110),
test22 overlaps test12 (110-200) and test13(200-210). So, there is no
one-to-one mapping.

Yep, thanks for the explanation.

#42Dmitry Dolgov
9erthalion6@gmail.com
In reply to: Dmitry Dolgov (#41)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Thu, 19 Jul 2018 at 21:04, Dmitry Dolgov <9erthalion6@gmail.com> wrote:

* Just to clarify - the iterating through all the partitions, is it the best
way of finding matching ranges? Correct me if I'm wrong, but from what I see
in the comments for PartitionBoundInfoData, all the ranges are arranged in
increasing order - maybe then it's possible to use binary search for finding
a matching range?

The loop in partition_range_bounds_merge() runs as many times as the
maximum of number of datums from the given partition bounds. So the
complexity is O(n) where n is the maximum of number of datums. With
binary search the complexity will increase to O(nlogn). I might be
missing something here.

Now I'm confused even more. Correct me if I'm wrong, but here is what I see
right now:

* We're trying to solve the standard problem of finding overlapping intervals
from two sets of intervals

* The current implementation implicitly compares every range from one side of a
join with every range from another side, which is O(n^2).

It's of course wrong, it's going to be O(max(m, n)) as you said, but
the point is still valid - if we have partitions A1, A2 from one side
and B1, ..., BN on another side, we can skip necessary the
partitions from B that are between e.g. A1 and A2 faster with
binary search.

#43Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Dmitry Dolgov (#42)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Fri, Jul 20, 2018 at 3:13 AM, Dmitry Dolgov <9erthalion6@gmail.com> wrote:

It's of course wrong, it's going to be O(max(m, n)) as you said, but
the point is still valid - if we have partitions A1, A2 from one side
and B1, ..., BN on another side, we can skip necessary the
partitions from B that are between e.g. A1 and A2 faster with
binary search.

That's possible only when there is no default partition and the join
is an inner join. For an outer join, we need to include all the
partitions on the outer side, so we can't just skip over some
partitions. In case of a default partition, it can take place of a
missing partition, so we can't skip partitions using binary search.
The code right now works for all the cases and is O(n). I agree that
it can be optimized for certain cases, but
1. those cases are rare enough that we can ignore those right now. How
many times we would encounter the case you have quoted, for example?
Usually the ranges will be continuous only differing in the first or
last partition e.g time-series data.
2. The code is enough complex right now and it's also a lot. Making it
complicated further is not the worth the rare use cases. If we get
complaints from the field, we can certainly improve it in future. But
I would wait for those complaints before improving it further.

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

#44Dmitry Dolgov
9erthalion6@gmail.com
In reply to: Ashutosh Bapat (#43)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Mon, 23 Jul 2018 at 10:38, Ashutosh Bapat <ashutosh.bapat@enterprisedb.com> wrote:

On Fri, Jul 20, 2018 at 3:13 AM, Dmitry Dolgov <9erthalion6@gmail.com> wrote:

It's of course wrong, it's going to be O(max(m, n)) as you said, but
the point is still valid - if we have partitions A1, A2 from one side
and B1, ..., BN on another side, we can skip necessary the
partitions from B that are between e.g. A1 and A2 faster with
binary search.

That's possible only when there is no default partition and the join
is an inner join. For an outer join, we need to include all the
partitions on the outer side, so we can't just skip over some
partitions. In case of a default partition, it can take place of a
missing partition, so we can't skip partitions using binary search.

But in this case I described above (when we have a partition A3 on one side,
and another matching partition B3 from other side, but separated by some other
partitions, e.g. B1, B2) it means that we will merge A3 with a default
partition from other side without actually needing that, am I right? In this
sense it's an overhead out of nothing.

1. those cases are rare enough that we can ignore those right now. How
many times we would encounter the case you have quoted, for example?
Usually the ranges will be continuous only differing in the first or
last partition e.g time-series data.

Well, unfortunately, I don't have enough context to discuss whether it's rare
or not.

#45Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Dmitry Dolgov (#44)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Thu, Jul 26, 2018 at 8:37 PM, Dmitry Dolgov <9erthalion6@gmail.com> wrote:

On Mon, 23 Jul 2018 at 10:38, Ashutosh Bapat <ashutosh.bapat@enterprisedb.com> wrote:

On Fri, Jul 20, 2018 at 3:13 AM, Dmitry Dolgov <9erthalion6@gmail.com> wrote:

It's of course wrong, it's going to be O(max(m, n)) as you said, but
the point is still valid - if we have partitions A1, A2 from one side
and B1, ..., BN on another side, we can skip necessary the
partitions from B that are between e.g. A1 and A2 faster with
binary search.

That's possible only when there is no default partition and the join
is an inner join. For an outer join, we need to include all the
partitions on the outer side, so we can't just skip over some
partitions. In case of a default partition, it can take place of a
missing partition, so we can't skip partitions using binary search.

But in this case I described above (when we have a partition A3 on one side,
and another matching partition B3 from other side, but separated by some other
partitions, e.g. B1, B2) it means that we will merge A3 with a default
partition from other side without actually needing that, am I right? In this
sense it's an overhead out of nothing.

No. We will join A3 with B3 since they have matching bounds. We will
compare B1 and B2's bounds with A3 (assuming that there are no bounds
before A3). Since they won't be compatible we will match default of A
to B1 and B2. That will of-course fail since we will try to match
multiple partitions to a single partition, but that's not the point of
your question I think. You are right that we could skip comparing A3
with B1 and B2 and directly land on B3. Any partitions skipped in
between will be matched with A's default partition. But as I have said
this would be rare and complicating the logic for a rare case doesn't
seem practical at this stage.

Apart from the complexity there's also a possibility that this
skipping will reduce the efficiency actually in normal cases. Consider
a case where A and B have exactly matching partitions. Current
partition matching algorithm compare any given range/bound only once
and we will have O(n) algorithm. If we use a binary search, however,
for every range comparison, it will be O(n log n) algorithm. There
will be unnecessary comparisons during binary search. The current
algorithm is always O(n), whereas binary search would be O(n log(n))
with a possibility of having sub-O(n) complexity in some cases. I
would go for an algorithm which has a consistent time complexity
always and which is efficient in normal cases, rather than worrying
about some cases which are not practical.

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

#46Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#45)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Fri, Jul 27, 2018 at 3:17 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Apart from the complexity there's also a possibility that this
skipping will reduce the efficiency actually in normal cases. Consider
a case where A and B have exactly matching partitions. Current
partition matching algorithm compare any given range/bound only once
and we will have O(n) algorithm. If we use a binary search, however,
for every range comparison, it will be O(n log n) algorithm. There
will be unnecessary comparisons during binary search. The current
algorithm is always O(n), whereas binary search would be O(n log(n))
with a possibility of having sub-O(n) complexity in some cases. I
would go for an algorithm which has a consistent time complexity
always and which is efficient in normal cases, rather than worrying
about some cases which are not practical.

Yeah, I think that's a good point. The normal case here will be that
the partition bounds are equal, or that there are a few extra
partitions on one side that don't exist on the other. We don't want
other cases to be crazily inefficient, but I suspect in practice that
if the partitioning bounds aren't pretty close to matching up exactly,
we're going to end up failing to be able to do a partition-wise join
at all. It's not very likely that somebody happens to have a case
where they've partitioned two tables in randomly different ways, but
then they decide to join them anyway, but then it turns out that the
partition bounds happen to be compatible enough that this algorithm
works.

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

#47Dmitry Dolgov
9erthalion6@gmail.com
In reply to: Robert Haas (#46)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Fri, 27 Jul 2018 at 20:13, Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Jul 27, 2018 at 3:17 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Apart from the complexity there's also a possibility that this
skipping will reduce the efficiency actually in normal cases. Consider
a case where A and B have exactly matching partitions. Current
partition matching algorithm compare any given range/bound only once
and we will have O(n) algorithm. If we use a binary search, however,
for every range comparison, it will be O(n log n) algorithm. There
will be unnecessary comparisons during binary search. The current
algorithm is always O(n), whereas binary search would be O(n log(n))
with a possibility of having sub-O(n) complexity in some cases. I
would go for an algorithm which has a consistent time complexity
always and which is efficient in normal cases, rather than worrying
about some cases which are not practical.

Yeah, I think that's a good point. The normal case here will be that
the partition bounds are equal, or that there are a few extra
partitions on one side that don't exist on the other. We don't want
other cases to be crazily inefficient, but I suspect in practice that
if the partitioning bounds aren't pretty close to matching up exactly,
we're going to end up failing to be able to do a partition-wise join
at all. It's not very likely that somebody happens to have a case
where they've partitioned two tables in randomly different ways, but
then they decide to join them anyway, but then it turns out that the
partition bounds happen to be compatible enough that this algorithm
works.

On Mon, 23 Jul 2018 at 10:38, Ashutosh Bapat <ashutosh.bapat@enterprisedb.com> wrote:
1. those cases are rare enough that we can ignore those right now. How
many times we would encounter the case you have quoted, for example?
Usually the ranges will be continuous only differing in the first or
last partition e.g time-series data.

Ok, but what about list partitioning? I believe the area of application for it
can be more diverse than mostly just for time-series, and in the patch almost
the same approach is used to merge list partitions.

Other than that everything seems fine from functionality point of view, and so
far I couldn't find any situation that produces a wrong plan. Some of the joins
were not that intuitive from the first glance, but at the end everything was
according to the documentation.
Taking this into account I wonder if it's possible somehow to give any hints in
an explain result about why it wasn't possible to apply partition wise join? If
something wasn't clear for me I ended up looking at the value of "merged" flag,
and to figure out actual reason one needs to trace the entire algorithm.

Besides that I checked the performance in some simple cases, no problems on
this side (but I'll also do some checks for more complicated joins).

And I still have some complaints about readability, although I can imagine that
it's just me:

* Many functions carry some unrelated arguments just to pass them through,
which obscures the purpose of a function.

* I want to suggest to introduce a new data structure for partitions mapping
instead of just keeping them in arrays (was it discussed already before?).

* What is the reason that almost everywhere in the patch there is a naming
convention "outer/inner" partition maps, and sometimes (e.g. in
generate_matching_part_pairs) it's "map1/map2"?

#48Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Dmitry Dolgov (#47)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Thu, Aug 23, 2018 at 4:01 PM, Dmitry Dolgov <9erthalion6@gmail.com> wrote:

On Fri, 27 Jul 2018 at 20:13, Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Jul 27, 2018 at 3:17 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Apart from the complexity there's also a possibility that this
skipping will reduce the efficiency actually in normal cases. Consider
a case where A and B have exactly matching partitions. Current
partition matching algorithm compare any given range/bound only once
and we will have O(n) algorithm. If we use a binary search, however,
for every range comparison, it will be O(n log n) algorithm. There
will be unnecessary comparisons during binary search. The current
algorithm is always O(n), whereas binary search would be O(n log(n))
with a possibility of having sub-O(n) complexity in some cases. I
would go for an algorithm which has a consistent time complexity
always and which is efficient in normal cases, rather than worrying
about some cases which are not practical.

Yeah, I think that's a good point. The normal case here will be that
the partition bounds are equal, or that there are a few extra
partitions on one side that don't exist on the other. We don't want
other cases to be crazily inefficient, but I suspect in practice that
if the partitioning bounds aren't pretty close to matching up exactly,
we're going to end up failing to be able to do a partition-wise join
at all. It's not very likely that somebody happens to have a case
where they've partitioned two tables in randomly different ways, but
then they decide to join them anyway, but then it turns out that the
partition bounds happen to be compatible enough that this algorithm
works.

On Mon, 23 Jul 2018 at 10:38, Ashutosh Bapat <ashutosh.bapat@enterprisedb.com> wrote:
1. those cases are rare enough that we can ignore those right now. How
many times we would encounter the case you have quoted, for example?
Usually the ranges will be continuous only differing in the first or
last partition e.g time-series data.

Ok, but what about list partitioning? I believe the area of application for it
can be more diverse than mostly just for time-series, and in the patch almost
the same approach is used to merge list partitions.

Same arguments hold for list partitioning as well. For a list
partitioned table the bounds are ordered by list datums and not
partitions, so it will be rather odd to have large runs of mismatching
datums, the case where binary search fares, from one of side of join.
So, my following argument still holds true

---

I would go for an algorithm which has a consistent time complexity
always and which is efficient in normal cases, rather than worrying
about some cases which are not practical.

---

Other than that everything seems fine from functionality point of view, and so
far I couldn't find any situation that produces a wrong plan. Some of the joins
were not that intuitive from the first glance, but at the end everything was
according to the documentation.

Thanks a lot for your tests and I am glad that you didn't find any failures.

Taking this into account I wonder if it's possible somehow to give any hints in
an explain result about why it wasn't possible to apply partition wise join? If
something wasn't clear for me I ended up looking at the value of "merged" flag,
and to figure out actual reason one needs to trace the entire algorithm.

That's kind of tricky in PostgreSQL. The optimizer doesn't always
report why a particular path was not chosen. Said that we could add
trace logs printing that information, however, the main difficulty is
reporting the relations being joined.See 0004 for example.

Besides that I checked the performance in some simple cases, no problems on
this side (but I'll also do some checks for more complicated joins).

Thanks a lot.

And I still have some complaints about readability, although I can imagine that
it's just me:

* Many functions carry some unrelated arguments just to pass them through,
which obscures the purpose of a function.

Can you please provide some examples?

* I want to suggest to introduce a new data structure for partitions mapping
instead of just keeping them in arrays (was it discussed already before?).

The best I could think of was a structure with just two arrays. So,
instead of encapsulating the arrays within a structure, I thought it
best to pass bare arrays around. If you have any other ideas, please
let me know.

* What is the reason that almost everywhere in the patch there is a naming
convention "outer/inner" partition maps, and sometimes (e.g. in
generate_matching_part_pairs) it's "map1/map2"?

You will find that the optimizer in general uses outer/inner,
rel1/rel2 nomenclature interchangeably. This patch-set just inherits
that. But yes, we should do more to straighten it out.

I won't be working on this actively in the next commitfest. I will be
glad if somebody else wants to take this up. If there's nobody,
probably we should mark this entry as "returned with feedback" in the
next commitfest.

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

#49Dmitry Dolgov
9erthalion6@gmail.com
In reply to: Ashutosh Bapat (#48)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Wed, 29 Aug 2018 at 09:32, Ashutosh Bapat <ashutosh.bapat@enterprisedb.com> wrote:

* Many functions carry some unrelated arguments just to pass them through,
which obscures the purpose of a function.

Can you please provide some examples?

E.g this chain with partsupfunc & collations:

partition_range_bounds_merge
-> partition_range_cmp
-> partition_range_bound_cmp
-> partition_range_merge_next_lb

I believe I already mentioned that before (although I'm not saying that I have
a solution right away, it just bothers me).

* I want to suggest to introduce a new data structure for partitions mapping
instead of just keeping them in arrays (was it discussed already before?).

The best I could think of was a structure with just two arrays. So,
instead of encapsulating the arrays within a structure, I thought it
best to pass bare arrays around. If you have any other ideas, please
let me know.

Well, what I had in mind is something like a structure Mapping with fields from
& to.

* What is the reason that almost everywhere in the patch there is a naming
convention "outer/inner" partition maps, and sometimes (e.g. in
generate_matching_part_pairs) it's "map1/map2"?

You will find that the optimizer in general uses outer/inner,
rel1/rel2 nomenclature interchangeably. This patch-set just inherits
that. But yes, we should do more to straighten it out.

Thanks, good to know.

I won't be working on this actively in the next commitfest. I will be
glad if somebody else wants to take this up. If there's nobody,
probably we should mark this entry as "returned with feedback" in the
next commitfest.

Since I'm more or less familiar with the code and I believe it's an interesting
feature, I can try to take over it for now if you don't mind (but without any
strong commitments to make it perfectly shining for the next CF).

#50Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Dmitry Dolgov (#49)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Thu, Aug 30, 2018 at 2:23 PM, Dmitry Dolgov <9erthalion6@gmail.com> wrote:

I won't be working on this actively in the next commitfest. I will be
glad if somebody else wants to take this up. If there's nobody,
probably we should mark this entry as "returned with feedback" in the
next commitfest.

Since I'm more or less familiar with the code and I believe it's an interesting
feature, I can try to take over it for now if you don't mind (but without any
strong commitments to make it perfectly shining for the next CF).

Please do. Thanks.

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

#51Dmitry Dolgov
9erthalion6@gmail.com
In reply to: Ashutosh Bapat (#50)
4 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Fri, 31 Aug 2018 at 08:23, Ashutosh Bapat <ashutosh.bapat@enterprisedb.com> wrote:

On Thu, Aug 30, 2018 at 2:23 PM, Dmitry Dolgov <9erthalion6@gmail.com> wrote:

I won't be working on this actively in the next commitfest. I will be
glad if somebody else wants to take this up. If there's nobody,
probably we should mark this entry as "returned with feedback" in the
next commitfest.

Since I'm more or less familiar with the code and I believe it's an interesting
feature, I can try to take over it for now if you don't mind (but without any
strong commitments to make it perfectly shining for the next CF).

Please do. Thanks.

I've noticed that the patch is outdated already, so here is the rebased
version. I also removed the last part with the extra tests since it
was something
internal and merged the debug message into the implementation part. Ashutosh,
please let me know if you're not happy with these modifications.

Other than that I haven't changed anything yet, but hope this will come soon.
And this version is more to keep it updated for those people who may be
interested.

Attachments:

0002-Targetlist-of-a-child-join-is-produced-by-translating-v11.patchapplication/octet-stream; name=0002-Targetlist-of-a-child-join-is-produced-by-translating-v11.patchDownload
From 45365cb382d027098636eccfee08c5f6fc831399 Mon Sep 17 00:00:00 2001
From: erthalion <9erthalion6@gmail.com>
Date: Tue, 11 Sep 2018 21:09:44 +0200
Subject: [PATCH 2/4] Targetlist of a child-join

The next patch adds more general partition matching algorithm for
partition-wise join. With that change, the first pair of joining
relations for the parent joinrel may not produce a partition-wise join
because of the restrictions in partition_bounds_merge(), to be added
in the next patch.  Hence the first pair of joining relations, which
is used to build the child join and presented to
build_child_join_rel(), for the child joinrel does not correspond to
the first pair of joining relations for the parent joinrel. The
targetlist built using different pairs have the targetlist entries
arranged in different order. An appendrel expects that all its
children have their targetlists ordered in the same fashion.  Hence
translate the parent's targetlist so that parent and child joinrels
have their targetlists in sync.

Basic partition-wise join commit
f49842d1ee31b976c681322f76025d7732e860f3 modified
build_joinrel_tlist() to build targetlist for child-join. With the
above changes we don't need it anymore.  Similarly, the same commit
added code to set_append_rel_size() to compute attr_needed for a
child-join relation. That change too is not needed with the above
change. This commit reverts those two changes.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/optimizer/util/relnode.c | 20 ++++++++++++++++++++
 1 file changed, 20 insertions(+)

diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 39f5729b91..2f1137c13d 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -802,6 +802,19 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 
 	appinfos = find_appinfos_by_relids(root, joinrel->relids, &nappinfos);
 
+	/*
+	 * The first pair of joining relations for the parent joinrel may not
+	 * produce a partition-wise join because of the restrictions in
+	 * partition_bounds_merge(). Hence the first pair of joining relations,
+	 * which is used to build the child join and presented here, for the child
+	 * joinrel does not correspond to the first pair of joining relations for
+	 * the parent joinrel. The targetlist built using different pairs have the
+	 * targetlist nodes arranged in different order. An appendrel expects that
+	 * all its children have their targetlists ordered in the same fashion.
+	 * Hence translate the parent's targetlist so that parent and child
+	 * joinrels have their targetlists in sync.
+	 */
+	joinrel->reltarget = copy_pathtarget(parent_joinrel->reltarget);
 	/* Set up reltarget struct */
 	build_child_join_reltarget(root, parent_joinrel, joinrel,
 							   nappinfos, appinfos);
@@ -907,6 +920,12 @@ build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 	Relids		relids = joinrel->relids;
 	ListCell   *vars;
 
+	/*
+	 * We only see parent joins. Targetlist of a child-join is computed by
+	 * translating corresponding parent join's targetlist.
+	 */
+	Assert(joinrel->reloptkind == RELOPT_JOINREL);
+
 	foreach(vars, input_rel->reltarget->exprs)
 	{
 		Var		   *var = (Var *) lfirst(vars);
@@ -934,6 +953,7 @@ build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 
 		/* Is it still needed above this joinrel? */
 		ndx = var->varattno - baserel->min_attr;
+
 		if (bms_nonempty_difference(baserel->attr_needed[ndx], relids))
 		{
 			/* Yup, add it to the output */
-- 
2.16.4

0001-Hash-partition-bound-equality-refactoring-v11.patchapplication/octet-stream; name=0001-Hash-partition-bound-equality-refactoring-v11.patchDownload
From e9cccc84246571e57aae1c91b81e4279eb3b4c64 Mon Sep 17 00:00:00 2001
From: erthalion <9erthalion6@gmail.com>
Date: Tue, 11 Sep 2018 21:08:07 +0200
Subject: [PATCH 1/4] Hash partition bound equality refactoring.

We need that to check whether two given hash bounds are equal into a
separate function, so that it can be called from multiple places.  Right
now it's only caller is partition_bounds_equal() but later we will use
it for merging partition bounds.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/partitioning/partbounds.c | 95 ++++++++++++++++++++++-------------
 1 file changed, 61 insertions(+), 34 deletions(-)

diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 4ed9920618..ea533090a4 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -52,6 +52,9 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 						 Expr **keyCol,
 						 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static bool partition_hbounds_equal(PartitionBoundInfo b1,
+									PartitionBoundInfo b2);
+
 
 /*
  * get_qual_from_partbound
@@ -92,6 +95,63 @@ get_qual_from_partbound(Relation rel, Relation parent,
 	return my_qual;
 }
 
+/*
+ * Are two hash partition bound collections logically equal?
+ *
+ * Hash partition bounds store modulus and remainder in datums array which are
+ * always integers irrespective of the number of partition keys and their data
+ * types. Hence we can compare the hash bound collection without any partition
+ * key specific information. Separating this logic in a function which does not
+ * require partition key specific information allows it be called from places
+ * where the partition key specific information is not completely available.
+ */
+static bool
+partition_hbounds_equal(PartitionBoundInfo b1, PartitionBoundInfo b2)
+{
+	int			greatest_modulus = get_hash_partition_greatest_modulus(b1);
+	int			i;
+
+	Assert(b1->strategy == PARTITION_STRATEGY_HASH &&
+		   b2->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * If two hash partitioned tables have different greatest moduli,
+	 * their partition schemes don't match.  For hash partitioned table,
+	 * the greatest modulus is given by the last datum and number of
+	 * partitions is given by ndatums.
+	 */
+	if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
+		return false;
+
+	/*
+	 * We arrange the partitions in the ascending order of their modulus and
+	 * remainders.  Also every modulus is factor of next larger modulus.
+	 * Therefore we can safely store index of a given partition in indexes
+	 * array at remainder of that partition.  Also entries at (remainder + N *
+	 * modulus) positions in indexes array are all same for (modulus,
+	 * remainder) specification for any partition.  Thus datums array from both
+	 * the given bounds are same, if and only if their indexes array will be
+	 * same.  So, it suffices to compare indexes array.
+	 */
+	for (i = 0; i < greatest_modulus; i++)
+		if (b1->indexes[i] != b2->indexes[i])
+			return false;
+
+#ifdef USE_ASSERT_CHECKING
+
+	/*
+	 * Nonetheless make sure that the bounds are indeed same when the indexes
+	 * match.  Hash partition bound stores modulus and remainder at
+	 * b1->datums[i][0] and b1->datums[i][1] position respectively.
+	 */
+	for (i = 0; i < b1->ndatums; i++)
+		Assert((b1->datums[i][0] == b2->datums[i][0] &&
+				b1->datums[i][1] == b2->datums[i][1]));
+#endif
+
+	return true;
+}
+
 /*
  * Are two partition bound collections logically equal?
  *
@@ -120,41 +180,8 @@ partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
 
 	if (b1->strategy == PARTITION_STRATEGY_HASH)
 	{
-		int			greatest_modulus = get_hash_partition_greatest_modulus(b1);
-
-		/*
-		 * If two hash partitioned tables have different greatest moduli,
-		 * their partition schemes don't match.
-		 */
-		if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
+		if (!partition_hbounds_equal(b1, b2))
 			return false;
-
-		/*
-		 * We arrange the partitions in the ascending order of their moduli
-		 * and remainders.  Also every modulus is factor of next larger
-		 * modulus.  Therefore we can safely store index of a given partition
-		 * in indexes array at remainder of that partition.  Also entries at
-		 * (remainder + N * modulus) positions in indexes array are all same
-		 * for (modulus, remainder) specification for any partition.  Thus
-		 * datums array from both the given bounds are same, if and only if
-		 * their indexes array will be same.  So, it suffices to compare
-		 * indexes array.
-		 */
-		for (i = 0; i < greatest_modulus; i++)
-			if (b1->indexes[i] != b2->indexes[i])
-				return false;
-
-#ifdef USE_ASSERT_CHECKING
-
-		/*
-		 * Nonetheless make sure that the bounds are indeed same when the
-		 * indexes match.  Hash partition bound stores modulus and remainder
-		 * at b1->datums[i][0] and b1->datums[i][1] position respectively.
-		 */
-		for (i = 0; i < b1->ndatums; i++)
-			Assert((b1->datums[i][0] == b2->datums[i][0] &&
-					b1->datums[i][1] == b2->datums[i][1]));
-#endif
 	}
 	else
 	{
-- 
2.16.4

0003-Partition-wise-join-for-1-1-1-0-0-1-partition-match-v11.patchapplication/octet-stream; name=0003-Partition-wise-join-for-1-1-1-0-0-1-partition-match-v11.patchDownload
From 317079743a96dc796644ee7d22346764be56baaa Mon Sep 17 00:00:00 2001
From: erthalion <9erthalion6@gmail.com>
Date: Tue, 11 Sep 2018 21:10:46 +0200
Subject: [PATCH 3/4] Partition-wise join for 1:1, 1:0, 0:1 partition matching.

Earlier version of partition-wise join implementation allowed
partition-wise join between two relations with exactly same partition
bounds. This commit allows partition-wise join to be applied under
following conditions

1. the partition bounds of joining relations are such that rows from
given partition on one side join can join with rows from maximum one
partition on the other side i.e. bounds of a given partition on one
side match/overlap with those of maximum one partition on the other
side. If the mapping happens to be m:n where m > 1 or n > 1, we have
to gang multiple partition relations together into a single relation.
This means that we have to add simple relations during join
processing, something which is not supported right now.  ALso, in such
a case, different pairs of joining relations can produce different
partition bounds for the same join relation, which again is not
supported right now.

2. For every partition on outer side that can contribute to the result
of an OUTER join, there exists at least one (taken along with item 1,
it means exactly one)  matching partition on the inner side. To
support partition-wise join when the inner matching partition doesn't
exist, we have to add a dummy base relation corresponding to the
non-existent inner partition. We don't have support to add base
relations during join processing.

3. For hash partitioned tables however, we support partition-wise join
only when the bounds exactly match. For hash partitioning it's unusual
to have missing partitions and hence generic partition matching is not
required.

With advanced partition matching, we won't be able to apply
partition-wise join when there are missing matching partitions. So for
a given N-way join, some sub-joins may not be partitioned, while the
overall N-way join is partitioned. We can not try partition-wise join
when one or both of the joining relations are not partitioned in one
of the pairs of joining relation. Skip partition-wise join for that
pair.

An example is A IJ B LJ C where B has an extra partition compared to A
and C. Join B LJ C requires dummy relation to join the extra partition
from B to a missing partition from C, and hence can not use
partition-wise join right now and hence BC is not partitioned.  The
extra partition in B gets eliminated in A IJ B and thus it can use
partition-wise join and is partitioned. Hence (AB)C can use
partition-wise join but not A(BC).

Not every pair of joining relation (including the one presented to
build_joinrel_partition_info()) for the same joinrel can use
partition-wise join or has both the relations partitioned. Hence we
calculate the partition bounds for the join relation in
try_partition_wise_join() instead of doing it here.

The partition bounds produced for the first pair that can use
partition-wise are saved in the RelOptInfo of the joinrel. Partition
bounds produced for subsequent pairs should match that produced by the
first pair.

Support for default partition in list partitioned tables
========================================================

When a list value is present in one of the joining relations and not
the other, and the other relation has default partition, match (join)
the partition containing that list value with the default partition.
If there are multiple matches, we can not proceed with the
partition-wise join similar to the case of matching non-default
partition. If the default partition happens to be on the outer side of
the join, the resulting join partition acts as a default partition as
it will contain all the values from the default partition. If
the partition containing the list value happens to be on the outer
side of the join, the resulting join partition is associated with the
list value, since no other partition key value from the default
partition makes it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join.  Such a
default partition may get joined to a partition from the inner side,
if inner side has a list value that is not present in the outer side.
But if such a join partner does not exist, we won't be able to use
partition-wise join since there is no (dummy) relation on the inner
side to join to.

Support for matching default partition in range partitioned tables
==================================================================

When a range is present in one of the joining relations and not the
other, and the other relation has default partition, match (join) the
partition corresponding range with the default partition. If two
ranges overlap but their ranges don't match exactly, the
non-overlapping portion from one range may join the previous
(non-overlapping portion near the lower bound, if exists), next
(non-overlapping portion near the upper bound, if exists) ranges from
the other relation or default partition from the other relation. This
causes multiple partitions on the other side to be joined with a
single partition on the first side; a case we don't support. Any range
from one side which doesn't overlap with any range on the other side,
may find its join partner in the default partition and the
corresponding range partition will need to be joined with the default
partition.  If the default partition happens to be on the outer side
of the join, the resulting join partition acts as a default partition
as it will contain all the values from the default partition. If the
non-partition corresponding to the range happens to be on the outer
side of the join, the resulting join partition is associated with that
range, since partition key values from the default partition outside
that range won't make it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join.  Such a
default partition may get joined to a non-default partition from the
inner side, if inner side has a range missing in the outer side.  But
if such a join partner does not exist, we won't be able to use
partition-wise join since there is no (dummy) relation on the inner
side to join to.

Even though partition-wise join paths are created, if there are costs
are higher than non-partition-wise join paths, parition-wise join is
not chosen as the final plan. The debug message in
try_partition_wise_join() helps to know whether a given pair of
joining relations used partition-wise join or not.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/optimizer/path/joinrels.c |   99 +-
 src/backend/optimizer/util/relnode.c  |   33 +-
 src/backend/partitioning/partbounds.c | 1724 +++++++++++++++++++++++++++++++++
 src/include/partitioning/partbounds.h |    6 +
 4 files changed, 1817 insertions(+), 45 deletions(-)

diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index d3d21fed5d..875b39e334 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1312,25 +1312,31 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 					   RelOptInfo *joinrel, SpecialJoinInfo *parent_sjinfo,
 					   List *parent_restrictlist)
 {
-	int			nparts;
 	int			cnt_parts;
+	PartitionScheme part_scheme;
+	PartitionBoundInfo join_boundinfo;
+	List	   *parts1;
+	List	   *parts2;
+	ListCell   *lc1;
+	ListCell   *lc2;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1343,32 +1349,67 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	 */
 	Assert(joinrel->part_scheme == rel1->part_scheme &&
 		   joinrel->part_scheme == rel2->part_scheme);
+	part_scheme = joinrel->part_scheme;
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * Get the list of matching partitions to be joined along with the
+	 * partition bounds of the join relation. Because of the restrictions
+	 * imposed by partition matching algorithm, not every pair of joining
+	 * relations for this join will be able to use partition-wise join. But all
+	 * those pairs which can use partition-wise join will produce the same
+	 * partition bounds for the join relation.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	join_boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											part_scheme->partsupfunc,
+											part_scheme->partcollation,
+											rel1->boundinfo, rel1->nparts,
+											rel2->boundinfo, rel2->nparts,
+											parent_sjinfo->jointype,
+											&parts1, &parts2);
+
+	if (join_boundinfo == NULL)
+		return;
+
+	if (joinrel->boundinfo == NULL)
+	{
+		Assert(joinrel->nparts == 0 && joinrel->part_rels == NULL);
+		joinrel->boundinfo = join_boundinfo;
+		joinrel->nparts = list_length(parts1);
+		Assert(joinrel->nparts == list_length(parts2));
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) *
+									joinrel->nparts);
+	}
+	else
+	{
+		Assert(partition_bounds_equal(part_scheme->partnatts,
+									  part_scheme->parttyplen,
+									  part_scheme->parttypbyval,
+									  join_boundinfo, joinrel->boundinfo));
+		/*
+		 * Every pair of joining relations should result in the same number
+		 * of child-joins.
+		 */
+		Assert(joinrel->nparts == list_length(parts1));
+		Assert(joinrel->nparts == list_length(parts2));
+		Assert(joinrel->part_rels);
+	}
 
-	nparts = joinrel->nparts;
+	elog(DEBUG3, "join between relations %s and %s is considered for partition-wise join.",
+		 bmsToString(rel1->relids), bmsToString(rel2->relids));
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	cnt_parts = 0;
+	forboth(lc1, parts1, lc2, parts2)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
+		int			part1 = lfirst_int(lc1);
+		int			part2 = lfirst_int(lc2);
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1376,6 +1417,10 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		Assert(part1 >= 0 && part2 >= 0);
+		child_rel1 = rel1->part_rels[part1];
+		child_rel2 = rel2->part_rels[part2];
+
 		/* We should never try to join two overlapping sets of rels. */
 		Assert(!bms_overlap(child_rel1->relids, child_rel2->relids));
 		child_joinrelids = bms_union(child_rel1->relids, child_rel2->relids);
@@ -1409,12 +1454,24 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 			joinrel->part_rels[cnt_parts] = child_joinrel;
 		}
 
+		/*
+		 * For every pair of joining relations, the set of matching partitions
+		 * would change. However, the base relation partitions constituting
+		 * the given child should remain same for all the joining pairs. Since
+		 * the order in which children are stored in the array of child-joins,
+		 * depends upon partition bounds of the join, which are same for all
+		 * the joining pairs, every joining pair yields the child-joins in the
+		 * same order.
+		 */
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
 
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
+		cnt_parts++;
 	}
+
+	Assert(cnt_parts == joinrel->nparts);
 }
 
 /*
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 2f1137c13d..4f15cd8e8b 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1627,7 +1627,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1640,24 +1640,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1669,17 +1651,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index ea533090a4..81024dc87f 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -54,6 +54,63 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 static List *get_range_nulltest(PartitionKey key);
 static bool partition_hbounds_equal(PartitionBoundInfo b1,
 									PartitionBoundInfo b2);
+static PartitionBoundInfo partition_range_bounds_merge(int partnatts,
+							 FmgrInfo *supfuncs, Oid *collations,
+							 PartitionBoundInfo boundinfo1, int nparts1,
+							 PartitionBoundInfo boundinfo2, int nparts2,
+							 JoinType jointype, List **parts1, List **parts2);
+static PartitionBoundInfo partition_list_bounds_merge(int partnatts,
+							FmgrInfo *partsupfunc, Oid *collations,
+							PartitionBoundInfo boundinfo1, int nparts1,
+							PartitionBoundInfo boundinfo2, int nparts2,
+							JoinType jointype, List **parts1, List **parts2);
+static PartitionBoundInfo partition_hash_bounds_merge(int partnatts,
+							FmgrInfo *partsupfunc,
+							Oid *partcollation, PartitionBoundInfo left_bi,
+							int left_nparts, PartitionBoundInfo right_bi,
+							int right_nparts, JoinType jointype, List **left_parts,
+							List **right_parts);
+static void generate_matching_part_pairs(int *mergemap1, int npart1,
+							 int *mergemap2, int nparts2, JoinType jointype,
+							 int nparts, List **parts1, List **parts2);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int map_and_merge_partitions(int *partmap1, int *mergemap1, int index1,
+						 int *partmap2, int *mergemap2, int index2,
+						 int *next_index);
+static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *collations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2);
+static bool partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
+						   Oid *collations, PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, int *ub_cmpval,
+						   int *lb_cmpval);
+static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes);
+static bool merge_default_partitions(PartitionBoundInfo outer_bi,
+						 int *outer_pmap,
+						 int *outer_mmap, PartitionBoundInfo inner_bi,
+						 int *inner_pmap, int *inner_mmap, JoinType jointype,
+						 int *next_index, int *default_index);
+static bool merge_null_partitions(PartitionBoundInfo outer_bi, int *outer_pmap,
+					  int *outer_mmap, PartitionBoundInfo inner_bi,
+					  int *inner_pmap, int *inner_mmap, JoinType jointype,
+					  int *next_index, int *null_index, int *default_index);
+static bool handle_missing_partition(int *missing_side_pmap,
+						 int *missing_side_mmap,
+						 int missing_side_default, int *other_pmap,
+						 int *other_mmap, int other_part,
+						 bool missing_side_outer,
+						 bool missing_side_inner,
+						 int *next_index, int *default_index,
+						 int *merged_index);
 
 
 /*
@@ -2332,3 +2389,1670 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * The function produces the partition bounds for a join between two relations
+ * whose partition bounds are given. The function also returns two lists of
+ * partition indexes one for each of the joining relations. Both the lists
+ * contain the same number of elements. The partition indexes at the same
+ * positions in the lists indicate the pair partitions, one from each side, to
+ * be joined and the position itself corresponds to the index of partition
+ * produced by that child-join in the partitioned join.
+ *
+ * The function returns NULL if we can not find the matching pair of
+ * partitions. This happens if 1. multiple partitions on one side match with
+ * one partition on the other side. 2. a given partition on the outer side
+ * doesn't have a matching partition on the inner side. We can not support the
+ * first case since we don't have a way to represent multiple partitions as a
+ * single relation (RelOptInfo) and then perform join using the ganged
+ * relation. We can not support the second case since the missing inner
+ * partition needs to be represented as an empty relation and we don't have a
+ * way to introduce empty relation during join planning after creating paths
+ * for all the base relations.
+ */
+extern PartitionBoundInfo
+partition_bounds_merge(int partnatts, FmgrInfo *partsupfunc,
+					   Oid *partcollation,
+					   PartitionBoundInfo outer_bi, int outer_nparts,
+					   PartitionBoundInfo inner_bi, int inner_nparts,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds;
+	char		strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_bi->strategy != inner_bi->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	strategy = outer_bi->strategy;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partnatts, partsupfunc,
+														partcollation,
+														outer_bi, outer_nparts,
+														inner_bi, inner_nparts,
+														jointype, outer_parts,
+														inner_parts);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(partnatts,
+														 partsupfunc,
+														 partcollation,
+														 outer_bi,
+														 outer_nparts,
+														 inner_bi,
+														 inner_nparts,
+														 jointype,
+														 outer_parts,
+														 inner_parts);
+			break;
+
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = partition_hash_bounds_merge(partnatts,
+														partsupfunc,
+														partcollation,
+														outer_bi, outer_nparts,
+														inner_bi, inner_nparts,
+														jointype, outer_parts,
+														inner_parts);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	Assert((*outer_parts == NIL || *inner_parts != NIL) &&
+		   (*inner_parts == NIL || *outer_parts != NIL));
+
+	return merged_bounds;
+}
+
+/*
+ * partition_get_range_bounds
+ *
+ * Given the index of lower bound in datums array, return lower and upper
+ * bounds and the index of the partition with that lower bound.
+ */
+static int
+partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
+						   PartitionRangeBound *lower,
+						   PartitionRangeBound *upper)
+{
+	int			part_index;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The lower bound should correspond to a valid partition. */
+	part_index = bi->indexes[lb_index + 1];
+	Assert(part_index >= 0);
+
+	lower->kind = bi->kind[lb_index];
+	lower->datums = bi->datums[lb_index];
+	lower->lower = true;
+	upper->kind = bi->kind[lb_index + 1];
+	upper->datums = bi->datums[lb_index + 1];
+	upper->lower = false;
+
+	return part_index;
+}
+
+/*
+ * partition_range_get_next_lb_index
+ *
+ * Given the index of lower bound in datums array return the
+ * index of lower bound of the next partition. When the given index corresponds
+ * to the last partition, return number of datums (ndatums).
+ */
+static int
+partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
+{
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The partition index corresponding to the upper bound should be valid. */
+	Assert(bi->indexes[lb_index + 1] >= 0);
+
+	/*
+	 * If there are no bounds left beyond the upper bound, we have reached the
+	 * last partition.
+	 */
+	if (lb_index + 2 < bi->ndatums)
+	{
+		/*
+		 * If the bound next to the upper bound corresponds to no partition,
+		 * that's the next lower bound of the next partition. Otherwise, the
+		 * current upper bound is the lower bound of the next partition.
+		 */
+		if (bi->indexes[lb_index + 2] < 0)
+			return lb_index + 2;
+		else
+			return lb_index + 1;
+	}
+	else
+		return bi->ndatums;
+}
+
+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *partcollations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2)
+{
+	return partition_rbound_cmp(partnatts, partsupfunc, partcollations,
+								bound1->datums, bound1->kind, bound1->lower,
+								bound2);
+}
+
+/*
+ * partition_range_cmp
+ *
+ * Compare the bounds of two range partitions. Set ub_cmpval <, = or > 0, if the
+ * first partition's upper bound is lower than, equal to or higher than the
+ * second partition's upper bound resp. Similarly set lb_cmpval <, =  or > 0,
+ * if the first partition's lower bound is lower than, equal to or higher than
+ * the second partition's lower bound resp.
+ *
+ * Return true, if the ranges overlap, otherwise return false.
+ */
+static bool
+partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
+					PartitionRangeBound *lower_bound1,
+					PartitionRangeBound *upper_bound1,
+					PartitionRangeBound *lower_bound2,
+					PartitionRangeBound *upper_bound2, int *ub_cmpval,
+					int *lb_cmpval)
+{
+	bool		overlap;
+
+	/*
+	 * Compare upper bound of the first partition with the lower bound of the
+	 * second and vice-versa. If lower bound is higher than the upper bound,
+	 * the partitions are not overlapping. All other cases indicate overlapping
+	 * partitions.
+	 */
+	if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+								  lower_bound1, upper_bound2) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = 1;
+		*lb_cmpval = 1;
+	}
+	else if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+									   lower_bound2, upper_bound1) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = -1;
+		*lb_cmpval = -1;
+	}
+	else
+	{
+		overlap = true;
+		*ub_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, upper_bound1,
+											   upper_bound2);
+		*lb_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, lower_bound1,
+											   lower_bound2);
+	}
+
+	return overlap;
+}
+
+/*
+ * partition_range_merge
+ *
+ * Merge the partition bounds of given two partitions such that the join
+ * between the given two partitions fits merged bounds.
+ *
+ * "merged_upper" will be set to one of the given upper bounds and
+ * "merged_lower" will be set to one of the given lower bounds.
+ */
+static void
+partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
+					  Oid *partcollations, JoinType jointype,
+					  PartitionRangeBound *left_lb,
+					  PartitionRangeBound *left_ub,
+					  PartitionRangeBound *right_lb,
+					  PartitionRangeBound *right_ub,
+					  PartitionRangeBound **merged_lb,
+					  PartitionRangeBound **merged_ub)
+{
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = left_ub;
+			*merged_lb = left_lb;
+			break;
+
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) < 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) > 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		case JOIN_FULL:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) > 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) < 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+
+	return;
+}
+
+/*
+ * Add the lower bound of the next range to the list of bounds, if the lower
+ * bound is higher or equal to the previous upper bound. If successful return
+ * true, otherwise false.
+ */
+static bool
+partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
+							  Oid *partcollations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		prev_ub.datums = llast(*merged_datums);
+		prev_ub.kind = llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  next_lb_datums, next_lb_kind, false,
+									  &prev_ub);
+	}
+
+	/*
+	 * The lower bound is lower than the last upper bound, thus does not fit
+	 * the bounds created so far and hence can not be merged with the existing
+	 * bounds.
+	 */
+	if (cmpval < 0)
+		return false;
+
+	/*
+	 * Add bounds of the new merged partition. If the next lower bound is
+	 * higher than the last upper bound, add new range with index
+	 * corresponding to the lower bound as -1. If the merged lower bound
+	 * is same as the last merged upper bound, the last upper bound will be
+	 * reused as the lower bound of the next range.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, next_lb_datums);
+		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	return true;
+}
+
+/*
+ * handle_missing_partition
+ *
+ * If a range appears in one of the joining relations but not the other, a row
+ * in the corresponding partition will not have any join partner in the other
+ * relation, unless the other relation has a default partition. If a given list
+ * value is present in one joining relation but not the other, the default
+ * partition on the other side may contain that value.
+ *
+ * In both these cases, such an extra partition forms a joining pair with the
+ * default partition, if any,  on the other side.
+ *
+ * If the default partition happens to be on the outer side of the join, the
+ * resultant partition will act as the default partition of the join relation.
+ * Otherwise the resultant partition will be associated with the range.
+ *
+ * When the default partition is not present in the other relation, the rows in
+ * the extra partition will be included in the bounds of the join result, if it
+ * appears on the outer side of the join, since all rows from the outer side
+ * are included in the join result.
+ *
+ * This function handles all these cases.
+ *
+ * missing_side_pmap, missing_side_mmap and missing_side_default are the
+ * partition map, merge map (See partition_range/list_bounds_merge() for
+ * details) and the index of default partition respectively corresponding the
+ * side with missing partition.
+ *
+ * other_pmap, other_mmap and other_part are the partition map, merge map (See
+ * partition_range/list_bounds_merge() for details) and the index of extra
+ * partition respectively corresponding to the side with the extra partition.
+ *
+ * It returns true if the matching succeeds, otherwise returns false.
+ */
+static bool
+handle_missing_partition(int *missing_side_pmap, int *missing_side_mmap,
+						 int missing_side_default, int *other_pmap,
+						 int *other_mmap, int other_part,
+						 bool missing_side_outer,
+						 bool missing_side_inner,
+						 int *next_index, int *default_index,
+						 int *merged_index)
+{
+	bool		missing_has_default = (missing_side_default != -1);
+
+	if (missing_has_default)
+	{
+		*merged_index = map_and_merge_partitions(missing_side_pmap,
+												 missing_side_mmap,
+												 missing_side_default,
+												 other_pmap, other_mmap,
+												 other_part,
+												 next_index);
+		if (*merged_index < 0)
+			return false;
+
+		if (missing_side_outer)
+		{
+			/*
+			 * Default partition on the outer side forms the default
+			 * partition of the join result.
+			 */
+			if (*default_index < 0)
+				*default_index = *merged_index;
+			else if(*default_index != *merged_index)
+			{
+				/*
+				 * Ended up with default partition on the outer side
+				 * being joined with multiple partitions on the inner
+				 * side. We don't support this case.
+				 */
+				return false;
+			}
+
+			/*
+			 * Since the merged partition acts as a default partition, it
+			 * doesn't need a separate index.
+			 */
+			*merged_index = -1;
+		}
+	}
+	else if (missing_side_inner)
+	{
+		/*
+		 * If this partition has already been mapped (say because we
+		 * found an overlapping range earlier), we know where does it
+		 * fit in the join result. Nothing to do in that case. Else
+		 * create a new merged partition.
+		 */
+		if (other_mmap[other_part] < 0)
+		{
+			other_mmap[other_part] = *next_index;
+			*next_index = *next_index + 1;
+			*merged_index = other_mmap[other_part];
+		}
+	}
+
+	return true;
+}
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations, PartitionBoundInfo outer_bi,
+							 int outer_nparts, PartitionBoundInfo inner_bi,
+							 int inner_nparts, JoinType jointype,
+							 List **outer_parts, List **inner_parts)
+{
+	int		   *outer_pmap;
+	int		   *outer_mmap;
+	int		   *inner_pmap;
+	int		   *inner_mmap;
+	int			cnt1;
+	int			cnt2;
+	int			outer_part;
+	int			inner_part;
+	PartitionBoundInfo merged_bounds = NULL;
+	bool		merged = true;
+	int			outer_lb_index;
+	int			inner_lb_index;
+	int			next_index;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	List	   *merged_kinds = NIL;
+	int			inner_default = inner_bi->default_index;
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+
+	/*
+	 * Allocate and initialize partition maps. We maintain four maps, two maps
+	 * for each joining relation. pmap[i] gives the partition from the other
+	 * relation which would join with ith partition of the given relation.
+	 * Partition i from the given relation will join with partition pmap[i]
+	 * from the other relation to produce partition mmap[i] of the join (merged
+	 * partition).
+	 *
+	 * pmap[i] = -1 indicates that ith partition of a given relation does not
+	 * have a matching partition from the other relation.
+	 *
+	 * mmap[i] = -1 indicates that ith partition of a given relation does not
+	 * contribute to the join result. That can happen only when the given
+	 * relation is the inner relation and it doesn't have a matching partition
+	 * from the outer relation, hence pmap[i] should be -1.
+	 *
+	 * In case of an outer join, every partition of the outer join will appear
+	 * in the join result, and thus has mmap[i] set for all i. But it's not
+	 * necessary that every partition on the outer side will have a matching
+	 * partition on the inner side. In such a case, we end up with pmap[i] = -1
+	 * and mmap[i] != -1.
+	 */
+	outer_pmap = (int *) palloc(sizeof(int) * outer_nparts);
+	outer_mmap = (int *) palloc(sizeof(int) * outer_nparts);
+	inner_pmap = (int *) palloc(sizeof(int) * inner_nparts);
+	inner_mmap = (int *) palloc(sizeof(int) * inner_nparts);
+
+	for (cnt1 = 0; cnt1 < outer_nparts; cnt1++)
+	{
+		outer_pmap[cnt1] = -1;
+		outer_mmap[cnt1] = -1;
+	}
+	for (cnt2 = 0; cnt2 < inner_nparts; cnt2++)
+	{
+		inner_pmap[cnt2] = -1;
+		inner_mmap[cnt2] = -1;
+	}
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = 0;
+	inner_lb_index = 0;
+	next_index = 0;
+	while (outer_lb_index < outer_bi->ndatums ||
+		   inner_lb_index < inner_bi->ndatums)
+	{
+		PartitionRangeBound outer_lb;
+		PartitionRangeBound outer_ub;
+		PartitionRangeBound inner_lb;
+		PartitionRangeBound inner_ub;
+		PartitionRangeBound *merged_lb = NULL;
+		PartitionRangeBound *merged_ub = NULL;
+		int			merged_index = -1;
+		bool		overlap;
+		bool		finished_outer = false;
+		bool		finished_inner = false;
+
+		/* Result of bounds comparison per partition_rbound_cmp(). */
+		int			ub_cmpval;	/* Upper bounds comparison result. */
+		int			lb_cmpval;	/* Lower bounds comparison result. */
+
+		/* Get the range bounds of the next pair of partitions. */
+		if (outer_lb_index < outer_bi->ndatums)
+			outer_part = partition_get_range_bounds(outer_bi, outer_lb_index,
+												&outer_lb, &outer_ub);
+		else
+			finished_outer = true;
+
+		if (inner_lb_index < inner_bi->ndatums)
+			inner_part = partition_get_range_bounds(inner_bi, inner_lb_index,
+												&inner_lb, &inner_ub);
+		else
+			finished_inner = true;
+
+		Assert(!finished_outer || !finished_inner);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (finished_outer)
+		{
+			overlap = false;
+			ub_cmpval = 1;
+			lb_cmpval = 1;
+		}
+		else if (finished_inner)
+		{
+			overlap = false;
+			ub_cmpval = -1;
+			lb_cmpval = -1;
+		}
+		else
+			overlap = partition_range_cmp(partnatts, partsupfuncs, partcollations,
+										  &outer_lb, &outer_ub, &inner_lb,
+										  &inner_ub, &ub_cmpval, &lb_cmpval);
+
+		if (overlap)
+		{
+			/*
+			 * The rows from overlapping portion of ranges on both sides may
+			 * join, hence the corresponding pair of partitions form a joining
+			 * pair. Match them and produce the bounds of the joint partition
+			 * and its index by merging the bounds according to the type of
+			 * join.
+			 */
+			partition_range_merge(partnatts, partsupfuncs, partcollations,
+								  jointype, &outer_lb, &outer_ub, &inner_lb,
+								  &inner_ub, &merged_lb, &merged_ub);
+			merged_index = map_and_merge_partitions(outer_pmap, outer_mmap,
+													outer_part, inner_pmap,
+													inner_mmap, inner_part,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				/* Failed to match the partitions. */
+				merged = false;
+				break;
+			}
+
+			/*
+			 * If the ranges overlap but don't exactly match, a row from
+			 * non-overlapping portion of the range from one side of join may
+			 * find its join partner in the previous or next overlapping
+			 * partition or default partition on the other side , if such a
+			 * partition exists. All those cases, if true, will cause one
+			 * partition from that side to match at least two partitions on the
+			 * other side; a case that we do not support now. Previous
+			 * partition has been delt with in the previous iteration of this
+			 * loop, next partition will be delt in the next iteration. We will
+			 * deal with the default partition here.
+			 */
+			if ((lb_cmpval < 0 && inner_has_default) ||
+				/* Non-overlapping range on the lower side of outer range. */
+				(lb_cmpval > 0 && outer_has_default) ||
+				/* Non-overlapping range on the lower side of inner range. */
+				(ub_cmpval < 0 && outer_has_default) ||
+				/* Non-overlapping range on the upper side of inner range. */
+				(ub_cmpval > 0 && inner_has_default))
+				/* Non-overlapping range on the upper side of outer range. */
+			{
+				merged = false;
+				break;
+			}
+		}
+
+		if (ub_cmpval == 0)
+		{
+			/* Upper bounds of both the ranges match. */
+			Assert(overlap);
+
+			/* Move to the next pair of partitions. */
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the inner side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &outer_lb;
+				merged_ub = &outer_ub;
+
+				/*
+				 * For a FULL join, inner relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the inner relation acts as
+				 * INNER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_inner = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_outer = (jointype == JOIN_FULL);
+				merged = handle_missing_partition(inner_pmap, inner_mmap,
+														inner_default, outer_pmap,
+														outer_mmap, outer_part,
+														missing_side_outer,
+														missing_side_inner,
+														&next_index,
+														&default_index,
+														&merged_index);
+
+				if (!merged)
+					break;
+			}
+
+			/* Move to the next partition on the outer side. */
+			Assert(!finished_outer);
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+		}
+		else
+		{
+			Assert(ub_cmpval > 0);
+
+			/* Upper bound of outer range higher than that of the inner. */
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the outer side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &inner_lb;
+				merged_ub = &inner_ub;
+
+				/*
+				 * For a FULL join, outer relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the outer relation acts as
+				 * OUTER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_outer = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_inner = (jointype == JOIN_FULL);
+				merged = handle_missing_partition(outer_pmap, outer_mmap,
+												  outer_default, inner_pmap,
+												  inner_mmap, inner_part,
+												  missing_side_outer,
+												  missing_side_inner,
+												  &next_index,
+												  &default_index,
+												  &merged_index);
+
+				if (!merged)
+					break;
+			}
+
+			/* Move to the next partition on the inner side. */
+			Assert (!finished_inner);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+
+		Assert(merged);
+
+		if (merged_index < 0)
+		{
+			/* We didn't find a new merged partition. */
+			continue;
+		}
+
+		/*
+		 * We have a valid partition index for the next partition of join. The
+		 * partition should have valid range.
+		 */
+		Assert(merged_lb && merged_ub);
+
+		/* Try merging new lower bound with the last upper bound. */
+		merged = partition_range_merge_next_lb(partnatts, partsupfuncs,
+											   partcollations,
+											   merged_lb->datums,
+											   merged_lb->kind, &merged_datums,
+											   &merged_kinds, &merged_indexes);
+		if (merged)
+		{
+			/* Add upper bound with the merged partition index. */
+			merged_datums = lappend(merged_datums, merged_ub->datums);
+			merged_kinds = lappend(merged_kinds, merged_ub->kind);
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+		}
+		else
+			break;
+	}
+
+	if (merged)
+		merged = merge_default_partitions(outer_bi, outer_pmap, outer_mmap,
+										  inner_bi, inner_pmap, inner_mmap,
+										  jointype, &next_index,
+										  &default_index);
+
+	/* Create PartitionBoundInfo for the join result. */
+	if (merged)
+	{
+		/* Use maps to match partition from the joining relations. */
+		generate_matching_part_pairs(outer_mmap, outer_nparts, inner_mmap,
+									 inner_nparts, jointype, next_index,
+									 outer_parts, inner_parts);
+
+		/* Craft a PartitionBoundInfo to return. */
+		if (*outer_parts && *inner_parts)
+		{
+			Assert(list_length(*outer_parts) == list_length(*inner_parts));
+			Assert(list_length(*outer_parts) == next_index);
+			merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+														  merged_datums,
+														  merged_indexes,
+														  merged_kinds,
+														  -1, default_index);
+		}
+	}
+
+	/* Free any memory we used in this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+	pfree(outer_pmap);
+	pfree(inner_pmap);
+	pfree(outer_mmap);
+	pfree(inner_mmap);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(int partnatts, FmgrInfo *partsupfunc,
+							Oid *partcollation, PartitionBoundInfo outer_bi,
+							int outer_nparts, PartitionBoundInfo inner_bi,
+							int inner_nparts, JoinType jointype,
+							List **outer_parts, List **inner_parts)
+{
+	int		   *outer_pmap;	/* outer to inner partition map */
+	int		   *outer_mmap;	/* outer to merged partition map */
+	int		   *inner_pmap;	/* inner to outer partition map */
+	int		   *inner_mmap;	/* inner to merged partition map */
+	int			cnto;
+	int			cnti;
+	bool		merged = true;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			next_index = 0;
+	int			null_index;
+	int			default_index = -1;
+	PartitionBoundInfo merged_bounds = NULL;
+	int		   *outer_indexes = outer_bi->indexes;
+	int		   *inner_indexes = inner_bi->indexes;
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	/*
+	 * Allocate and initialize partition maps. We maintain four maps, two maps
+	 * for each joining relation. pmap[i] gives the partition from the other
+	 * relation which would join with ith partition of the given relation.
+	 * Partition i from the given relation will join with partition pmap[i]
+	 * from the other relation to produce partition mmap[i] of the join (merged
+	 * partition).
+	 *
+	 * pmap[i] = -1 indicates that ith partition of a given relation does not
+	 * have a matching partition from the other relation.
+	 *
+	 * mmap[i] = -1 indicates that ith partition of a given relation does not
+	 * contribute to the join result. That can happen only when the given
+	 * relation is the inner relation and it doesn't have a matching partition
+	 * from the outer relation, hence pmap[i] should be -1.
+	 *
+	 * In case of an outer join, every partition of the outer join will appear
+	 * in the join result, and thus has mmap[i] set for all i. But it's not
+	 * necessary that every partition on the outer side will have a matching
+	 * partition on the inner side. In such a case, we end up with pmap[i] = -1
+	 * and mmap[i] != -1.
+	 */
+	outer_pmap = (int *) palloc(sizeof(int) * outer_nparts);
+	outer_mmap = (int *) palloc(sizeof(int) * outer_nparts);
+	inner_pmap = (int *) palloc(sizeof(int) * inner_nparts);
+	inner_mmap = (int *) palloc(sizeof(int) * inner_nparts);
+
+	for (cnto = 0; cnto < outer_nparts; cnto++)
+	{
+		outer_pmap[cnto] = -1;
+		outer_mmap[cnto] = -1;
+	}
+	for (cnti = 0; cnti < inner_nparts; cnti++)
+	{
+		inner_pmap[cnti] = -1;
+		inner_mmap[cnti] = -1;
+	}
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			o_index;
+		int			i_index;
+		int			cmpval;
+		int			merged_index = -1;
+		Datum	   *merged_datum;
+		bool		finished_inner;
+		bool		finished_outer;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+		{
+			finished_outer = true;
+			odatums = NULL;
+			o_index = -1;
+		}
+		else
+		{
+			finished_outer = false;
+			odatums = outer_bi->datums[cnto];
+			o_index = outer_indexes[cnto];
+		}
+
+		if (cnti >= inner_bi->ndatums)
+		{
+			finished_inner = true;
+			idatums = NULL;
+			i_index = -1;
+		}
+		else
+		{
+			finished_inner = false;
+			idatums = inner_bi->datums[cnti];
+			i_index = inner_indexes[cnti];
+		}
+
+		/* If we exhausted both the sides, we won't enter the loop. */
+		Assert(!finished_inner || !finished_outer);
+
+		if (finished_outer)
+			cmpval = 1;
+		else if (finished_inner)
+			cmpval = -1;
+		else
+		{
+			/* Every list datum should map to a valid partition index. */
+			Assert(o_index >= 0 && i_index >= 0 &&
+				   odatums != NULL && idatums != NULL);
+
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0], idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			merged_index = map_and_merge_partitions(outer_pmap, outer_mmap,
+													o_index, inner_pmap,
+													inner_mmap, i_index,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				merged = false;
+				break;
+			}
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			/* A datum missing from the inner side. */
+			merged_index = -1;
+			merged_datum = odatums;
+
+			/*
+			 * For a FULL join, inner relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the inner relation acts as
+			 * INNER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_inner = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_outer = (jointype == JOIN_FULL);
+			merged = handle_missing_partition(inner_pmap, inner_mmap,
+													inner_default, outer_pmap,
+													outer_mmap, o_index,
+													missing_side_outer,
+													missing_side_inner,
+													&next_index,
+													&default_index,
+													&merged_index);
+
+			if (!merged)
+				break;
+
+			/* Move to the next datum on the outer side. */
+			Assert(!finished_outer);
+			cnto++;
+		}
+		else
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			Assert(cmpval > 0);
+
+			/* A datum missing from the outer side. */
+			merged_index = -1;
+			merged_datum = idatums;
+
+			/*
+			 * For a FULL join, outer relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the outer relation acts as
+			 * OUTER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_outer = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_inner = (jointype == JOIN_FULL);
+			merged = handle_missing_partition(outer_pmap, outer_mmap,
+											  outer_default, inner_pmap,
+											  inner_mmap, i_index,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index);
+
+			if (!merged)
+				break;
+
+			/* Move to the next datum on the right side. */
+			Assert(!finished_inner);
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	if (merged)
+		merged = merge_null_partitions(outer_bi, outer_pmap, outer_mmap,
+									   inner_bi, inner_pmap, inner_mmap,
+									   jointype, &next_index, &null_index,
+									   &default_index);
+
+	if (merged)
+		merged = merge_default_partitions(outer_bi, outer_pmap, outer_mmap,
+										  inner_bi, inner_pmap, inner_mmap,
+										  jointype, &next_index,
+										  &default_index);
+
+	/* If successful build the output structures. */
+	if (merged)
+	{
+
+		/* Use maps to match partition from the joining relations. */
+		generate_matching_part_pairs(outer_mmap, outer_nparts, inner_mmap,
+									 inner_nparts, jointype, next_index,
+									 outer_parts, inner_parts);
+		/* Craft a PartitionBoundInfo to return. */
+		if (*outer_parts && *inner_parts)
+		{
+			Assert(list_length(*outer_parts) == list_length(*inner_parts));
+			Assert(list_length(*outer_parts) == next_index);
+			merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+														  merged_datums,
+														  merged_indexes, NIL,
+														  null_index, default_index);
+		}
+	}
+
+	/* Free up all extra memory before returning from this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	pfree(outer_pmap);
+	pfree(inner_pmap);
+	pfree(outer_mmap);
+	pfree(inner_mmap);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_bounds_merge()'s arm for hash partitioned tables.
+ *
+ * If the given two hash bounds are same, the function returns the first one
+ * without any change, alongwith the lists of matching partitions. Otherwise it
+ * returns NULL.
+ *
+ * We could try merging the bounds when both the bounds have same greatest
+ * modulii. But there seems to be hardly any requirement for the same.
+ */
+static PartitionBoundInfo
+partition_hash_bounds_merge(int partnatts, FmgrInfo *partsupfunc,
+							Oid *partcollation, PartitionBoundInfo outer_bi,
+							int outer_nparts, PartitionBoundInfo inner_bi,
+							int inner_nparts, JoinType jointype,
+							List **outer_parts, List **inner_parts)
+{
+	int			nparts;
+	int			cnt;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * Hash partitioned table does not have explicit NULL accepting partition
+	 * and also does not have a default partition.
+	 */
+	Assert(!partition_bound_has_default(outer_bi) &&
+		   !partition_bound_has_default(inner_bi));
+	Assert(!partition_bound_accepts_nulls(outer_bi) &&
+		   !partition_bound_accepts_nulls(inner_bi));
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+
+	if (outer_nparts != inner_nparts)
+		return NULL;
+	nparts = outer_nparts;
+
+	if (outer_bi->ndatums != inner_bi->ndatums ||
+		!partition_hbounds_equal(outer_bi, inner_bi))
+		return NULL;
+
+	 /*
+	  * Cook up list of matching partitions. Since bounds are exactly same the
+	  * partitions at the same position from both the relations match.
+	  */
+	for (cnt = 0; cnt < nparts; cnt++)
+	{
+		*outer_parts = lappend_int(*outer_parts, cnt);
+		*inner_parts = lappend_int(*inner_parts, cnt);
+	}
+
+	return outer_bi;
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * If the two given partitions (given by index1 and index2 resp.) are already
+ * mapped to each other return the index of corresponding partition in the
+ * merged set of partitions.  If they do not have a merged partition associated
+ * with them, assign a new merged partition index.  If the partitions are
+ * already mapped and their mapped partitions are different from each other,
+ * they can not be merged, so return -1.
+ *
+ * partmap1[i] gives the partition of relation 2 which matches ith partition of
+ * relation 1. Similarly for partmap2.
+ *
+ * mergemap1[i] gives the partition in the merged set to which ith partition of
+ * relation 1 maps to. Similarly for mergemap2.
+ *
+ * index1 and index2 are the indexes of matching partition from respective
+ * relations.
+ *
+ * *next_index is used and incremented when the given partitions require a new
+ * merged partition.
+ */
+static int
+map_and_merge_partitions(int *partmap1, int *mergemap1, int index1,
+						 int *partmap2, int *mergemap2, int index2,
+						 int *next_index)
+{
+	int			merged_index;
+
+	/*
+	 * If both the partitions are not mapped to each other, update the
+	 * maps.
+	 */
+	if (partmap1[index1] < 0 && partmap2[index2] < 0)
+	{
+		partmap1[index1] = index2;
+		partmap2[index2] = index1;
+	}
+
+	/*
+	 * If the given to partitions map to each other, find the corresponding
+	 * merged partition index .
+	 */
+	if (partmap1[index1] == index2 && partmap2[index2] == index1)
+	{
+		/*
+		 * If both the partitions are mapped to the same merged partition, get
+		 * the index of merged partition.
+		 */
+		if (mergemap1[index1] == mergemap2[index2])
+		{
+			merged_index = mergemap1[index1];
+
+			/*
+			 * If the given two partitions do not have a merged partition
+			 * associated with them, allocate a new merged partition.
+			 */
+			if (merged_index < 0)
+			{
+				merged_index = *next_index;
+				*next_index = *next_index + 1;
+				mergemap1[index1] = merged_index;
+				mergemap2[index2] = merged_index;
+			}
+		}
+
+		/*
+		 * If partition from one relation was mapped to a merged partition but
+		 * not the partition from the other relation, map the same merged
+		 * partition to the partition from other relation, since matching
+		 * partitions map to the same merged partition.
+		 */
+		else if (mergemap1[index1] >= 0 && mergemap2[index2] < 0)
+		{
+			mergemap2[index2] = mergemap1[index1];
+			merged_index = mergemap1[index1];
+		}
+		else if (mergemap1[index1] < 0 && mergemap2[index2] >= 0)
+		{
+			mergemap1[index1] = mergemap2[index2];
+			merged_index = mergemap2[index2];
+		}
+		else
+		{
+			Assert(mergemap1[index1] != mergemap2[index2] &&
+				   mergemap1[index1] >= 0 && mergemap2[index2] >= 0);
+
+			/*
+			 * Both the partitions map to different merged partitions. This
+			 * means that multiple partitions from one relation matches to one
+			 * partition from the other relation. Partition-wise join does not
+			 * handle this case right now, since it requires ganging multiple
+			 * partitions together (into one RelOptInfo).
+			 */
+			merged_index = -1;
+		}
+	}
+	else
+	{
+		/*
+		 * Multiple partitions from one relation map to one partition from the
+		 * other relation. Partition-wise join does not handle this case right
+		 * now, since it requires ganging multiple partitions together (into
+		 * one RelOptInfo).
+		 */
+		merged_index = -1;
+	}
+
+	return merged_index;
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * mergemap1 and mergemap2 map each partition from either side of the join to a
+ * merged partition resp. E.g. mergemap1[i] gives the merged partition to which
+ * ith partition of first relation maps and mergemap2[j] gives the merged
+ * partition to which jth partition of second relation maps. If mergemap1[i] =
+ * mergemap2[j], i and j form the matching pair of partitions.
+ *
+ * Given these maps this function produces the list pairs of partitions which
+ * when joined produce the merged partitions in the order of merged partition
+ * indexes.
+ *
+ * nparts1 and nparts2 are the number of partitions of the joining relations
+ * resp.
+ *
+ * nparts is the number of merged partitions.
+ *
+ * If successful, the pairs of partitions are returned as two separate lists,
+ * parts1 and parts2 resp., one for each side. Otherwise, those lists will be
+ * set to NIL.
+ */
+static void
+generate_matching_part_pairs(int *mergemap1, int nparts1, int *mergemap2,
+							 int nparts2, JoinType jointype, int nparts,
+							 List **parts1, List **parts2)
+{
+	bool		merged = true;
+	int		  **matching_parts;
+	int			cnt1;
+	int			cnt2;
+
+	matching_parts = (int **) palloc(sizeof(int *) * 2);
+	matching_parts[0] = (int *) palloc(sizeof(int) * nparts);
+	matching_parts[1] = (int *) palloc(sizeof(int) * nparts);
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt1 = 0; cnt1 < nparts; cnt1++)
+	{
+		matching_parts[0][cnt1] = -1;
+		matching_parts[1][cnt1] = -1;
+	}
+
+	/* Set pairs of matching partitions. */
+	for (cnt1 = 0; cnt1 < nparts1; cnt1++)
+	{
+		if (mergemap1[cnt1] >= 0)
+		{
+			Assert(mergemap1[cnt1] < nparts);
+			matching_parts[0][mergemap1[cnt1]] = cnt1;
+		}
+	}
+	for (cnt2 = 0; cnt2 < nparts2; cnt2++)
+	{
+		if (mergemap2[cnt2] >= 0)
+		{
+			Assert(mergemap2[cnt2] < nparts);
+			matching_parts[1][mergemap2[cnt2]] = cnt2;
+		}
+	}
+
+	/*
+	 * If we have a partition missing on an inner side, we need to add a dummy
+	 * relation which joins with the outer partition. If the inner relation
+	 * happens to be a base relation, it will require adding a dummy child
+	 * base relation during join processing. Right now, we freeze the base
+	 * relation arrays like PlannerInfo::simple_rte_array after planning for
+	 * base relations. Adding a new (dummy) base relation would require some
+	 * changes to that. So, right now, we do not implement partition-wise join
+	 * in such cases.
+	 */
+	for (cnt1 = 0; cnt1 < nparts; cnt1++)
+	{
+		int			part1 = matching_parts[0][cnt1];
+		int			part2 = matching_parts[1][cnt1];
+
+		/* At least one of the partitions should exist. */
+		Assert(part1 >= 0 || part2 >= 0);
+
+		switch (jointype)
+		{
+			case JOIN_INNER:
+			case JOIN_SEMI:
+
+				/*
+				 * An inner or semi join can not return any row when the
+				 * matching partition on either side is missing. We should
+				 * have eliminated all such cases while merging the bounds.
+				 */
+				Assert(part1 >= 0 && part2 >= 0);
+				break;
+
+			case JOIN_LEFT:
+			case JOIN_ANTI:
+				Assert(part1 >= 0);
+				if (part2 < 0)
+					merged = false;
+				break;
+
+			case JOIN_FULL:
+				if (part1 < 0 || part2 < 0)
+					merged = false;
+				break;
+
+			default:
+				elog(ERROR, "unrecognized join type: %d", (int) jointype);
+		}
+
+		if (!merged)
+			break;
+
+		*parts1 = lappend_int(*parts1, part1);
+		*parts2 = lappend_int(*parts2, part2);
+	}
+
+	pfree(matching_parts[0]);
+	pfree(matching_parts[1]);
+	pfree(matching_parts);
+
+	if (!merged)
+	{
+		list_free(*parts1);
+		list_free(*parts2);
+		*parts1 = NIL;
+		*parts2 = NIL;
+	}
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * Merge default partitions from both sides, if any, and assign the default
+ * partition for the join result, if necessary.
+ *
+ * If both the relations have default partitions, try mapping those to each
+ * other. If the mapping succeeds corresponding merged partition will act as
+ * the default partition of the join result.
+ *
+ * If inner side of the join has default but not the outer side, rows in it
+ * won't appear in the join result. So don't create a default partition. If
+ * outer side of the join has default but not the inner side, rows in it will
+ * appear in the join result, so create a default merged partition.
+ */
+static bool
+merge_default_partitions(PartitionBoundInfo outer_bi, int *outer_pmap,
+						 int *outer_mmap, PartitionBoundInfo inner_bi,
+						 int *inner_pmap, int *inner_mmap, JoinType jointype,
+						 int *next_index, int *default_index)
+{
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	bool		merged = true;
+
+	if (!outer_has_default && !inner_has_default)
+		Assert(*default_index < 0);
+	else if (outer_has_default && !inner_has_default)
+	{
+		if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+			jointype == JOIN_ANTI)
+		{
+			if (outer_mmap[outer_default] < 0)
+			{
+				outer_mmap[outer_default] = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = outer_mmap[outer_default];
+			}
+			else
+				Assert(*default_index == outer_mmap[outer_default]);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else if (!outer_has_default && inner_has_default)
+	{
+		if (jointype == JOIN_FULL)
+		{
+			if (inner_mmap[inner_default] < 0)
+			{
+				inner_mmap[inner_default] = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = inner_mmap[inner_default];
+			}
+			else
+				Assert(*default_index == inner_mmap[inner_default]);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		*default_index = map_and_merge_partitions(outer_pmap, outer_mmap,
+												 outer_default, inner_pmap,
+												 inner_mmap, inner_default,
+												 next_index);
+
+		if (*default_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
+
+/*
+ * merge_null_partitions
+ *
+ * Merge NULL partitions, i.e. a partition that can hold NULL values for a list
+ * partitioned table, if any. Find the index of merged partition to which the
+ * NULL values would belong in the join result. If one joining relation has a
+ * NULL partition but not the other, try matching it with the default partition
+ * from the other relation since the default partition may have rows with NULL
+ * partition key. We can eliminate a NULL partition when it appears only on the
+ * inner side of the join and the outer side doesn't have a default partition.
+ *
+ * When the equality operator used for join is strict, two NULL values will not
+ * be considered as equal, and thus a NULL partition can be eliminated for an
+ * inner join. But we don't check the strictness operator here.
+ */
+static bool
+merge_null_partitions(PartitionBoundInfo outer_bi, int *outer_pmap,
+					  int *outer_mmap, PartitionBoundInfo inner_bi,
+					  int *inner_pmap, int *inner_mmap, JoinType jointype,
+					  int *next_index, int *null_index, int *default_index)
+{
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	int			outer_ni = outer_bi->null_index;
+	int			inner_ni = inner_bi->null_index;
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	bool		merged = true;
+
+	if (!outer_has_null && !inner_has_null)
+		*null_index = -1;
+	else if (outer_has_null && !inner_has_null)
+	{
+		int			merged_index;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, inner relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the inner relation acts as
+		 * INNER relation. For INNER and SEMI join OUTER and INNER
+		 * differentiation is immaterial.
+		 */
+		missing_side_inner = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_outer = (jointype == JOIN_FULL);
+
+		merged = handle_missing_partition(inner_pmap, inner_mmap,
+										  inner_default, outer_pmap,
+										  outer_mmap, outer_ni,
+										  missing_side_outer,
+										  missing_side_inner, next_index,
+										  default_index, &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join to which the outer null partition maps
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = outer_mmap[outer_ni];
+	}
+	else if (!outer_has_null && inner_has_null)
+	{
+		int			merged_index;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, outer relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the outer relation acts as OUTER
+		 * relation. For INNER and SEMI join OUTER and INNER differentiation is
+		 * immaterial.
+		 */
+		missing_side_outer = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_inner = (jointype == JOIN_FULL);
+		merged = handle_missing_partition(outer_pmap, outer_mmap,
+										  outer_default, inner_pmap,
+										  inner_mmap, inner_ni,
+										  missing_side_outer,
+										  missing_side_inner,
+										  next_index, default_index,
+										  &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join, to which the outer side null partition maps,
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = inner_mmap[inner_ni];
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		*null_index = map_and_merge_partitions(outer_pmap, outer_mmap,
+											  outer_ni, inner_pmap,
+											  inner_mmap, inner_ni,
+											  next_index);
+		if (*null_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index c7535e32fc..cb063b53d9 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -146,5 +146,11 @@ extern int partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 							  int nvalues, Datum *values, bool *is_equal);
 extern int partition_hash_bsearch(PartitionBoundInfo boundinfo,
 					   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   PartitionBoundInfo outer_bi, int outer_nparts,
+					   PartitionBoundInfo inner_bi, int inner_nparts,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
-- 
2.16.4

0004-Tests-for-0-1-1-1-and-1-0-partition-matching-v11.patchapplication/octet-stream; name=0004-Tests-for-0-1-1-1-and-1-0-partition-matching-v11.patchDownload
From ab1fa6f9b6e6c013f3e71e6b4ae9175763749874 Mon Sep 17 00:00:00 2001
From: erthalion <9erthalion6@gmail.com>
Date: Tue, 11 Sep 2018 21:11:55 +0200
Subject: [PATCH 4/4] Tests for 0:1, 1:1 and 1:0 partition matching

Tests for 0:1, 1:1 and 1:0 partition matching

Rajkumar Raghuvanshi and Ashutosh Bapat.
---
 src/test/regress/expected/partition_join.out | 4131 +++++++++++++++++++++-----
 src/test/regress/sql/partition_join.sql      |  427 ++-
 2 files changed, 3855 insertions(+), 703 deletions(-)

diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 3ba3aaf2d8..5d6c7d2c67 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -8,59 +8,86 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-(4 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+(7 rows)
 
 -- left outer join, with whole-row reference; partitionwise join does not apply
 EXPLAIN (COSTS OFF)
@@ -72,35 +99,50 @@ SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER
    ->  Hash Right Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(22 rows)
 
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-      t1      |      t2      
---------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
- (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
- (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
- (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
- (550,0,0550) | 
-(12 rows)
+       t1       |       t2       
+----------------+----------------
+ (-250,0,-0250) | 
+ (-200,0,-0200) | 
+ (-150,0,-0150) | (0,-150,-0150)
+ (-100,0,-0100) | 
+ (-50,0,-0050)  | 
+ (0,0,0000)     | (0,0,0000)
+ (50,0,0050)    | 
+ (100,0,0100)   | 
+ (150,0,0150)   | (0,150,0150)
+ (200,0,0200)   | 
+ (250,0,0250)   | 
+ (300,0,0300)   | (0,300,0300)
+ (350,0,0350)   | 
+ (400,0,0400)   | 
+ (450,0,0450)   | (0,450,0450)
+ (500,0,0500)   | 
+ (550,0,0550)   | 
+ (600,0,0600)   | (0,600,0600)
+ (650,0,0650)   | 
+ (700,0,0700)   | 
+ (750,0,0750)   | (0,750,0750)
+(21 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
@@ -112,35 +154,53 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
    ->  Append
          ->  Hash Right Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Right Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t2_2.b)
-(20 rows)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+      |       | -225 | -0225
+      |       |  -75 | -0075
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(14 rows)
 
 -- full outer join, with placeholder vars
 EXPLAIN (COSTS OFF)
@@ -148,8 +208,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                             QUERY PLAN                            
 ------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               Filter: (((50) = prt1_p0.a) OR ((75) = prt2_p0.b))
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
@@ -174,7 +242,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               Filter: (((50) = prt1_p4.a) OR ((75) = prt2_p4.b))
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
@@ -211,8 +287,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Hash Left Join
+               Hash Cond: (prt1_p0.a = b)
+               ->  Seq Scan on prt1_p0
+                     Filter: ((a < 450) AND (b = 0))
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
          ->  Hash Left Join
                Hash Cond: (prt1_p1.a = b)
                ->  Seq Scan on prt1_p1
@@ -227,29 +310,42 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                ->  Hash
                      ->  Seq Scan on prt1_p2
                            Filter: ((a < 450) AND (b = 0))
-(17 rows)
+(24 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-(9 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+(14 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
                          QUERY PLAN                         
 ------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = b)
+               Filter: ((prt1_p0.b = 0) OR (a = 0))
+               ->  Seq Scan on prt1_p0
+                     Filter: (a < 450)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = b)
                Filter: ((prt1_p1.b = 0) OR (a = 0))
@@ -274,64 +370,153 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                ->  Hash
                      ->  Result
                            One-Time Filter: false
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt2_p4.b = a)
+               Filter: ((b = 0) OR (prt2_p4.a = 0))
+               ->  Seq Scan on prt2_p4
+                     Filter: (b > 250)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
-(12 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+      |       | 375 | 0375
+      |       | 450 | 0450
+      |       | 525 | 0525
+      |       | 600 | 0600
+      |       | 675 | 0675
+      |       | 750 | 0750
+(20 rows)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Semi Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Semi Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                            Filter: (a = 0)
-         ->  Nested Loop Semi Join
-               Join Filter: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
-               ->  Materialize
-                     ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
-(24 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt1_p3 t2_3
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+                           Filter: (b = 0)
+(37 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
 
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
@@ -342,27 +527,82 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
    ->  Append
          ->  Hash Anti Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Hash Anti Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Hash Anti Join
                Hash Cond: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
-(17 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(27 rows)
 
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
-  sum  |         avg          | sum  |         avg         
--------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+  sum  |         avg          | sum  |        avg         
+-------+----------------------+------+--------------------
+ 95550 | 273.0000000000000000 | 2200 | 6.2857142857142857
 (1 row)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Append
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p0 t1
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+               Index Cond: (a = t1.b)
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.b = t2_1.a)
+         ->  Seq Scan on prt2_p1 t1_1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p1 t2_1
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p2 t1_2
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+               Index Cond: (a = t1_2.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p3 t1_3
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+               Index Cond: (a = t1_3.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p4 t1_4
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+               Index Cond: (a = t1_4.b)
+(27 rows)
+
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+  b   |   c   
+------+-------
+ -225 | -0225
+  -75 | -0075
+   75 | 0075
+  225 | 0225
+  375 | 0375
+  525 | 0525
+  675 | 0675
+(7 rows)
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -374,49 +614,74 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
+                     ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
                            Index Cond: (a = t1.a)
-                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t3
                            Index Cond: (b = t2.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_1
+                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
                            Index Cond: (a = t1_1.a)
-                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3_1
                            Index Cond: (b = t2_1.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_2
+                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
                            Index Cond: (a = t1_2.a)
-                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_2
                            Index Cond: (b = t2_2.a)
-(27 rows)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                           Index Cond: (a = t1_3.a)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_3
+                           Index Cond: (b = t2_3.a)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                           Index Cond: (a = t1_4.a)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t3_4
+                           Index Cond: (b = t2_4.a)
+(43 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   | t2a | t3a | least 
------+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
- 100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
- 250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
- 400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
- 550 | 0 | 0550 |     |     |      
-(12 rows)
+  a   | b |   c   | t2a  | t3a | least 
+------+---+-------+------+-----+-------
+ -250 | 0 | -0250 |      |     |      
+ -200 | 0 | -0200 |      |     |      
+ -150 | 0 | -0150 | -150 |   0 |  -150
+ -100 | 0 | -0100 |      |     |      
+  -50 | 0 | -0050 |      |     |      
+    0 | 0 | 0000  |    0 |   0 |     0
+   50 | 0 | 0050  |      |     |      
+  100 | 0 | 0100  |      |     |      
+  150 | 0 | 0150  |  150 |   0 |   150
+  200 | 0 | 0200  |      |     |      
+  250 | 0 | 0250  |      |     |      
+  300 | 0 | 0300  |  300 |   0 |   300
+  350 | 0 | 0350  |      |     |      
+  400 | 0 | 0400  |      |     |      
+  450 | 0 | 0450  |  450 |   0 |   450
+  500 | 0 | 0500  |      |     |      
+  550 | 0 | 0550  |      |     |      
+  600 | 0 | 0600  |  600 |   0 |   600
+  650 | 0 | 0650  |      |     |      
+  700 | 0 | 0700  |      |     |      
+  750 | 0 | 0750  |  750 |   0 |   750
+(21 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
@@ -430,64 +695,95 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
          Hash Cond: ((t1.c)::text = (t2.c)::text)
          Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Seq Scan on prt1_p1 t1
-               ->  Seq Scan on prt1_p2 t1_1
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
          ->  Hash
                ->  Append
                      ->  Hash Join
                            Hash Cond: (t2.a = t3.b)
-                           ->  Seq Scan on prt1_p1 t2
+                           ->  Seq Scan on prt1_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t3
+                                 ->  Seq Scan on prt2_p0 t3
                      ->  Hash Join
                            Hash Cond: (t2_1.a = t3_1.b)
-                           ->  Seq Scan on prt1_p2 t2_1
+                           ->  Seq Scan on prt1_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t3_1
+                                 ->  Seq Scan on prt2_p1 t3_1
                      ->  Hash Join
                            Hash Cond: (t2_2.a = t3_2.b)
-                           ->  Seq Scan on prt1_p3 t2_2
+                           ->  Seq Scan on prt1_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p3 t3_2
-(26 rows)
+                                 ->  Seq Scan on prt2_p2 t3_2
+                     ->  Hash Join
+                           Hash Cond: (t2_3.a = t3_3.b)
+                           ->  Seq Scan on prt1_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t3_3
+                     ->  Hash Join
+                           Hash Cond: (t2_4.a = t3_4.b)
+                           ->  Seq Scan on prt1_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t3_4
+(38 rows)
 
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, t2.b t2b, t2.c t2c, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
-  a  | t2a | t2c  
------+-----+------
-   0 |   0 | 0000
-  50 |     | 
- 100 |     | 
- 150 | 150 | 0150
- 200 |     | 
- 250 |     | 
- 300 | 300 | 0300
- 350 |     | 
- 400 |     | 
- 450 | 450 | 0450
- 500 |     | 
- 550 |     | 
-(12 rows)
+  a   | t2a  |  t2c  
+------+------+-------
+ -250 |      | 
+ -200 |      | 
+ -150 | -150 | -0150
+ -100 |      | 
+  -50 |      | 
+    0 |    0 | 0000
+   50 |      | 
+  100 |      | 
+  150 |  150 | 0150
+  200 |      | 
+  250 |      | 
+  300 |  300 | 0300
+  350 |      | 
+  400 |      | 
+  450 |  450 | 0450
+  500 |      | 
+  550 |      | 
+  600 |  600 | 0600
+  650 |      | 
+  700 |      | 
+  750 |  750 | 0750
+(21 rows)
 
 --
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
@@ -498,32 +794,49 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
    ->  Append
          ->  Hash Join
                Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
-               ->  Seq Scan on prt2_e_p1 t2
+               ->  Seq Scan on prt2_e_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 t1
+                     ->  Seq Scan on prt1_e_p0 t1
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
-               ->  Seq Scan on prt2_e_p2 t2_1
+               Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+               ->  Seq Scan on prt1_e_p1 t1_1
+                     Filter: (c = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 t1_1
-                           Filter: (c = 0)
+                     ->  Seq Scan on prt2_e_p1 t2_1
          ->  Hash Join
                Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
-               ->  Seq Scan on prt2_e_p3 t2_2
+               ->  Seq Scan on prt2_e_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 t1_2
+                     ->  Seq Scan on prt1_e_p2 t1_2
                            Filter: (c = 0)
-(21 rows)
+         ->  Hash Join
+               Hash Cond: (((t2_3.b + t2_3.a) / 2) = ((t1_3.a + t1_3.b) / 2))
+               ->  Seq Scan on prt2_e_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p3 t1_3
+                           Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (((t2_4.b + t2_4.a) / 2) = ((t1_4.a + t1_4.b) / 2))
+               ->  Seq Scan on prt2_e_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4 t1_4
+                           Filter: (c = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
- 150 | 0 | 150 | 0
- 300 | 0 | 300 | 0
- 450 | 0 | 450 | 0
-(4 rows)
+  a   | c |  b   | c 
+------+---+------+---
+ -250 | 0 | -250 | 0
+ -100 | 0 | -100 | 0
+    0 | 0 |    0 | 0
+    0 | 0 |    0 | 0
+  150 | 0 |  150 | 0
+  300 | 0 |  300 | 0
+  450 | 0 |  450 | 0
+  600 | 0 |  600 | 0
+  750 | 0 |  750 | 0
+(9 rows)
 
 --
 -- N-way join
@@ -536,154 +849,232 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = ((t3.a + t3.b) / 2))
+               Join Filter: (t1.a = t2.b)
                ->  Hash Join
-                     Hash Cond: (t2.b = t1.a)
-                     ->  Seq Scan on prt2_p1 t2
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on prt1_e_p0 t3
                      ->  Hash
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                     Index Cond: (((a + b) / 2) = t2.b)
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
+                     Index Cond: (b = ((t3.a + t3.b) / 2))
          ->  Nested Loop
                Join Filter: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_1.b = t1_1.a)
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
+               ->  Index Scan using iprt1_e_p4_ab2 on prt1_e_p1 t3_1
                      Index Cond: (((a + b) / 2) = t2_1.b)
          ->  Nested Loop
                Join Filter: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_2.b = t1_2.a)
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
+               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_2
                      Index Cond: (((a + b) / 2) = t2_2.b)
-(33 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = ((t3_3.a + t3_3.b) / 2))
+               ->  Nested Loop
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t2_3.b)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t2_4.b)
+               ->  Hash Join
+                     Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                     ->  Seq Scan on prt1_e_p4 t3_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_p4 t1_4
+                                 Filter: (b = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = ((t3_4.a + t3_4.b) / 2))
+(52 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
-(4 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(8 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                          QUERY PLAN                          
---------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Hash Right Join
                Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
-               ->  Seq Scan on prt1_e_p1 t3
+               ->  Seq Scan on prt1_e_p0 t3
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2.b = t1.a)
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_1.a)
-               ->  Seq Scan on prt1_e_p2 t3_1
+               ->  Seq Scan on prt1_e_p1 t3_1
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_1.b = t1_1.a)
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_2.a)
-               ->  Seq Scan on prt1_e_p3 t3_2
+               ->  Seq Scan on prt1_e_p2 t3_2
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_2.b = t1_2.a)
-                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                                        Filter: (b = 0)
-(33 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (t1_3.a = b)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (t1_3.a = ((a + b) / 2))
+         ->  Hash Right Join
+               Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+               ->  Seq Scan on prt1_e_p4 t3_4
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_4.b = t1_4.a)
+                           ->  Seq Scan on prt2_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_p4 t1_4
+                                       Filter: (b = 0)
+(51 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                            QUERY PLAN                             
--------------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p1 t3
+                           ->  Seq Scan on prt1_e_p0 t3
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p1_b on prt2_p1 t2
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
                      Index Cond: (t1.a = b)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p2 t3_1
+                           ->  Seq Scan on prt1_e_p1 t3_1
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
+               ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
                      Index Cond: (t1_1.a = b)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p3 t3_2
+                           ->  Seq Scan on prt1_e_p2 t3_2
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
                      Index Cond: (t1_2.a = b)
-(30 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_e_p3 t3_3
+                           Filter: (c = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                           Index Cond: (a = ((t3_3.a + t3_3.b) / 2))
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (t1_3.a = b)
+         ->  Nested Loop Left Join
+               ->  Hash Right Join
+                     Hash Cond: (t1_4.a = ((t3_4.a + t3_4.b) / 2))
+                     ->  Seq Scan on prt1_p4 t1_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p4 t3_4
+                                 Filter: (c = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (t1_4.a = b)
+(47 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -692,21 +1083,34 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                                                    QUERY PLAN                                                   
 ----------------------------------------------------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   Sort Key: prt1_p0.a, prt2_p0.b, ((prt1_e_p0.a + prt1_e_p0.b))
    ->  Append
          ->  Hash Full Join
-               Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
-               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               Hash Cond: (prt1_p0.a = ((prt1_e_p0.a + prt1_e_p0.b) / 2))
+               Filter: ((prt1_p0.a = (50)) OR (prt2_p0.b = (75)) OR (((prt1_e_p0.a + prt1_e_p0.b) / 2) = (50)))
                ->  Hash Full Join
-                     Hash Cond: (prt1_p1.a = prt2_p1.b)
-                     ->  Seq Scan on prt1_p1
+                     Hash Cond: (prt1_p0.a = prt2_p0.b)
+                     ->  Seq Scan on prt1_p0
                            Filter: (b = 0)
                      ->  Hash
-                           ->  Seq Scan on prt2_p1
+                           ->  Seq Scan on prt2_p0
                                  Filter: (a = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1
+                     ->  Seq Scan on prt1_e_p0
                            Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = prt1_p1.a)
+               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               ->  Seq Scan on prt1_e_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on prt1_p1
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1
+                                       Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
                Filter: ((prt1_p2.a = (50)) OR (prt2_p2.b = (75)) OR (((prt1_e_p2.a + prt1_e_p2.b) / 2) = (50)))
@@ -733,7 +1137,20 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                ->  Hash
                      ->  Seq Scan on prt1_e_p3
                            Filter: (c = 0)
-(42 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = ((prt1_e_p4.a + prt1_e_p4.b) / 2))
+               Filter: ((prt1_p4.a = (50)) OR (prt2_p4.b = (75)) OR (((prt1_e_p4.a + prt1_e_p4.b) / 2) = (50)))
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = prt2_p4.b)
+                     ->  Seq Scan on prt1_p4
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4
+                                 Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4
+                           Filter: (c = 0)
+(68 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
@@ -751,173 +1168,261 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = t1_3.b)
+               Join Filter: (t1.a = t1_5.b)
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Join
-                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
-                           ->  Seq Scan on prt1_e_p1 t2
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t1_3
+                                 ->  Seq Scan on prt2_p0 t1_5
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
                      Index Cond: (a = ((t2.a + t2.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_1.a = t1_4.b)
+               Join Filter: (t1_1.a = t1_6.b)
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Join
-                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
-                           ->  Seq Scan on prt1_e_p2 t2_1
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_6.b)
+                           ->  Seq Scan on prt1_e_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t1_4
+                                 ->  Seq Scan on prt2_p1 t1_6
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
                      Index Cond: (a = ((t2_1.a + t2_1.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_2.a = t1_5.b)
+               Join Filter: (t1_2.a = t1_7.b)
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
+                           ->  Seq Scan on prt2_p2 t1_7
                                  Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
+                           ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t2_2
+                                 Index Cond: (((a + b) / 2) = t1_7.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t1_8.b)
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Nested Loop
+                           ->  Seq Scan on prt2_p3 t1_8
+                                 Filter: (a = 0)
+                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_3
+                                 Index Cond: (((a + b) / 2) = t1_8.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = ((t2_3.a + t2_3.b) / 2))
+                     Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t1_9.b)
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Join
+                           Hash Cond: (((t2_4.a + t2_4.b) / 2) = t1_9.b)
+                           ->  Seq Scan on prt1_e_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t1_9
+                                       Filter: (a = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = ((t2_4.a + t2_4.b) / 2))
+                     Filter: (b = 0)
+(66 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                                  QUERY PLAN                                   
--------------------------------------------------------------------------------
+                                   QUERY PLAN                                    
+---------------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
-                           ->  Seq Scan on prt2_p1 t1_3
+                           Hash Cond: (t1_5.b = ((t1_10.a + t1_10.b) / 2))
+                           ->  Seq Scan on prt2_p0 t1_5
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t1_6
+                                 ->  Seq Scan on prt1_e_p0 t1_10
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
-                     Index Cond: (a = t1_3.b)
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t1_5.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
-                           ->  Seq Scan on prt2_p2 t1_4
+                           Hash Cond: (t1_6.b = ((t1_11.a + t1_11.b) / 2))
+                           ->  Seq Scan on prt2_p1 t1_6
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t1_7
+                                 ->  Seq Scan on prt1_e_p1 t1_11
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
-                     Index Cond: (a = t1_4.b)
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
+                     Index Cond: (a = t1_6.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_7.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_7.b = ((t1_12.a + t1_12.b) / 2))
+                           ->  Seq Scan on prt2_p2 t1_7
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p2 t1_12
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t1_7.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  Unique
                      ->  Sort
-                           Sort Key: t1_5.b
+                           Sort Key: t1_8.b
                            ->  Hash Semi Join
-                                 Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
-                                 ->  Seq Scan on prt2_p3 t1_5
+                                 Hash Cond: (t1_8.b = ((t1_13.a + t1_13.b) / 2))
+                                 ->  Seq Scan on prt2_p3 t1_8
                                  ->  Hash
-                                       ->  Seq Scan on prt1_e_p3 t1_8
+                                       ->  Seq Scan on prt1_e_p3 t1_13
                                              Filter: (c = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t1_5.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t1_8.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_9.b = ((t1_14.a + t1_14.b) / 2))
+                           ->  Seq Scan on prt2_p4 t1_9
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t1_14
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = t1_9.b)
                      Filter: (b = 0)
-(40 rows)
+(64 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 -- test merge joins
 SET enable_hashjoin TO off;
 SET enable_nestloop TO off;
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                           QUERY PLAN                           
-----------------------------------------------------------------
+                            QUERY PLAN                            
+------------------------------------------------------------------
  Merge Append
    Sort Key: t1.a
    ->  Merge Semi Join
-         Merge Cond: (t1.a = t1_3.b)
+         Merge Cond: (t1.a = t1_5.b)
          ->  Sort
                Sort Key: t1.a
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+               Merge Cond: (t1_5.b = (((t1_10.a + t1_10.b) / 2)))
                ->  Sort
-                     Sort Key: t1_3.b
-                     ->  Seq Scan on prt2_p1 t1_3
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on prt2_p0 t1_5
                ->  Sort
-                     Sort Key: (((t1_6.a + t1_6.b) / 2))
-                     ->  Seq Scan on prt1_e_p1 t1_6
+                     Sort Key: (((t1_10.a + t1_10.b) / 2))
+                     ->  Seq Scan on prt1_e_p0 t1_10
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_1.a = t1_4.b)
+         Merge Cond: (t1_1.a = t1_6.b)
          ->  Sort
                Sort Key: t1_1.a
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+               Merge Cond: (t1_6.b = (((t1_11.a + t1_11.b) / 2)))
                ->  Sort
-                     Sort Key: t1_4.b
-                     ->  Seq Scan on prt2_p2 t1_4
+                     Sort Key: t1_6.b
+                     ->  Seq Scan on prt2_p1 t1_6
                ->  Sort
-                     Sort Key: (((t1_7.a + t1_7.b) / 2))
-                     ->  Seq Scan on prt1_e_p2 t1_7
+                     Sort Key: (((t1_11.a + t1_11.b) / 2))
+                     ->  Seq Scan on prt1_e_p1 t1_11
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_2.a = t1_5.b)
+         Merge Cond: (t1_2.a = t1_7.b)
          ->  Sort
                Sort Key: t1_2.a
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+               Merge Cond: (t1_7.b = (((t1_12.a + t1_12.b) / 2)))
                ->  Sort
-                     Sort Key: t1_5.b
-                     ->  Seq Scan on prt2_p3 t1_5
+                     Sort Key: t1_7.b
+                     ->  Seq Scan on prt2_p2 t1_7
                ->  Sort
-                     Sort Key: (((t1_8.a + t1_8.b) / 2))
-                     ->  Seq Scan on prt1_e_p3 t1_8
+                     Sort Key: (((t1_12.a + t1_12.b) / 2))
+                     ->  Seq Scan on prt1_e_p2 t1_12
                            Filter: (c = 0)
-(47 rows)
+   ->  Merge Semi Join
+         Merge Cond: (t1_3.a = t1_8.b)
+         ->  Sort
+               Sort Key: t1_3.a
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_8.b = (((t1_13.a + t1_13.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_8.b
+                     ->  Seq Scan on prt2_p3 t1_8
+               ->  Sort
+                     Sort Key: (((t1_13.a + t1_13.b) / 2))
+                     ->  Seq Scan on prt1_e_p3 t1_13
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_4.a = t1_9.b)
+         ->  Sort
+               Sort Key: t1_4.a
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_9.b = (((t1_14.a + t1_14.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_9.b
+                     ->  Seq Scan on prt2_p4 t1_9
+               ->  Sort
+                     Sort Key: (((t1_14.a + t1_14.b) / 2))
+                     ->  Seq Scan on prt1_e_p4 t1_14
+                           Filter: (c = 0)
+(77 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
@@ -934,14 +1439,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
                            ->  Sort
                                  Sort Key: (((t3.a + t3.b) / 2))
-                                 ->  Seq Scan on prt1_e_p1 t3
+                                 ->  Seq Scan on prt1_e_p0 t3
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1.a
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                ->  Sort
                      Sort Key: t2.b
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Merge Left Join
                Merge Cond: (t1_1.a = t2_1.b)
                ->  Sort
@@ -950,14 +1455,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
                            ->  Sort
                                  Sort Key: (((t3_1.a + t3_1.b) / 2))
-                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                 ->  Seq Scan on prt1_e_p1 t3_1
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_1.a
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                ->  Sort
                      Sort Key: t2_1.b
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Merge Left Join
                Merge Cond: (t1_2.a = t2_2.b)
                ->  Sort
@@ -966,32 +1471,74 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
                            ->  Sort
                                  Sort Key: (((t3_2.a + t3_2.b) / 2))
-                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                 ->  Seq Scan on prt1_e_p2 t3_2
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_2.a
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                ->  Sort
                      Sort Key: t2_2.b
-                     ->  Seq Scan on prt2_p3 t2_2
-(51 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Merge Left Join
+               Merge Cond: (t1_3.a = t2_3.b)
+               ->  Sort
+                     Sort Key: t1_3.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_3.a + t3_3.b) / 2)) = t1_3.a)
+                           ->  Sort
+                                 Sort Key: (((t3_3.a + t3_3.b) / 2))
+                                 ->  Seq Scan on prt1_e_p3 t3_3
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_3.a
+                                 ->  Seq Scan on prt1_p3 t1_3
+               ->  Sort
+                     Sort Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Merge Left Join
+               Merge Cond: (t1_4.a = t2_4.b)
+               ->  Sort
+                     Sort Key: t1_4.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_4.a + t3_4.b) / 2)) = t1_4.a)
+                           ->  Sort
+                                 Sort Key: (((t3_4.a + t3_4.b) / 2))
+                                 ->  Seq Scan on prt1_e_p4 t3_4
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_4.a
+                                 ->  Seq Scan on prt1_p4 t1_4
+               ->  Sort
+                     Sort Key: t2_4.b
+                     ->  Seq Scan on prt2_p4 t2_4
+(83 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- MergeAppend on nullable column
 EXPLAIN (COSTS OFF)
@@ -999,8 +1546,18 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Merge Left Join
+               Merge Cond: (prt1_p0.a = b)
+               ->  Sort
+                     Sort Key: prt1_p0.a
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
+               ->  Sort
+                     Sort Key: b
+                     ->  Result
+                           One-Time Filter: false
          ->  Merge Left Join
                Merge Cond: (prt1_p1.a = b)
                ->  Sort
@@ -1021,21 +1578,26 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                      Sort Key: prt2_p2.b
                      ->  Seq Scan on prt2_p2
                            Filter: (b > 250)
-(23 rows)
+(33 rows)
 
 SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  b  
------+-----
-   0 |    
-  50 |    
- 100 |    
- 150 |    
- 200 |    
- 250 |    
- 300 | 300
- 350 |    
- 400 |    
-(9 rows)
+  a   |  b  
+------+-----
+ -250 |    
+ -200 |    
+ -150 |    
+ -100 |    
+  -50 |    
+    0 |    
+   50 |    
+  100 |    
+  150 |    
+  200 |    
+  250 |    
+  300 | 300
+  350 |    
+  400 |    
+(14 rows)
 
 -- merge join when expression with whole-row reference needs to be sorted;
 -- partitionwise join does not apply
@@ -1049,175 +1611,2412 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
          Sort Key: t1.a, ((((t1.*)::prt1))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
    ->  Sort
          Sort Key: t2.b, ((((t2.*)::prt2))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt2_p1 t2
-                     ->  Seq Scan on prt2_p2 t2_1
-                     ->  Seq Scan on prt2_p3 t2_2
-(16 rows)
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+(20 rows)
 
 SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.b ORDER BY t1.a;
- a  | b  
-----+----
-  0 |  0
-  6 |  6
- 12 | 12
- 18 | 18
- 24 | 24
-(5 rows)
+  a  |  b  
+-----+-----
+ -24 | -24
+ -18 | -18
+ -12 | -12
+  -6 |  -6
+   0 |   0
+   6 |   6
+  12 |  12
+  18 |  18
+  24 |  24
+(9 rows)
 
 RESET enable_hashjoin;
 RESET enable_nestloop;
---
--- partitioned by multiple columns
---
-CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
-CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
-ANALYZE prt1_m;
-CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
-CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
-ANALYZE prt2_m;
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-                                                             QUERY PLAN                                                             
-------------------------------------------------------------------------------------------------------------------------------------
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p4 t2_3
+               ->  Seq Scan on prt2_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_4
+                           Filter: (b = 0)
+(22 rows)
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p3 t1_2
+                           Filter: (b = 0)
+(21 rows)
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -150 | -0150 | 0 | -0150
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+(7 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (t1_3.a = b)
+         ->  Hash Right Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 |   | 
+ -200 | -0200 |   | 
+ -150 | -0150 | 0 | -0150
+ -100 | -0100 |   | 
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+(21 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -200 | 0 | -0200
+ -100 | 0 | -0100
+  -50 | 0 | -0050
+   50 | 0 | 0050
+  100 | 0 | 0100
+  200 | 0 | 0200
+  250 | 0 | 0250
+  350 | 0 | 0350
+  400 | 0 | 0400
+  500 | 0 | 0500
+  550 | 0 | 0550
+  650 | 0 | 0650
+  700 | 0 | 0700
+(14 rows)
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+                             QUERY PLAN                              
+---------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t3.c
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p0_a on prt1_p0 t3
+                           Index Cond: (a = t2.b)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t2.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_1.a = t2_1.b)
+                           ->  Seq Scan on prt1_p1 t3_1
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1 t2_1
+                                       Filter: (a = 0)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p2_a on prt1_p2 t3_2
+                           Index Cond: (a = t2_2.b)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t3_3
+                           Index Cond: (a = t2_3.b)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_4.a = t2_4.b)
+                           ->  Seq Scan on prt1_p4 t3_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t2_4
+                                       Filter: (a = 0)
+(47 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+  a   | a |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.b
+   ->  Hash Right Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p5 t2_5
+                           Filter: (a = 0)
+(24 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: (t1.a = t2.b)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.a, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
+                     Filter: (a = 0)
+(22 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
+               ->  Seq Scan on prt1_m_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p1
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
+               ->  Seq Scan on prt1_m_p2
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p2
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
+               ->  Seq Scan on prt1_m_p3
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p3
+                           Filter: (c = 0)
+(24 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+  a  | c |  b  | c 
+-----+---+-----+---
+   0 | 0 |   0 | 0
+  50 | 0 |     |  
+ 100 | 0 |     |  
+ 150 | 0 | 150 | 0
+ 200 | 0 |     |  
+ 250 | 0 |     |  
+ 300 | 0 | 300 | 0
+ 350 | 0 |     |  
+ 400 | 0 |     |  
+ 450 | 0 | 450 | 0
+ 500 | 0 |     |  
+ 550 | 0 |     |  
+     |   |  75 | 0
+     |   | 225 | 0
+     |   | 375 | 0
+     |   | 525 | 0
+(16 rows)
+
+--
+-- tests for list partitioned tables.
+--
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
+--
+-- list partitioned by expression
+--
+CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
+ANALYZE plt1_e;
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+                                           QUERY PLAN                                           
+------------------------------------------------------------------------------------------------
+ GroupAggregate
+   Group Key: t1.c, t2.c, t3.c
+   ->  Sort
+         Sort Key: t1.c, t3.c
+         ->  Append
+               ->  Hash Join
+                     Hash Cond: ((t1.c)::text = ltrim(t3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2.b = t1.b) AND ((t2.c)::text = (t1.c)::text))
+                           ->  Seq Scan on plt2_p4 t2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p4 t1
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p4 t3
+               ->  Hash Join
+                     Hash Cond: ((t1_1.c)::text = ltrim(t3_1.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t1_1.b = t2_1.b) AND ((t1_1.c)::text = (t2_1.c)::text))
+                           ->  Seq Scan on plt1_p1 t1_1
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p1 t2_1
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p1 t3_1
+               ->  Hash Join
+                     Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t1_2.b = t2_2.b) AND ((t1_2.c)::text = (t2_2.c)::text))
+                           ->  Seq Scan on plt1_p2 t1_2
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p2 t2_2
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p2 t3_2
+               ->  Hash Join
+                     Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2_3.b = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text))
+                           ->  Seq Scan on plt2_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p3 t3_3
+(41 rows)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p5 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_3
+                                 ->  Seq Scan on plt2_p3 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
  Sort
-   Sort Key: prt1_m_p1.a, prt2_m_p1.b
-   ->  Append
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
-               ->  Seq Scan on prt1_m_p1
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p1
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
-               ->  Seq Scan on prt1_m_p2
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p2
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
-               ->  Seq Scan on prt1_m_p3
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p3
-                           Filter: (c = 0)
-(24 rows)
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
 
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
-  50 | 0 |     |  
- 100 | 0 |     |  
- 150 | 0 | 150 | 0
- 200 | 0 |     |  
- 250 | 0 |     |  
- 300 | 0 | 300 | 0
- 350 | 0 |     |  
- 400 | 0 |     |  
- 450 | 0 | 450 | 0
- 500 | 0 |     |  
- 550 | 0 |     |  
-     |   |  75 | 0
-     |   | 225 | 0
-     |   | 375 | 0
-     |   | 525 | 0
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
 (16 rows)
 
---
--- tests for list partitioned tables.
---
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
---
--- list partitioned by expression
---
-CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1_e;
--- test partition matching with N-way join
+-- semi join
 EXPLAIN (COSTS OFF)
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                   QUERY PLAN                                   
---------------------------------------------------------------------------------
- GroupAggregate
-   Group Key: t1.c, t2.c, t3.c
-   ->  Sort
-         Sort Key: t1.c, t3.c
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
          ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.c = ltrim(t3.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1.b = t2.b) AND (t1.c = t2.c))
-                           ->  Seq Scan on plt1_p1 t1
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.c = ltrim(t3_1.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
-                           ->  Seq Scan on plt1_p2 t1_1
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.c = ltrim(t3_2.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
-                           ->  Seq Scan on plt1_p3 t1_2
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p3 t3_2
-(32 rows)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Seq Scan on plt2_p2 t2_2
+                                 ->  Seq Scan on plt2_p3 t2_3
+(22 rows)
 
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  75.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 123.0000000000000000 | 123.0000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 225.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 273.0000000000000000 | 273.0000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 375.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 423.0000000000000000 | 423.0000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 525.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 573.0000000000000000 | 573.0000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1242,22 +4041,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
 --------------------------------------------------
  Hash Left Join
    Hash Cond: (t2.b = a)
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t3.a = t2.b)
-               ->  Seq Scan on prt1_p1 t3
-               ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
-         ->  Hash Join
-               Hash Cond: (t3_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t3_1
-               ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
-         ->  Hash Join
-               Hash Cond: (t3_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t3_2
-               ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
+   ->  Hash Join
+         Hash Cond: (t3.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t3
+               ->  Seq Scan on prt1_p1 t3_1
+               ->  Seq Scan on prt1_p2 t3_2
+               ->  Seq Scan on prt1_p3 t3_3
+               ->  Seq Scan on prt1_p4 t3_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
    ->  Hash
          ->  Result
                One-Time Filter: false
@@ -1272,16 +4071,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 --
 -- tests for hash partitioned tables.
@@ -1357,41 +4162,9 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  273.0000000000000000 | 273.0000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
 (6 rows)
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
- Sort
-   Sort Key: t1.a
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
-               ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
-               ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
-                           Filter: (b = 0)
-(21 rows)
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
@@ -1406,26 +4179,24 @@ SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c
    Sort Key: t1.c
    ->  HashAggregate
          Group Key: t1.c, t2.c
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t2.c = t1.c)
-                     ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p1 t1
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on plt1_p4 t1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_1.c = t1_1.c)
-                     ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_p2 t1_1
+                           ->  Seq Scan on plt1_p1 t1_1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_2.c = t1_2.c)
-                     ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Seq Scan on plt1_p2 t1_2
                                  Filter: ((a % 25) = 0)
-(23 rows)
+                           ->  Seq Scan on plt1_p3 t1_3
+                                 Filter: ((a % 25) = 0)
+(21 rows)
 
 --
 -- multiple levels of partitioning
@@ -1827,64 +4598,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
  Hash Join
    Hash Cond: (t1.a = t2.a)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt4_n_p1 t2
                ->  Seq Scan on prt4_n_p2 t2_1
                ->  Seq Scan on prt4_n_p3 t2_2
-(11 rows)
+(13 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a and t1.a = t3.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                        QUERY PLAN                        
+----------------------------------------------------------
  Hash Join
-   Hash Cond: (t2.a = t1.a)
+   Hash Cond: (t3.b = t1.a)
    ->  Append
-         ->  Seq Scan on prt4_n_p1 t2
-         ->  Seq Scan on prt4_n_p2 t2_1
-         ->  Seq Scan on prt4_n_p3 t2_2
+         ->  Seq Scan on prt2_p0 t3
+         ->  Seq Scan on prt2_p1 t3_1
+         ->  Seq Scan on prt2_p2 t3_2
+         ->  Seq Scan on prt2_p3 t3_3
+         ->  Seq Scan on prt2_p4 t3_4
+         ->  Seq Scan on prt2_p5 t3_5
    ->  Hash
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.a = t3_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_2.b)
-                     ->  Seq Scan on prt1_p3 t1_2
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3 t3_2
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt4_n_p1 t2
+                           ->  Seq Scan on prt4_n_p2 t2_1
+                           ->  Seq Scan on prt4_n_p3 t2_2
 (23 rows)
 
 -- partitionwise join can not be applied if there are no equi-join conditions
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (t1.a < b)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (t1.a < b)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (t1.a < b)
-(12 rows)
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+(16 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partitionwise join
@@ -1970,16 +4747,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
-                     ->  Seq Scan on plt1_p1 t3
-                     ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3
+                     ->  Seq Scan on plt1_p1 t3_1
+                     ->  Seq Scan on plt1_p2 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partitionwise join can not be applied for a join between list and range
 -- partitioned table
@@ -1990,14 +4768,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
  Hash Full Join
    Hash Cond: ((t2.c)::text = (t1.c)::text)
    ->  Append
-         ->  Seq Scan on prt1_p1 t2
-         ->  Seq Scan on prt1_p2 t2_1
-         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p0 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p2 t2_2
+         ->  Seq Scan on prt1_p3 t2_3
+         ->  Seq Scan on prt1_p4 t2_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt1_n_p1 t1
                ->  Seq Scan on prt1_n_p2 t1_1
-(10 rows)
+(12 rows)
 
 -- partitionwise join can not be applied if only one of joining table has
 -- default partition
@@ -2013,16 +4793,23 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
    ->  Hash Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index c1c9859651..0e884835fb 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -10,25 +10,39 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -67,11 +81,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -93,20 +115,30 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 
 EXPLAIN (COSTS OFF)
@@ -169,6 +201,128 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -193,28 +347,79 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
 -- test partition matching with N-way join
@@ -222,6 +427,175 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
@@ -267,27 +641,18 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
 ALTER TABLE plt2 DETACH PARTITION plt2_p3;
 ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
 ANALYZE plt2;
-
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.a % 25 = 0 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
 --
 -- multiple levels of partitioning
 --
-- 
2.16.4

#52Ashutosh Bapat
ashutosh.bapat.oss@gmail.com
In reply to: Dmitry Dolgov (#51)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Thu, Sep 13, 2018 at 1:45 AM Dmitry Dolgov <9erthalion6@gmail.com> wrote:

On Fri, 31 Aug 2018 at 08:23, Ashutosh Bapat <

ashutosh.bapat@enterprisedb.com> wrote:

On Thu, Aug 30, 2018 at 2:23 PM, Dmitry Dolgov <9erthalion6@gmail.com>

wrote:

I won't be working on this actively in the next commitfest. I will be
glad if somebody else wants to take this up. If there's nobody,
probably we should mark this entry as "returned with feedback" in the
next commitfest.

Since I'm more or less familiar with the code and I believe it's an

interesting

feature, I can try to take over it for now if you don't mind (but

without any

strong commitments to make it perfectly shining for the next CF).

Please do. Thanks.

I've noticed that the patch is outdated already, so here is the rebased
version. I also removed the last part with the extra tests since it
was something
internal

I am fine with that. It was never meant to be committed. I used to run
those tests to make sure that any changes to the core logic do not break
any working scenarios. Whenever I found a new failure in the extra tests
which wasn't there in tests to be committed, I used to move that test from
the first to the second. Over the time, the number of new failures in extra
has reduced and recently I didn't see any extra failures. So, may be it's
time for the extra tests to be dropped. I will suggest that keep the extra
tests running from time to time and certainly around the time the feature
gets committed.

and merged the debug message into the implementation part. Ashutosh,
please let me know if you're not happy with these modifications.

Robert Haas raised objections, and I agreed to those, about a similar debug
message I had included in the basic partition-wise join patches. I think
those reasons still apply, so you will need to remove the debug message
before the patches get committed. Said that the debug message is a good
debugging aid, so keeping it around till that time is a good idea.

Other than that I haven't changed anything yet, but hope this will come
soon.
And this version is more to keep it updated for those people who may be
interested.

Thanks.

--
Best Wishes,
Ashutosh Bapat

#53Dmitry Dolgov
9erthalion6@gmail.com
In reply to: Ashutosh Bapat (#52)
4 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Mon, 17 Sep 2018 at 11:20, Ashutosh Bapat <ashutosh.bapat.oss@gmail.com> wrote:

I am fine with that. It was never meant to be committed. I used to run those
tests to make sure that any changes to the core logic do not break any
working scenarios. Whenever I found a new failure in the extra tests which
wasn't there in tests to be committed, I used to move that test from the
first to the second. Over the time, the number of new failures in extra has
reduced and recently I didn't see any extra failures. So, may be it's time
for the extra tests to be dropped. I will suggest that keep the extra tests
running from time to time and certainly around the time the feature gets
committed.

Great, that's exactly what I'm doing right now - I keep these tests locally
to monitor any significant failures except any changes in plans, but without
including it into the patch series.

Since most of my complaints about the patch were related to code readability,
I modified it a bit to show more clearly what I have in mind. I haven't changed
anything about the functionality, just adjusted some parts of it:

* removed some unused arguments (looks like they were added for consistency in
all higher level branches, but not all of them were actually in use)

* replaced representation for partition mapping (instead of int arrays there is
a structure, that allows to replace 0/1 with more meaningful from/to)

* tried to make naming a bit more consistent - so, if a function doesn't
explicitely say anything about outer/inner, we have partmap1/partmap2,
otherwise outermap/innermap. I don't really like this style with
partmap1/partmap2 or index1/index2, but it seems consistent with the rest of
the code in partbounds.c

* removed some state representation flags, e.g. merged - instead, if there is a
situation when we can't merge, functions will return NULL right away

* removed questionable debugging statement

Ashutosh, can you please take a look at it and confirm (or not) that you also
think these changes have improved readability a bit. If we're on the same page
about that, I'll continue in this direction.

Attachments:

0001-Hash-partition-bound-equality-refactoring-v12.patchapplication/octet-stream; name=0001-Hash-partition-bound-equality-refactoring-v12.patchDownload
From 9188bbcb48d523e6df6c3e752f4f3c8b86d751da Mon Sep 17 00:00:00 2001
From: erthalion <9erthalion6@gmail.com>
Date: Tue, 11 Sep 2018 21:08:07 +0200
Subject: [PATCH 1/4] Hash partition bound equality refactoring.

Separate the code to check whether two given hash bounds are equal into a
separate function, so that it can be called from multiple places. Right now
it's only caller is partition_bounds_equal() but later we will use it for
merging partition bounds.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/partitioning/partbounds.c | 95 ++++++++++++++++++++++-------------
 1 file changed, 61 insertions(+), 34 deletions(-)

diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 4ed9920618..ea533090a4 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -52,6 +52,9 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 						 Expr **keyCol,
 						 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static bool partition_hbounds_equal(PartitionBoundInfo b1,
+									PartitionBoundInfo b2);
+
 
 /*
  * get_qual_from_partbound
@@ -92,6 +95,63 @@ get_qual_from_partbound(Relation rel, Relation parent,
 	return my_qual;
 }
 
+/*
+ * Are two hash partition bound collections logically equal?
+ *
+ * Hash partition bounds store modulus and remainder in datums array which are
+ * always integers irrespective of the number of partition keys and their data
+ * types. Hence we can compare the hash bound collection without any partition
+ * key specific information. Separating this logic in a function which does not
+ * require partition key specific information allows it be called from places
+ * where the partition key specific information is not completely available.
+ */
+static bool
+partition_hbounds_equal(PartitionBoundInfo b1, PartitionBoundInfo b2)
+{
+	int			greatest_modulus = get_hash_partition_greatest_modulus(b1);
+	int			i;
+
+	Assert(b1->strategy == PARTITION_STRATEGY_HASH &&
+		   b2->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * If two hash partitioned tables have different greatest moduli,
+	 * their partition schemes don't match.  For hash partitioned table,
+	 * the greatest modulus is given by the last datum and number of
+	 * partitions is given by ndatums.
+	 */
+	if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
+		return false;
+
+	/*
+	 * We arrange the partitions in the ascending order of their modulus and
+	 * remainders.  Also every modulus is factor of next larger modulus.
+	 * Therefore we can safely store index of a given partition in indexes
+	 * array at remainder of that partition.  Also entries at (remainder + N *
+	 * modulus) positions in indexes array are all same for (modulus,
+	 * remainder) specification for any partition.  Thus datums array from both
+	 * the given bounds are same, if and only if their indexes array will be
+	 * same.  So, it suffices to compare indexes array.
+	 */
+	for (i = 0; i < greatest_modulus; i++)
+		if (b1->indexes[i] != b2->indexes[i])
+			return false;
+
+#ifdef USE_ASSERT_CHECKING
+
+	/*
+	 * Nonetheless make sure that the bounds are indeed same when the indexes
+	 * match.  Hash partition bound stores modulus and remainder at
+	 * b1->datums[i][0] and b1->datums[i][1] position respectively.
+	 */
+	for (i = 0; i < b1->ndatums; i++)
+		Assert((b1->datums[i][0] == b2->datums[i][0] &&
+				b1->datums[i][1] == b2->datums[i][1]));
+#endif
+
+	return true;
+}
+
 /*
  * Are two partition bound collections logically equal?
  *
@@ -120,41 +180,8 @@ partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
 
 	if (b1->strategy == PARTITION_STRATEGY_HASH)
 	{
-		int			greatest_modulus = get_hash_partition_greatest_modulus(b1);
-
-		/*
-		 * If two hash partitioned tables have different greatest moduli,
-		 * their partition schemes don't match.
-		 */
-		if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
+		if (!partition_hbounds_equal(b1, b2))
 			return false;
-
-		/*
-		 * We arrange the partitions in the ascending order of their moduli
-		 * and remainders.  Also every modulus is factor of next larger
-		 * modulus.  Therefore we can safely store index of a given partition
-		 * in indexes array at remainder of that partition.  Also entries at
-		 * (remainder + N * modulus) positions in indexes array are all same
-		 * for (modulus, remainder) specification for any partition.  Thus
-		 * datums array from both the given bounds are same, if and only if
-		 * their indexes array will be same.  So, it suffices to compare
-		 * indexes array.
-		 */
-		for (i = 0; i < greatest_modulus; i++)
-			if (b1->indexes[i] != b2->indexes[i])
-				return false;
-
-#ifdef USE_ASSERT_CHECKING
-
-		/*
-		 * Nonetheless make sure that the bounds are indeed same when the
-		 * indexes match.  Hash partition bound stores modulus and remainder
-		 * at b1->datums[i][0] and b1->datums[i][1] position respectively.
-		 */
-		for (i = 0; i < b1->ndatums; i++)
-			Assert((b1->datums[i][0] == b2->datums[i][0] &&
-					b1->datums[i][1] == b2->datums[i][1]));
-#endif
 	}
 	else
 	{
-- 
2.16.4

0002-Targetlist-of-a-child-join-is-produced-by-translating-v12.patchapplication/octet-stream; name=0002-Targetlist-of-a-child-join-is-produced-by-translating-v12.patchDownload
From 63a22ab73ff205702f8aaaaf0dd3f32b8dcc6a2f Mon Sep 17 00:00:00 2001
From: erthalion <9erthalion6@gmail.com>
Date: Tue, 11 Sep 2018 21:09:44 +0200
Subject: [PATCH 2/4] Targetlist of a child-join is produced by translating
 that of parent join

The next patch adds more general partition matching algorithm for
partition-wise join. With that change, the first pair of joining
relations for the parent joinrel may not produce a partition-wise join
because of the restrictions in partition_bounds_merge(), to be added
in the next patch.  Hence the first pair of joining relations, which
is used to build the child join and presented to
build_child_join_rel(), for the child joinrel does not correspond to
the first pair of joining relations for the parent joinrel. The
targetlist built using different pairs have the targetlist entries
arranged in different order. An appendrel expects that all its
children have their targetlists ordered in the same fashion.  Hence
translate the parent's targetlist so that parent and child joinrels
have their targetlists in sync.

Basic partition-wise join commit
f49842d1ee31b976c681322f76025d7732e860f3 modified
build_joinrel_tlist() to build targetlist for child-join. With the
above changes we don't need it anymore.  Similarly, the same commit
added code to set_append_rel_size() to compute attr_needed for a
child-join relation. That change too is not needed with the above
change. This commit reverts those two changes.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/optimizer/util/relnode.c | 20 ++++++++++++++++++++
 1 file changed, 20 insertions(+)

diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 39f5729b91..2f1137c13d 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -802,6 +802,19 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 
 	appinfos = find_appinfos_by_relids(root, joinrel->relids, &nappinfos);
 
+	/*
+	 * The first pair of joining relations for the parent joinrel may not
+	 * produce a partition-wise join because of the restrictions in
+	 * partition_bounds_merge(). Hence the first pair of joining relations,
+	 * which is used to build the child join and presented here, for the child
+	 * joinrel does not correspond to the first pair of joining relations for
+	 * the parent joinrel. The targetlist built using different pairs have the
+	 * targetlist nodes arranged in different order. An appendrel expects that
+	 * all its children have their targetlists ordered in the same fashion.
+	 * Hence translate the parent's targetlist so that parent and child
+	 * joinrels have their targetlists in sync.
+	 */
+	joinrel->reltarget = copy_pathtarget(parent_joinrel->reltarget);
 	/* Set up reltarget struct */
 	build_child_join_reltarget(root, parent_joinrel, joinrel,
 							   nappinfos, appinfos);
@@ -907,6 +920,12 @@ build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 	Relids		relids = joinrel->relids;
 	ListCell   *vars;
 
+	/*
+	 * We only see parent joins. Targetlist of a child-join is computed by
+	 * translating corresponding parent join's targetlist.
+	 */
+	Assert(joinrel->reloptkind == RELOPT_JOINREL);
+
 	foreach(vars, input_rel->reltarget->exprs)
 	{
 		Var		   *var = (Var *) lfirst(vars);
@@ -934,6 +953,7 @@ build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 
 		/* Is it still needed above this joinrel? */
 		ndx = var->varattno - baserel->min_attr;
+
 		if (bms_nonempty_difference(baserel->attr_needed[ndx], relids))
 		{
 			/* Yup, add it to the output */
-- 
2.16.4

0003-Partition-wise-join-for-1-1-1-0-0-1-partition-matching-v12.patchapplication/octet-stream; name=0003-Partition-wise-join-for-1-1-1-0-0-1-partition-matching-v12.patchDownload
From 9b649057b2224e93ce39e3b1e5bbb3c9b1cd122a Mon Sep 17 00:00:00 2001
From: erthalion <9erthalion6@gmail.com>
Date: Tue, 11 Sep 2018 21:10:46 +0200
Subject: [PATCH 3/4] Partition-wise join for 1:1, 1:0, 0:1 partition matching.

Earlier version of partition-wise join implementation allowed
partition-wise join between two relations with exactly same partition
bounds. This commit allows partition-wise join to be applied under
following conditions

1. the partition bounds of joining relations are such that rows from
given partition on one side join can join with rows from maximum one
partition on the other side i.e. bounds of a given partition on one
side match/overlap with those of maximum one partition on the other
side. If the mapping happens to be m:n where m > 1 or n > 1, we have
to gang multiple partition relations together into a single relation.
This means that we have to add simple relations during join
processing, something which is not supported right now.  ALso, in such
a case, different pairs of joining relations can produce different
partition bounds for the same join relation, which again is not
supported right now.

2. For every partition on outer side that can contribute to the result
of an OUTER join, there exists at least one (taken along with item 1,
it means exactly one)  matching partition on the inner side. To
support partition-wise join when the inner matching partition doesn't
exist, we have to add a dummy base relation corresponding to the
non-existent inner partition. We don't have support to add base
relations during join processing.

3. For hash partitioned tables however, we support partition-wise join
only when the bounds exactly match. For hash partitioning it's unusual
to have missing partitions and hence generic partition matching is not
required.

With advanced partition matching, we won't be able to apply
partition-wise join when there are missing matching partitions. So for
a given N-way join, some sub-joins may not be partitioned, while the
overall N-way join is partitioned. We can not try partition-wise join
when one or both of the joining relations are not partitioned in one
of the pairs of joining relation. Skip partition-wise join for that
pair.

An example is A IJ B LJ C where B has an extra partition compared to A
and C. Join B LJ C requires dummy relation to join the extra partition
from B to a missing partition from C, and hence can not use
partition-wise join right now and hence BC is not partitioned.  The
extra partition in B gets eliminated in A IJ B and thus it can use
partition-wise join and is partitioned. Hence (AB)C can use
partition-wise join but not A(BC).

Not every pair of joining relation (including the one presented to
build_joinrel_partition_info()) for the same joinrel can use
partition-wise join or has both the relations partitioned. Hence we
calculate the partition bounds for the join relation in
try_partition_wise_join() instead of doing it here.

The partition bounds produced for the first pair that can use
partition-wise are saved in the RelOptInfo of the joinrel. Partition
bounds produced for subsequent pairs should match that produced by the
first pair.

Support for default partition in list partitioned tables
========================================================

When a list value is present in one of the joining relations and not
the other, and the other relation has default partition, match (join)
the partition containing that list value with the default partition.
If there are multiple matches, we can not proceed with the
partition-wise join similar to the case of matching non-default
partition. If the default partition happens to be on the outer side of
the join, the resulting join partition acts as a default partition as
it will contain all the values from the default partition. If
the partition containing the list value happens to be on the outer
side of the join, the resulting join partition is associated with the
list value, since no other partition key value from the default
partition makes it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join.  Such a
default partition may get joined to a partition from the inner side,
if inner side has a list value that is not present in the outer side.
But if such a join partner does not exist, we won't be able to use
partition-wise join since there is no (dummy) relation on the inner
side to join to.

Support for matching default partition in range partitioned tables
==================================================================

When a range is present in one of the joining relations and not the
other, and the other relation has default partition, match (join) the
partition corresponding range with the default partition. If two
ranges overlap but their ranges don't match exactly, the
non-overlapping portion from one range may join the previous
(non-overlapping portion near the lower bound, if exists), next
(non-overlapping portion near the upper bound, if exists) ranges from
the other relation or default partition from the other relation. This
causes multiple partitions on the other side to be joined with a
single partition on the first side; a case we don't support. Any range
from one side which doesn't overlap with any range on the other side,
may find its join partner in the default partition and the
corresponding range partition will need to be joined with the default
partition.  If the default partition happens to be on the outer side
of the join, the resulting join partition acts as a default partition
as it will contain all the values from the default partition. If the
non-partition corresponding to the range happens to be on the outer
side of the join, the resulting join partition is associated with that
range, since partition key values from the default partition outside
that range won't make it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join.  Such a
default partition may get joined to a non-default partition from the
inner side, if inner side has a range missing in the outer side.  But
if such a join partner does not exist, we won't be able to use
partition-wise join since there is no (dummy) relation on the inner
side to join to.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/optimizer/path/joinrels.c |   95 +-
 src/backend/optimizer/util/relnode.c  |   33 +-
 src/backend/partitioning/partbounds.c | 1670 ++++++++++++++++++++++++++++++++-
 src/include/partitioning/partbounds.h |   11 +
 4 files changed, 1763 insertions(+), 46 deletions(-)

diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index d3d21fed5d..e476079143 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1312,25 +1312,31 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 					   RelOptInfo *joinrel, SpecialJoinInfo *parent_sjinfo,
 					   List *parent_restrictlist)
 {
-	int			nparts;
 	int			cnt_parts;
+	PartitionScheme part_scheme;
+	PartitionBoundInfo join_boundinfo;
+	List	   *parts1;
+	List	   *parts2;
+	ListCell   *lc1;
+	ListCell   *lc2;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1343,32 +1349,63 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	 */
 	Assert(joinrel->part_scheme == rel1->part_scheme &&
 		   joinrel->part_scheme == rel2->part_scheme);
+	part_scheme = joinrel->part_scheme;
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * Get the list of matching partitions to be joined along with the
+	 * partition bounds of the join relation. Because of the restrictions
+	 * imposed by partition matching algorithm, not every pair of joining
+	 * relations for this join will be able to use partition-wise join. But all
+	 * those pairs which can use partition-wise join will produce the same
+	 * partition bounds for the join relation.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	join_boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											part_scheme->partsupfunc,
+											part_scheme->partcollation,
+											rel1, rel2,
+											parent_sjinfo->jointype,
+											&parts1, &parts2);
+
+	if (join_boundinfo == NULL)
+		return;
 
-	nparts = joinrel->nparts;
+	if (joinrel->boundinfo == NULL)
+	{
+		Assert(joinrel->nparts == 0 && joinrel->part_rels == NULL);
+		joinrel->boundinfo = join_boundinfo;
+		joinrel->nparts = list_length(parts1);
+		Assert(joinrel->nparts == list_length(parts2));
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) *
+									joinrel->nparts);
+	}
+	else
+	{
+		Assert(partition_bounds_equal(part_scheme->partnatts,
+									  part_scheme->parttyplen,
+									  part_scheme->parttypbyval,
+									  join_boundinfo, joinrel->boundinfo));
+		/*
+		 * Every pair of joining relations should result in the same number
+		 * of child-joins.
+		 */
+		Assert(joinrel->nparts == list_length(parts1));
+		Assert(joinrel->nparts == list_length(parts2));
+		Assert(joinrel->part_rels);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	cnt_parts = 0;
+	forboth(lc1, parts1, lc2, parts2)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
+		int			part1 = lfirst_int(lc1);
+		int			part2 = lfirst_int(lc2);
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1376,6 +1413,10 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		Assert(part1 >= 0 && part2 >= 0);
+		child_rel1 = rel1->part_rels[part1];
+		child_rel2 = rel2->part_rels[part2];
+
 		/* We should never try to join two overlapping sets of rels. */
 		Assert(!bms_overlap(child_rel1->relids, child_rel2->relids));
 		child_joinrelids = bms_union(child_rel1->relids, child_rel2->relids);
@@ -1409,12 +1450,24 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 			joinrel->part_rels[cnt_parts] = child_joinrel;
 		}
 
+		/*
+		 * For every pair of joining relations, the set of matching partitions
+		 * would change. However, the base relation partitions constituting
+		 * the given child should remain same for all the joining pairs. Since
+		 * the order in which children are stored in the array of child-joins,
+		 * depends upon partition bounds of the join, which are same for all
+		 * the joining pairs, every joining pair yields the child-joins in the
+		 * same order.
+		 */
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
 
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
+		cnt_parts++;
 	}
+
+	Assert(cnt_parts == joinrel->nparts);
 }
 
 /*
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 2f1137c13d..4f15cd8e8b 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1627,7 +1627,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1640,24 +1640,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1669,17 +1651,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index ea533090a4..68179d665d 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -54,7 +54,58 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 static List *get_range_nulltest(PartitionKey key);
 static bool partition_hbounds_equal(PartitionBoundInfo b1,
 									PartitionBoundInfo b2);
-
+static PartitionBoundInfo partition_range_bounds_merge(
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 List **outer_parts, List **inner_parts,
+							 JoinType jointype, int partnatts,
+							 FmgrInfo *supfuncs, Oid *collations);
+static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype);
+static PartitionBoundInfo partition_hash_bounds_merge(RelOptInfo *outer_rel,
+							RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype);
+static void generate_matching_part_pairs(PartitionMap *outer_maps,
+										 PartitionMap *inner_maps,
+										 int nparts1, int nparts2,
+										 JoinType jointype, int nparts,
+										 List **parts1, List **parts2);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int map_and_merge_partitions(PartitionMap *outer_maps,
+										PartitionMap *inner_maps,
+										int index1, int index2, int *next_index);
+static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *collations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2);
+static bool partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
+						   Oid *collations, PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, int *ub_cmpval,
+						   int *lb_cmpval);
+static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes);
+static bool merge_default_partitions(PartitionBoundInfo outer_bi,
+						 			 PartitionBoundInfo inner_bi,
+						 			 PartitionMap *outer_maps,
+						 			 PartitionMap *inner_maps,
+						 			 JoinType jointype,
+									 int *next_index, int *default_index);
+static bool merge_null_partitions(PartitionBoundInfo outer_bi,
+								  PartitionBoundInfo inner_bi,
+								  PartitionMap *outer_maps,
+								  PartitionMap *inner_maps,
+								  JoinType jointype,
+								  int *next_index, int *null_index,
+								  int *default_index);
 
 /*
  * get_qual_from_partbound
@@ -2332,3 +2383,1620 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * The function produces the partition bounds for a join between two relations
+ * whose partition bounds are given. The function also returns two lists of
+ * partition indexes one for each of the joining relations. Both the lists
+ * contain the same number of elements. The partition indexes at the same
+ * positions in the lists indicate the pair partitions, one from each side, to
+ * be joined and the position itself corresponds to the index of partition
+ * produced by that child-join in the partitioned join.
+ *
+ * The function returns NULL if we can not find the matching pair of
+ * partitions. This happens if 1. multiple partitions on one side match with
+ * one partition on the other side. 2. a given partition on the outer side
+ * doesn't have a matching partition on the inner side. We can not support the
+ * first case since we don't have a way to represent multiple partitions as a
+ * single relation (RelOptInfo) and then perform join using the ganged
+ * relation. We can not support the second case since the missing inner
+ * partition needs to be represented as an empty relation and we don't have a
+ * way to introduce empty relation during join planning after creating paths
+ * for all the base relations.
+ */
+extern PartitionBoundInfo
+partition_bounds_merge(int partnatts, FmgrInfo *partsupfunc,
+					   Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
+{
+	PartitionBoundInfo 	merged_bounds;
+	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
+					   	inner_binfo = inner_rel->boundinfo;
+	char				strategy = outer_binfo->strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partsupfunc,
+														partcollation,
+														outer_rel, inner_rel,
+														outer_parts, inner_parts,
+														jointype);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(outer_rel, inner_rel,
+														 outer_parts, inner_parts,
+														 jointype, partnatts,
+														 partsupfunc,
+														 partcollation);
+			break;
+
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = partition_hash_bounds_merge(outer_rel, inner_rel,
+														outer_parts, inner_parts,
+														jointype);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	Assert((*outer_parts == NIL || *inner_parts != NIL) &&
+		   (*inner_parts == NIL || *outer_parts != NIL));
+
+	return merged_bounds;
+}
+
+/*
+ * partition_get_range_bounds
+ *
+ * Given the index of lower bound in datums array, return lower and upper
+ * bounds and the index of the partition with that lower bound.
+ */
+static int
+partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
+						   PartitionRangeBound *lower,
+						   PartitionRangeBound *upper)
+{
+	int			part_index;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The lower bound should correspond to a valid partition. */
+	part_index = bi->indexes[lb_index + 1];
+	Assert(part_index >= 0);
+
+	lower->kind = bi->kind[lb_index];
+	lower->datums = bi->datums[lb_index];
+	lower->lower = true;
+	upper->kind = bi->kind[lb_index + 1];
+	upper->datums = bi->datums[lb_index + 1];
+	upper->lower = false;
+
+	return part_index;
+}
+
+/*
+ * partition_range_get_next_lb_index
+ *
+ * Given the index of lower bound in datums array return the
+ * index of lower bound of the next partition. When the given index corresponds
+ * to the last partition, return number of datums (ndatums).
+ */
+static int
+partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
+{
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The partition index corresponding to the upper bound should be valid. */
+	Assert(bi->indexes[lb_index + 1] >= 0);
+
+	/*
+	 * If there are no bounds left beyond the upper bound, we have reached the
+	 * last partition.
+	 */
+	if (lb_index + 2 < bi->ndatums)
+	{
+		/*
+		 * If the bound next to the upper bound corresponds to no partition,
+		 * that's the next lower bound of the next partition. Otherwise, the
+		 * current upper bound is the lower bound of the next partition.
+		 */
+		if (bi->indexes[lb_index + 2] < 0)
+			return lb_index + 2;
+		else
+			return lb_index + 1;
+	}
+	else
+		return bi->ndatums;
+}
+
+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *partcollations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2)
+{
+	return partition_rbound_cmp(partnatts, partsupfunc, partcollations,
+								bound1->datums, bound1->kind, bound1->lower,
+								bound2);
+}
+
+/*
+ * partition_range_cmp
+ *
+ * Compare the bounds of two range partitions. Set ub_cmpval <, = or > 0, if the
+ * first partition's upper bound is lower than, equal to or higher than the
+ * second partition's upper bound resp. Similarly set lb_cmpval <, =  or > 0,
+ * if the first partition's lower bound is lower than, equal to or higher than
+ * the second partition's lower bound resp.
+ *
+ * Return true, if the ranges overlap, otherwise return false.
+ */
+static bool
+partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
+					PartitionRangeBound *lower_bound1,
+					PartitionRangeBound *upper_bound1,
+					PartitionRangeBound *lower_bound2,
+					PartitionRangeBound *upper_bound2, int *ub_cmpval,
+					int *lb_cmpval)
+{
+	bool		overlap;
+
+	/*
+	 * Compare upper bound of the first partition with the lower bound of the
+	 * second and vice-versa. If lower bound is higher than the upper bound,
+	 * the partitions are not overlapping. All other cases indicate overlapping
+	 * partitions.
+	 */
+	if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+								  lower_bound1, upper_bound2) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = 1;
+		*lb_cmpval = 1;
+	}
+	else if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+									   lower_bound2, upper_bound1) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = -1;
+		*lb_cmpval = -1;
+	}
+	else
+	{
+		overlap = true;
+		*ub_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, upper_bound1,
+											   upper_bound2);
+		*lb_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, lower_bound1,
+											   lower_bound2);
+	}
+
+	return overlap;
+}
+
+/*
+ * partition_range_merge
+ *
+ * Merge the partition bounds of given two partitions such that the join
+ * between the given two partitions fits merged bounds.
+ *
+ * "merged_upper" will be set to one of the given upper bounds and
+ * "merged_lower" will be set to one of the given lower bounds.
+ */
+static void
+partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
+					  Oid *partcollations, JoinType jointype,
+					  PartitionRangeBound *left_lb,
+					  PartitionRangeBound *left_ub,
+					  PartitionRangeBound *right_lb,
+					  PartitionRangeBound *right_ub,
+					  PartitionRangeBound **merged_lb,
+					  PartitionRangeBound **merged_ub)
+{
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = left_ub;
+			*merged_lb = left_lb;
+			break;
+
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) < 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) > 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		case JOIN_FULL:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) > 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) < 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+
+	return;
+}
+
+/*
+ * Add the lower bound of the next range to the list of bounds, if the lower
+ * bound is higher or equal to the previous upper bound. If successful return
+ * true, otherwise false.
+ */
+static bool
+partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
+							  Oid *partcollations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		prev_ub.datums = llast(*merged_datums);
+		prev_ub.kind = llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  next_lb_datums, next_lb_kind, false,
+									  &prev_ub);
+	}
+
+	/*
+	 * The lower bound is lower than the last upper bound, thus does not fit
+	 * the bounds created so far and hence can not be merged with the existing
+	 * bounds.
+	 */
+	if (cmpval < 0)
+		return false;
+
+	/*
+	 * Add bounds of the new merged partition. If the next lower bound is
+	 * higher than the last upper bound, add new range with index
+	 * corresponding to the lower bound as -1. If the merged lower bound
+	 * is same as the last merged upper bound, the last upper bound will be
+	 * reused as the lower bound of the next range.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, next_lb_datums);
+		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	return true;
+}
+
+/*
+ * handle_missing_partition
+ *
+ * If a range appears in one of the joining relations but not the other, a row
+ * in the corresponding partition will not have any join partner in the other
+ * relation, unless the other relation has a default partition. If a given list
+ * value is present in one joining relation but not the other, the default
+ * partition on the other side may contain that value.
+ *
+ * In both these cases, such an extra partition forms a joining pair with the
+ * default partition, if any,  on the other side.
+ *
+ * If the default partition happens to be on the outer side of the join, the
+ * resultant partition will act as the default partition of the join relation.
+ * Otherwise the resultant partition will be associated with the range.
+ *
+ * When the default partition is not present in the other relation, the rows in
+ * the extra partition will be included in the bounds of the join result, if it
+ * appears on the outer side of the join, since all rows from the outer side
+ * are included in the join result.
+ *
+ * This function handles all these cases.
+ *
+ * maps_with_missing and missing_side_default are the partition maps (See
+ * partition_range/list_bounds_merge() for details) and the index of default
+ * partition respectively corresponding the side with missing partition.
+ *
+ * maps_with_extra and extra_part are the partition maps (See
+ * partition_range/list_bounds_merge() for details) and the index of extra
+ * partition respectively corresponding to the side with the extra partition.
+ *
+ * It returns true if the matching succeeds, otherwise returns false.
+ */
+static bool
+handle_missing_partition(PartitionMap *maps_with_missing,
+						 PartitionMap *maps_with_extra,
+						 int missing_side_default,
+						 int extra_part,
+						 bool missing_side_outer,
+						 bool missing_side_inner,
+						 int *next_index, int *default_index,
+						 int *merged_index)
+{
+	bool missing_has_default = (missing_side_default != -1);
+
+	if (missing_has_default)
+	{
+		*merged_index = map_and_merge_partitions(maps_with_missing,
+												 maps_with_extra,
+												 missing_side_default,
+												 extra_part,
+												 next_index);
+		if (*merged_index < 0)
+			return false;
+
+		if (missing_side_outer)
+		{
+			/*
+			 * Default partition on the outer side forms the default
+			 * partition of the join result.
+			 */
+			if (*default_index < 0)
+				*default_index = *merged_index;
+			else if(*default_index != *merged_index)
+			{
+				/*
+				 * Ended up with default partition on the outer side
+				 * being joined with multiple partitions on the inner
+				 * side. We don't support this case.
+				 */
+				return false;
+			}
+
+			/*
+			 * Since the merged partition acts as a default partition, it
+			 * doesn't need a separate index.
+			 */
+			*merged_index = -1;
+		}
+	}
+	else if (missing_side_inner)
+	{
+		/*
+		 * If this partition has already been mapped (say because we
+		 * found an overlapping range earlier), we know where does it
+		 * fit in the join result. Nothing to do in that case. Else
+		 * create a new merged partition.
+		 */
+		PartitionMap *extra_map = &maps_with_extra[extra_part];
+		if (extra_map->to < 0)
+		{
+			extra_map->to = *next_index;
+			*next_index = *next_index + 1;
+			*merged_index = extra_map->to;
+		}
+	}
+
+	return true;
+}
+
+static PartitionMap*
+init_partition_map(RelOptInfo *rel)
+{
+	int i, nparts = rel->nparts;
+	PartitionMap *map;
+
+	map = (PartitionMap *) palloc(sizeof(PartitionMap) * nparts);
+
+	for (i = 0; i < nparts; i++)
+	{
+		map[i].from = -1;
+		map[i].to = -1;
+	}
+
+	return map;
+}
+
+/*
+ * Allocate and initialize partition maps. We maintain four maps, two maps
+ * for each joining relation. pmap[i] gives the partition from the other
+ * relation which would join with ith partition of the given relation.
+ * Partition i from the given relation will join with partition pmap[i]
+ * from the other relation to produce partition mmap[i] of the join (merged
+ * partition).
+ *
+ * pmap[i] = -1 indicates that ith partition of a given relation does not
+ * have a matching partition from the other relation.
+ *
+ * mmap[i] = -1 indicates that ith partition of a given relation does not
+ * contribute to the join result. That can happen only when the given
+ * relation is the inner relation and it doesn't have a matching partition
+ * from the outer relation, hence pmap[i] should be -1.
+ *
+ * In case of an outer join, every partition of the outer join will appear
+ * in the join result, and thus has mmap[i] set for all i. But it's not
+ * necessary that every partition on the outer side will have a matching
+ * partition on the inner side. In such a case, we end up with pmap[i] = -1
+ * and mmap[i] != -1.
+ */
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 List **outer_parts, List **inner_parts,
+							 JoinType jointype, int partnatts,
+							 FmgrInfo *partsupfuncs, Oid *partcollations)
+
+{
+	PartitionMap *outer_maps = NULL;
+	PartitionMap *inner_maps = NULL;
+	int			outer_part = 0;
+	int			inner_part = 0;
+	PartitionBoundInfo merged_bounds = NULL;
+	int			outer_lb_index;
+	int			inner_lb_index;
+	int			next_index;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	List	   *merged_kinds = NIL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			inner_default = inner_bi->default_index;
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	outer_maps = init_partition_map(outer_rel);
+	inner_maps = init_partition_map(inner_rel);
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = 0;
+	inner_lb_index = 0;
+	next_index = 0;
+	while (outer_lb_index < outer_bi->ndatums ||
+		   inner_lb_index < inner_bi->ndatums)
+	{
+		PartitionRangeBound outer_lb, outer_ub,
+							inner_lb, inner_ub,
+							*merged_lb = NULL,
+							*merged_ub = NULL;
+
+		int			merged_index = -1;
+		bool		overlap;
+		bool		finished_outer = false;
+		bool		finished_inner = false;
+
+		/* Result of bounds comparison per partition_rbound_cmp(). */
+		int			ub_cmpval;	/* Upper bounds comparison result. */
+		int			lb_cmpval;	/* Lower bounds comparison result. */
+
+		/* Get the range bounds of the next pair of partitions. */
+		if (outer_lb_index < outer_bi->ndatums)
+			outer_part = partition_get_range_bounds(outer_bi, outer_lb_index,
+												&outer_lb, &outer_ub);
+		else
+			finished_outer = true;
+
+		if (inner_lb_index < inner_bi->ndatums)
+			inner_part = partition_get_range_bounds(inner_bi, inner_lb_index,
+												&inner_lb, &inner_ub);
+		else
+			finished_inner = true;
+
+		Assert(!finished_outer || !finished_inner);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (finished_outer)
+		{
+			overlap = false;
+			ub_cmpval = 1;
+			lb_cmpval = 1;
+		}
+		else if (finished_inner)
+		{
+			overlap = false;
+			ub_cmpval = -1;
+			lb_cmpval = -1;
+		}
+		else
+			overlap = partition_range_cmp(partnatts, partsupfuncs, partcollations,
+										  &outer_lb, &outer_ub, &inner_lb,
+										  &inner_ub, &ub_cmpval, &lb_cmpval);
+
+		if (overlap)
+		{
+			/*
+			 * The rows from overlapping portion of ranges on both sides may
+			 * join, hence the corresponding pair of partitions form a joining
+			 * pair. Match them and produce the bounds of the joint partition
+			 * and its index by merging the bounds according to the type of
+			 * join.
+			 */
+			partition_range_merge(partnatts, partsupfuncs, partcollations,
+								  jointype, &outer_lb, &outer_ub, &inner_lb,
+								  &inner_ub, &merged_lb, &merged_ub);
+
+			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+													outer_part, inner_part,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				/* Failed to match the partitions. */
+				return NULL;
+			}
+
+			/*
+			 * If the ranges overlap but don't exactly match, a row from
+			 * non-overlapping portion of the range from one side of join may
+			 * find its join partner in the previous or next overlapping
+			 * partition or default partition on the other side , if such a
+			 * partition exists. All those cases, if true, will cause one
+			 * partition from that side to match at least two partitions on the
+			 * other side; a case that we do not support now. Previous
+			 * partition has been delt with in the previous iteration of this
+			 * loop, next partition will be delt in the next iteration. We will
+			 * deal with the default partition here.
+			 */
+			if ((lb_cmpval < 0 && inner_has_default) ||
+				/* Non-overlapping range on the lower side of outer range. */
+				(lb_cmpval > 0 && outer_has_default) ||
+				/* Non-overlapping range on the lower side of inner range. */
+				(ub_cmpval < 0 && outer_has_default) ||
+				/* Non-overlapping range on the upper side of inner range. */
+				(ub_cmpval > 0 && inner_has_default))
+				/* Non-overlapping range on the upper side of outer range. */
+				return NULL;
+		}
+
+		if (ub_cmpval == 0)
+		{
+			/* Upper bounds of both the ranges match. */
+			Assert(overlap);
+
+			/* Move to the next pair of partitions. */
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the inner side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &outer_lb;
+				merged_ub = &outer_ub;
+
+				/*
+				 * For a FULL join, inner relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the inner relation acts as
+				 * INNER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_inner = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_outer = (jointype == JOIN_FULL);
+				if (!handle_missing_partition(inner_maps,
+											  outer_maps,
+											  inner_default,
+											  outer_part,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the outer side. */
+			Assert(!finished_outer);
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+		}
+		else
+		{
+			Assert(ub_cmpval > 0);
+
+			/* Upper bound of outer range higher than that of the inner. */
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the outer side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &inner_lb;
+				merged_ub = &inner_ub;
+
+				/*
+				 * For a FULL join, outer relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the outer relation acts as
+				 * OUTER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_outer = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_inner = (jointype == JOIN_FULL);
+
+				if (!handle_missing_partition(outer_maps,
+											  inner_maps,
+											  outer_default,
+											  inner_part,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the inner side. */
+			Assert (!finished_inner);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+
+		if (merged_index < 0)
+		{
+			/* We didn't find a new merged partition. */
+			continue;
+		}
+
+		/*
+		 * We have a valid partition index for the next partition of join. The
+		 * partition should have valid range.
+		 */
+		Assert(merged_lb && merged_ub);
+
+		/* Try merging new lower bound with the last upper bound. */
+		if (!partition_range_merge_next_lb(partnatts, partsupfuncs,
+										   partcollations,
+										   merged_lb->datums,
+										   merged_lb->kind, &merged_datums,
+										   &merged_kinds, &merged_indexes))
+			return NULL;
+
+		/* Add upper bound with the merged partition index. */
+		merged_datums = lappend(merged_datums, merged_ub->datums);
+		merged_kinds = lappend(merged_kinds, merged_ub->kind);
+		merged_indexes = lappend_int(merged_indexes, merged_index);
+	}
+
+	if (!merge_default_partitions(outer_bi, inner_bi,
+										  outer_maps, inner_maps,
+										  jointype, &next_index,
+										  &default_index))
+		return NULL;
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_maps, inner_maps,
+								 outer_nparts, inner_nparts,
+								 jointype, next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes,
+													  merged_kinds,
+													  -1, default_index);
+	}
+
+	/* Free any memory we used in this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype)
+{
+	PartitionMap *outer_maps = NULL;
+	PartitionMap *inner_maps = NULL;
+	int			cnto;
+	int			cnti;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			next_index = 0;
+	int			null_index;
+	int			default_index = -1;
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			      *outer_indexes = outer_bi->indexes;
+	int			      *inner_indexes = inner_bi->indexes;
+	int				   outer_default = outer_bi->default_index;
+	int				   inner_default = inner_bi->default_index;
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	outer_maps = init_partition_map(outer_rel);
+	inner_maps = init_partition_map(inner_rel);
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			o_index;
+		int			i_index;
+		int			cmpval;
+		int			merged_index = -1;
+		Datum	   *merged_datum;
+		bool		finished_inner;
+		bool		finished_outer;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+		{
+			finished_outer = true;
+			odatums = NULL;
+			o_index = -1;
+		}
+		else
+		{
+			finished_outer = false;
+			odatums = outer_bi->datums[cnto];
+			o_index = outer_indexes[cnto];
+		}
+
+		if (cnti >= inner_bi->ndatums)
+		{
+			finished_inner = true;
+			idatums = NULL;
+			i_index = -1;
+		}
+		else
+		{
+			finished_inner = false;
+			idatums = inner_bi->datums[cnti];
+			i_index = inner_indexes[cnti];
+		}
+
+		/* If we exhausted both the sides, we won't enter the loop. */
+		Assert(!finished_inner || !finished_outer);
+
+		if (finished_outer)
+			cmpval = 1;
+		else if (finished_inner)
+			cmpval = -1;
+		else
+		{
+			/* Every list datum should map to a valid partition index. */
+			Assert(o_index >= 0 && i_index >= 0 &&
+				   odatums != NULL && idatums != NULL);
+
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0], idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+													o_index, i_index,
+													&next_index);
+
+			if (merged_index < 0)
+				return NULL;
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			/* A datum missing from the inner side. */
+			merged_index = -1;
+			merged_datum = odatums;
+
+			/*
+			 * For a FULL join, inner relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the inner relation acts as
+			 * INNER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_inner = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_outer = (jointype == JOIN_FULL);
+
+			if (!handle_missing_partition(inner_maps,
+										  outer_maps,
+										  inner_default,
+										  o_index,
+										  missing_side_outer,
+										  missing_side_inner,
+										  &next_index,
+										  &default_index,
+										  &merged_index))
+				return NULL;
+
+			/* Move to the next datum on the outer side. */
+			Assert(!finished_outer);
+			cnto++;
+		}
+		else
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			Assert(cmpval > 0);
+
+			/* A datum missing from the outer side. */
+			merged_index = -1;
+			merged_datum = idatums;
+
+			/*
+			 * For a FULL join, outer relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the outer relation acts as
+			 * OUTER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_outer = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_inner = (jointype == JOIN_FULL);
+
+			if (!handle_missing_partition(outer_maps,
+										  inner_maps,
+										  outer_default,
+										  i_index,
+										  missing_side_outer,
+										  missing_side_inner,
+										  &next_index,
+										  &default_index,
+										  &merged_index))
+				return NULL;
+
+			/* Move to the next datum on the right side. */
+			Assert(!finished_inner);
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	if (!merge_null_partitions(outer_bi, inner_bi,
+							   outer_maps, inner_maps,
+							   jointype, &next_index, &null_index,
+							   &default_index))
+		return NULL;
+
+	if (!merge_default_partitions(outer_bi, inner_bi,
+								  outer_maps, inner_maps,
+								  jointype, &next_index,
+								  &default_index))
+		return NULL;
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_maps, inner_maps,
+								 outer_nparts, inner_nparts,
+								 jointype, next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes, NIL,
+													  null_index, default_index);
+	}
+
+	/* Free up all extra memory before returning from this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_bounds_merge()'s arm for hash partitioned tables.
+ *
+ * If the given two hash bounds are same, the function returns the first one
+ * without any change, alongwith the lists of matching partitions. Otherwise it
+ * returns NULL.
+ *
+ * We could try merging the bounds when both the bounds have same greatest
+ * modulii. But there seems to be hardly any requirement for the same.
+ */
+static PartitionBoundInfo
+partition_hash_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype)
+{
+	int			nparts;
+	int			cnt;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * Hash partitioned table does not have explicit NULL accepting partition
+	 * and also does not have a default partition.
+	 */
+	Assert(!partition_bound_has_default(outer_bi) &&
+		   !partition_bound_has_default(inner_bi));
+	Assert(!partition_bound_accepts_nulls(outer_bi) &&
+		   !partition_bound_accepts_nulls(inner_bi));
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+
+	if (outer_nparts != inner_nparts)
+		return NULL;
+	nparts = outer_nparts;
+
+	if (outer_bi->ndatums != inner_bi->ndatums ||
+		!partition_hbounds_equal(outer_bi, inner_bi))
+		return NULL;
+
+	 /*
+	  * Cook up list of matching partitions. Since bounds are exactly same the
+	  * partitions at the same position from both the relations match.
+	  */
+	for (cnt = 0; cnt < nparts; cnt++)
+	{
+		*outer_parts = lappend_int(*outer_parts, cnt);
+		*inner_parts = lappend_int(*inner_parts, cnt);
+	}
+
+	return outer_bi;
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * If the two given partitions (given by index1 and index2 resp.) are
+ * already mapped to each other return the index of corresponding partition in
+ * the merged set of partitions.  If they do not have a merged partition
+ * associated with them, assign a new merged partition index.  If the
+ * partitions are already mapped and their mapped partitions are different from
+ * each other, they can not be merged, so return -1.
+ *
+ * partmaps1[i] gives the mapping of partitions for both relations. It
+ * describes which partition of relation 2 matches ith partition of relation 1,
+ * and which partition in the merged set matches ith partition of relation 1
+ * maps to. Similarly for partmap2.
+ *
+ * index1 and index2 are the indexes of matching partition from respective
+ * relations.
+ *
+ * *next_index is used and incremented when the given partitions require a new
+ * merged partition.
+ */
+
+static int
+map_and_merge_partitions(PartitionMap *partmaps1, PartitionMap *partmaps2,
+						 int index1, int index2, int *next_index)
+{
+	PartitionMap 	*partmap1 = &partmaps1[index1];
+	PartitionMap 	*partmap2 = &partmaps2[index2];
+	int				merged_index;
+
+	/*
+	 * If both the partitions are not mapped to each other, update the
+	 * maps.
+	 */
+	if (partmap1->from < 0 && partmap2->from < 0)
+	{
+		partmap1->from = index2;
+		partmap2->from = index1;
+	}
+
+	/*
+	 * If the given to partitions map to each other, find the corresponding
+	 * merged partition index .
+	 */
+	if (partmap1->from == index2 && partmap2->from == index1)
+	{
+		/*
+		 * If both the partitions are mapped to the same merged partition, get
+		 * the index of merged partition.
+		 */
+		if (partmap1->to == partmap2->to)
+		{
+			merged_index = partmap1->to;
+
+			/*
+			 * If the given two partitions do not have a merged partition
+			 * associated with them, allocate a new merged partition.
+			 */
+			if (merged_index < 0)
+			{
+				merged_index = *next_index;
+				*next_index = *next_index + 1;
+				partmap1->to = merged_index;
+				partmap2->to = merged_index;
+			}
+		}
+
+		/*
+		 * If partition from one relation was mapped to a merged partition but
+		 * not the partition from the other relation, map the same merged
+		 * partition to the partition from other relation, since matching
+		 * partitions map to the same merged partition.
+		 */
+		else if (partmap1->to >= 0 && partmap2->to < 0)
+		{
+			partmap2->to = partmap1->to;
+			merged_index = partmap1->to;
+		}
+		else if (partmap1->to < 0 && partmap2->to >= 0)
+		{
+			partmap1->to = partmap2->to;
+			merged_index = partmap2->to;
+		}
+		else
+		{
+			Assert(partmap1->to != partmap2->to &&
+				   partmap1->to >= 0 && partmap2->to >= 0);
+
+			/*
+			 * Both the partitions map to different merged partitions. This
+			 * means that multiple partitions from one relation matches to one
+			 * partition from the other relation. Partition-wise join does not
+			 * handle this case right now, since it requires ganging multiple
+			 * partitions together (into one RelOptInfo).
+			 */
+			merged_index = -1;
+		}
+	}
+	else
+	{
+		/*
+		 * Multiple partitions from one relation map to one partition from the
+		 * other relation. Partition-wise join does not handle this case right
+		 * now, since it requires ganging multiple partitions together (into
+		 * one RelOptInfo).
+		 */
+		merged_index = -1;
+	}
+
+	return merged_index;
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * partmaps1 map each partition from either side of the join to a merged
+ * partition resp. E.g. partmaps1[i].to gives the merged partition to which ith
+ * partition of first relation maps. Similarly for partmap2. If
+ * partmaps1[i].to == partmaps2[j].to, i and j form the matching pair of
+ * partitions.
+ *
+ * Given these maps this function produces the list pairs of partitions which
+ * when joined produce the merged partitions in the order of merged partition
+ * indexes.
+ *
+ * nparts1 and nparts2 are the number of partitions of the joining relations
+ * resp.
+ *
+ * nparts is the number of merged partitions.
+ *
+ * If successful, the pairs of partitions are returned as two separate lists,
+ * parts1 and parts2 resp., one for each side. Otherwise, those lists will be
+ * set to NIL.
+ */
+static void
+generate_matching_part_pairs(PartitionMap *partmaps1, PartitionMap *partmaps2,
+							 int nparts1, int nparts2,
+							 JoinType jointype, int nparts,
+							 List **matched_parts1, List **matched_parts2)
+{
+	bool	merged = true;
+	int		*matching1 = (int *) palloc(sizeof(int) * nparts),
+			*matching2 = (int *) palloc(sizeof(int) * nparts);
+	int 	i;
+
+	*matched_parts1 = NIL;
+	*matched_parts2 = NIL;
+
+	/* Set pairs of matching partitions. */
+	for (i = 0; i < nparts; i++)
+	{
+		if (i >= nparts1)
+			matching1[i] = -1;
+		else
+		{
+			PartitionMap outer_map = partmaps1[i];
+
+			if (outer_map.to >= 0)
+			{
+				Assert(outer_map.to < nparts);
+				matching1[outer_map.to] = i;
+			}
+		}
+
+		if (i >= nparts2)
+			matching2[i] = -1;
+		else
+		{
+			PartitionMap inner_map = partmaps2[i];
+
+			if (inner_map.to >= 0)
+			{
+				Assert(inner_map.to < nparts);
+				matching2[inner_map.to] = i;
+			}
+		}
+	}
+
+	/*
+	 * If we have a partition missing on an inner side, we need to add a dummy
+	 * relation which joins with the outer partition. If the inner relation
+	 * happens to be a base relation, it will require adding a dummy child
+	 * base relation during join processing. Right now, we freeze the base
+	 * relation arrays like PlannerInfo::simple_rte_array after planning for
+	 * base relations. Adding a new (dummy) base relation would require some
+	 * changes to that. So, right now, we do not implement partition-wise join
+	 * in such cases.
+	 */
+	for (i = 0; i < nparts; i++)
+	{
+		int			part1 = matching1[i];
+		int			part2 = matching2[i];
+
+		/* At least one of the partitions should exist. */
+		Assert(part1 >= 0 || part2 >= 0);
+
+		switch (jointype)
+		{
+			case JOIN_INNER:
+			case JOIN_SEMI:
+
+				/*
+				 * An inner or semi join can not return any row when the
+				 * matching partition on either side is missing. We should
+				 * have eliminated all such cases while merging the bounds.
+				 */
+				Assert(part1 >= 0 && part2 >= 0);
+				break;
+
+			case JOIN_LEFT:
+			case JOIN_ANTI:
+				Assert(part1 >= 0);
+				if (part2 < 0)
+					merged = false;
+				break;
+
+			case JOIN_FULL:
+				if (part1 < 0 || part2 < 0)
+					merged = false;
+				break;
+
+			default:
+				elog(ERROR, "unrecognized join type: %d", (int) jointype);
+		}
+
+		if (!merged)
+			break;
+
+		*matched_parts1 = lappend_int(*matched_parts1, part1);
+		*matched_parts2 = lappend_int(*matched_parts2, part2);
+	}
+
+	pfree(matching1);
+	pfree(matching2);
+
+	if (!merged)
+	{
+		list_free(*matched_parts1);
+		list_free(*matched_parts2);
+		*matched_parts1 = NIL;
+		*matched_parts2 = NIL;
+	}
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * Merge default partitions from both sides, if any, and assign the default
+ * partition for the join result, if necessary.
+ *
+ * If both the relations have default partitions, try mapping those to each
+ * other. If the mapping succeeds corresponding merged partition will act as
+ * the default partition of the join result.
+ *
+ * If inner side of the join has default but not the outer side, rows in it
+ * won't appear in the join result. So don't create a default partition. If
+ * outer side of the join has default but not the inner side, rows in it will
+ * appear in the join result, so create a default merged partition.
+ */
+static bool
+merge_default_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+						 PartitionMap *outer_maps, PartitionMap *inner_maps,
+						 JoinType jointype, int *next_index, int *default_index)
+{
+	int				outer_default = outer_bi->default_index;
+	int				inner_default = inner_bi->default_index;
+	bool			outer_has_default = partition_bound_has_default(outer_bi);
+	bool			inner_has_default = partition_bound_has_default(inner_bi);
+	bool			merged = true;
+	PartitionMap 	*outer_default_map = NULL;
+	PartitionMap 	*inner_default_map = NULL;
+
+	if (outer_has_default)
+		outer_default_map = &outer_maps[outer_default];
+
+	if (inner_has_default)
+		inner_default_map = &inner_maps[inner_default];
+
+	if (!outer_has_default && !inner_has_default)
+		Assert(*default_index < 0);
+	else if (outer_default_map != NULL && inner_default_map == NULL)
+	{
+		if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+			jointype == JOIN_ANTI)
+		{
+			if (outer_default_map->to < 0)
+			{
+				outer_default_map->to = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = outer_default_map->to;
+			}
+			else
+				Assert(*default_index == outer_default_map->to);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else if (outer_default_map == NULL && inner_default_map != NULL)
+	{
+		if (jointype == JOIN_FULL)
+		{
+			if (inner_default_map->to < 0)
+			{
+				inner_default_map->to = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = inner_default_map->to;
+			}
+			else
+				Assert(*default_index == inner_default_map->to);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		*default_index = map_and_merge_partitions(outer_maps, inner_maps,
+												  outer_default, inner_default,
+												  next_index);
+
+		if (*default_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
+
+/*
+ * merge_null_partitions
+ *
+ * Merge NULL partitions, i.e. a partition that can hold NULL values for a list
+ * partitioned table, if any. Find the index of merged partition to which the
+ * NULL values would belong in the join result. If one joining relation has a
+ * NULL partition but not the other, try matching it with the default partition
+ * from the other relation since the default partition may have rows with NULL
+ * partition key. We can eliminate a NULL partition when it appears only on the
+ * inner side of the join and the outer side doesn't have a default partition.
+ *
+ * When the equality operator used for join is strict, two NULL values will not
+ * be considered as equal, and thus a NULL partition can be eliminated for an
+ * inner join. But we don't check the strictness operator here.
+ */
+static bool
+merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_maps, PartitionMap *inner_maps,
+					  JoinType jointype, int *next_index,
+					  int *null_index, int *default_index)
+{
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	int			outer_ni = outer_bi->null_index;
+	int			inner_ni = inner_bi->null_index;
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	bool		merged = true;
+
+	if (!outer_has_null && !inner_has_null)
+		*null_index = -1;
+	else if (outer_has_null && !inner_has_null)
+	{
+		int			merged_index;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, inner relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the inner relation acts as
+		 * INNER relation. For INNER and SEMI join OUTER and INNER
+		 * differentiation is immaterial.
+		 */
+		missing_side_inner = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_outer = (jointype == JOIN_FULL);
+
+		merged = handle_missing_partition(inner_maps,
+										  outer_maps,
+										  inner_default,
+										  outer_ni,
+										  missing_side_outer,
+										  missing_side_inner, next_index,
+										  default_index, &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join to which the outer null partition maps
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = outer_maps[outer_ni].to;
+	}
+	else if (!outer_has_null && inner_has_null)
+	{
+		int			merged_index;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, outer relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the outer relation acts as OUTER
+		 * relation. For INNER and SEMI join OUTER and INNER differentiation is
+		 * immaterial.
+		 */
+		missing_side_outer = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_inner = (jointype == JOIN_FULL);
+		merged = handle_missing_partition(outer_maps,
+										  inner_maps,
+										  outer_default,
+										  inner_ni,
+										  missing_side_outer,
+										  missing_side_inner,
+										  next_index, default_index,
+										  &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join, to which the outer side null partition maps,
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = inner_maps[inner_ni].to;
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		*null_index = map_and_merge_partitions(outer_maps,
+											   inner_maps,
+											   outer_ni,
+											   inner_ni,
+											   next_index);
+		if (*null_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index c7535e32fc..bf11713371 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -104,6 +104,12 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+typedef struct PartitionMap
+{
+	int from;
+	int to;
+} PartitionMap;
+
 extern int	get_hash_partition_greatest_modulus(PartitionBoundInfo b);
 extern uint64 compute_partition_hash_value(int partnatts, FmgrInfo *partsupfunc,
 							 Datum *values, bool *isnull);
@@ -146,5 +152,10 @@ extern int partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 							  int nvalues, Datum *values, bool *is_equal);
 extern int partition_hash_bsearch(PartitionBoundInfo boundinfo,
 					   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
-- 
2.16.4

0004-Tests-for-0-1-1-1-and-1-0-partition-matching-v12.patchapplication/octet-stream; name=0004-Tests-for-0-1-1-1-and-1-0-partition-matching-v12.patchDownload
From adddb61b30b19b3c16ce4594bcab543ce4aa7fa5 Mon Sep 17 00:00:00 2001
From: erthalion <9erthalion6@gmail.com>
Date: Tue, 11 Sep 2018 21:11:55 +0200
Subject: [PATCH 4/4] Tests for 0:1, 1:1 and 1:0 partition matching

Rajkumar Raghuvanshi and Ashutosh Bapat.
---
 src/test/regress/expected/partition_join.out | 4131 +++++++++++++++++++++-----
 src/test/regress/sql/partition_join.sql      |  427 ++-
 2 files changed, 3855 insertions(+), 703 deletions(-)

diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 3ba3aaf2d8..5d6c7d2c67 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -8,59 +8,86 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-(4 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+(7 rows)
 
 -- left outer join, with whole-row reference; partitionwise join does not apply
 EXPLAIN (COSTS OFF)
@@ -72,35 +99,50 @@ SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER
    ->  Hash Right Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(22 rows)
 
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-      t1      |      t2      
---------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
- (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
- (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
- (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
- (550,0,0550) | 
-(12 rows)
+       t1       |       t2       
+----------------+----------------
+ (-250,0,-0250) | 
+ (-200,0,-0200) | 
+ (-150,0,-0150) | (0,-150,-0150)
+ (-100,0,-0100) | 
+ (-50,0,-0050)  | 
+ (0,0,0000)     | (0,0,0000)
+ (50,0,0050)    | 
+ (100,0,0100)   | 
+ (150,0,0150)   | (0,150,0150)
+ (200,0,0200)   | 
+ (250,0,0250)   | 
+ (300,0,0300)   | (0,300,0300)
+ (350,0,0350)   | 
+ (400,0,0400)   | 
+ (450,0,0450)   | (0,450,0450)
+ (500,0,0500)   | 
+ (550,0,0550)   | 
+ (600,0,0600)   | (0,600,0600)
+ (650,0,0650)   | 
+ (700,0,0700)   | 
+ (750,0,0750)   | (0,750,0750)
+(21 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
@@ -112,35 +154,53 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
    ->  Append
          ->  Hash Right Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Right Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t2_2.b)
-(20 rows)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+      |       | -225 | -0225
+      |       |  -75 | -0075
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(14 rows)
 
 -- full outer join, with placeholder vars
 EXPLAIN (COSTS OFF)
@@ -148,8 +208,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                             QUERY PLAN                            
 ------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               Filter: (((50) = prt1_p0.a) OR ((75) = prt2_p0.b))
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
@@ -174,7 +242,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               Filter: (((50) = prt1_p4.a) OR ((75) = prt2_p4.b))
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
@@ -211,8 +287,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Hash Left Join
+               Hash Cond: (prt1_p0.a = b)
+               ->  Seq Scan on prt1_p0
+                     Filter: ((a < 450) AND (b = 0))
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
          ->  Hash Left Join
                Hash Cond: (prt1_p1.a = b)
                ->  Seq Scan on prt1_p1
@@ -227,29 +310,42 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                ->  Hash
                      ->  Seq Scan on prt1_p2
                            Filter: ((a < 450) AND (b = 0))
-(17 rows)
+(24 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-(9 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+(14 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
                          QUERY PLAN                         
 ------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = b)
+               Filter: ((prt1_p0.b = 0) OR (a = 0))
+               ->  Seq Scan on prt1_p0
+                     Filter: (a < 450)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = b)
                Filter: ((prt1_p1.b = 0) OR (a = 0))
@@ -274,64 +370,153 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                ->  Hash
                      ->  Result
                            One-Time Filter: false
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt2_p4.b = a)
+               Filter: ((b = 0) OR (prt2_p4.a = 0))
+               ->  Seq Scan on prt2_p4
+                     Filter: (b > 250)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
-(12 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+      |       | 375 | 0375
+      |       | 450 | 0450
+      |       | 525 | 0525
+      |       | 600 | 0600
+      |       | 675 | 0675
+      |       | 750 | 0750
+(20 rows)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Semi Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Semi Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                            Filter: (a = 0)
-         ->  Nested Loop Semi Join
-               Join Filter: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
-               ->  Materialize
-                     ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
-(24 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt1_p3 t2_3
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+                           Filter: (b = 0)
+(37 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
 
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
@@ -342,27 +527,82 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
    ->  Append
          ->  Hash Anti Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Hash Anti Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Hash Anti Join
                Hash Cond: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
-(17 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(27 rows)
 
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
-  sum  |         avg          | sum  |         avg         
--------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+  sum  |         avg          | sum  |        avg         
+-------+----------------------+------+--------------------
+ 95550 | 273.0000000000000000 | 2200 | 6.2857142857142857
 (1 row)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Append
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p0 t1
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+               Index Cond: (a = t1.b)
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.b = t2_1.a)
+         ->  Seq Scan on prt2_p1 t1_1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p1 t2_1
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p2 t1_2
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+               Index Cond: (a = t1_2.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p3 t1_3
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+               Index Cond: (a = t1_3.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p4 t1_4
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+               Index Cond: (a = t1_4.b)
+(27 rows)
+
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+  b   |   c   
+------+-------
+ -225 | -0225
+  -75 | -0075
+   75 | 0075
+  225 | 0225
+  375 | 0375
+  525 | 0525
+  675 | 0675
+(7 rows)
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -374,49 +614,74 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
+                     ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
                            Index Cond: (a = t1.a)
-                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t3
                            Index Cond: (b = t2.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_1
+                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
                            Index Cond: (a = t1_1.a)
-                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3_1
                            Index Cond: (b = t2_1.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_2
+                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
                            Index Cond: (a = t1_2.a)
-                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_2
                            Index Cond: (b = t2_2.a)
-(27 rows)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                           Index Cond: (a = t1_3.a)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_3
+                           Index Cond: (b = t2_3.a)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                           Index Cond: (a = t1_4.a)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t3_4
+                           Index Cond: (b = t2_4.a)
+(43 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   | t2a | t3a | least 
------+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
- 100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
- 250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
- 400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
- 550 | 0 | 0550 |     |     |      
-(12 rows)
+  a   | b |   c   | t2a  | t3a | least 
+------+---+-------+------+-----+-------
+ -250 | 0 | -0250 |      |     |      
+ -200 | 0 | -0200 |      |     |      
+ -150 | 0 | -0150 | -150 |   0 |  -150
+ -100 | 0 | -0100 |      |     |      
+  -50 | 0 | -0050 |      |     |      
+    0 | 0 | 0000  |    0 |   0 |     0
+   50 | 0 | 0050  |      |     |      
+  100 | 0 | 0100  |      |     |      
+  150 | 0 | 0150  |  150 |   0 |   150
+  200 | 0 | 0200  |      |     |      
+  250 | 0 | 0250  |      |     |      
+  300 | 0 | 0300  |  300 |   0 |   300
+  350 | 0 | 0350  |      |     |      
+  400 | 0 | 0400  |      |     |      
+  450 | 0 | 0450  |  450 |   0 |   450
+  500 | 0 | 0500  |      |     |      
+  550 | 0 | 0550  |      |     |      
+  600 | 0 | 0600  |  600 |   0 |   600
+  650 | 0 | 0650  |      |     |      
+  700 | 0 | 0700  |      |     |      
+  750 | 0 | 0750  |  750 |   0 |   750
+(21 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
@@ -430,64 +695,95 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
          Hash Cond: ((t1.c)::text = (t2.c)::text)
          Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Seq Scan on prt1_p1 t1
-               ->  Seq Scan on prt1_p2 t1_1
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
          ->  Hash
                ->  Append
                      ->  Hash Join
                            Hash Cond: (t2.a = t3.b)
-                           ->  Seq Scan on prt1_p1 t2
+                           ->  Seq Scan on prt1_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t3
+                                 ->  Seq Scan on prt2_p0 t3
                      ->  Hash Join
                            Hash Cond: (t2_1.a = t3_1.b)
-                           ->  Seq Scan on prt1_p2 t2_1
+                           ->  Seq Scan on prt1_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t3_1
+                                 ->  Seq Scan on prt2_p1 t3_1
                      ->  Hash Join
                            Hash Cond: (t2_2.a = t3_2.b)
-                           ->  Seq Scan on prt1_p3 t2_2
+                           ->  Seq Scan on prt1_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p3 t3_2
-(26 rows)
+                                 ->  Seq Scan on prt2_p2 t3_2
+                     ->  Hash Join
+                           Hash Cond: (t2_3.a = t3_3.b)
+                           ->  Seq Scan on prt1_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t3_3
+                     ->  Hash Join
+                           Hash Cond: (t2_4.a = t3_4.b)
+                           ->  Seq Scan on prt1_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t3_4
+(38 rows)
 
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, t2.b t2b, t2.c t2c, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
-  a  | t2a | t2c  
------+-----+------
-   0 |   0 | 0000
-  50 |     | 
- 100 |     | 
- 150 | 150 | 0150
- 200 |     | 
- 250 |     | 
- 300 | 300 | 0300
- 350 |     | 
- 400 |     | 
- 450 | 450 | 0450
- 500 |     | 
- 550 |     | 
-(12 rows)
+  a   | t2a  |  t2c  
+------+------+-------
+ -250 |      | 
+ -200 |      | 
+ -150 | -150 | -0150
+ -100 |      | 
+  -50 |      | 
+    0 |    0 | 0000
+   50 |      | 
+  100 |      | 
+  150 |  150 | 0150
+  200 |      | 
+  250 |      | 
+  300 |  300 | 0300
+  350 |      | 
+  400 |      | 
+  450 |  450 | 0450
+  500 |      | 
+  550 |      | 
+  600 |  600 | 0600
+  650 |      | 
+  700 |      | 
+  750 |  750 | 0750
+(21 rows)
 
 --
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
@@ -498,32 +794,49 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
    ->  Append
          ->  Hash Join
                Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
-               ->  Seq Scan on prt2_e_p1 t2
+               ->  Seq Scan on prt2_e_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 t1
+                     ->  Seq Scan on prt1_e_p0 t1
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
-               ->  Seq Scan on prt2_e_p2 t2_1
+               Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+               ->  Seq Scan on prt1_e_p1 t1_1
+                     Filter: (c = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 t1_1
-                           Filter: (c = 0)
+                     ->  Seq Scan on prt2_e_p1 t2_1
          ->  Hash Join
                Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
-               ->  Seq Scan on prt2_e_p3 t2_2
+               ->  Seq Scan on prt2_e_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 t1_2
+                     ->  Seq Scan on prt1_e_p2 t1_2
                            Filter: (c = 0)
-(21 rows)
+         ->  Hash Join
+               Hash Cond: (((t2_3.b + t2_3.a) / 2) = ((t1_3.a + t1_3.b) / 2))
+               ->  Seq Scan on prt2_e_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p3 t1_3
+                           Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (((t2_4.b + t2_4.a) / 2) = ((t1_4.a + t1_4.b) / 2))
+               ->  Seq Scan on prt2_e_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4 t1_4
+                           Filter: (c = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
- 150 | 0 | 150 | 0
- 300 | 0 | 300 | 0
- 450 | 0 | 450 | 0
-(4 rows)
+  a   | c |  b   | c 
+------+---+------+---
+ -250 | 0 | -250 | 0
+ -100 | 0 | -100 | 0
+    0 | 0 |    0 | 0
+    0 | 0 |    0 | 0
+  150 | 0 |  150 | 0
+  300 | 0 |  300 | 0
+  450 | 0 |  450 | 0
+  600 | 0 |  600 | 0
+  750 | 0 |  750 | 0
+(9 rows)
 
 --
 -- N-way join
@@ -536,154 +849,232 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = ((t3.a + t3.b) / 2))
+               Join Filter: (t1.a = t2.b)
                ->  Hash Join
-                     Hash Cond: (t2.b = t1.a)
-                     ->  Seq Scan on prt2_p1 t2
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on prt1_e_p0 t3
                      ->  Hash
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                     Index Cond: (((a + b) / 2) = t2.b)
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
+                     Index Cond: (b = ((t3.a + t3.b) / 2))
          ->  Nested Loop
                Join Filter: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_1.b = t1_1.a)
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
+               ->  Index Scan using iprt1_e_p4_ab2 on prt1_e_p1 t3_1
                      Index Cond: (((a + b) / 2) = t2_1.b)
          ->  Nested Loop
                Join Filter: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_2.b = t1_2.a)
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
+               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_2
                      Index Cond: (((a + b) / 2) = t2_2.b)
-(33 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = ((t3_3.a + t3_3.b) / 2))
+               ->  Nested Loop
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t2_3.b)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t2_4.b)
+               ->  Hash Join
+                     Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                     ->  Seq Scan on prt1_e_p4 t3_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_p4 t1_4
+                                 Filter: (b = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = ((t3_4.a + t3_4.b) / 2))
+(52 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
-(4 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(8 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                          QUERY PLAN                          
---------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Hash Right Join
                Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
-               ->  Seq Scan on prt1_e_p1 t3
+               ->  Seq Scan on prt1_e_p0 t3
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2.b = t1.a)
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_1.a)
-               ->  Seq Scan on prt1_e_p2 t3_1
+               ->  Seq Scan on prt1_e_p1 t3_1
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_1.b = t1_1.a)
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_2.a)
-               ->  Seq Scan on prt1_e_p3 t3_2
+               ->  Seq Scan on prt1_e_p2 t3_2
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_2.b = t1_2.a)
-                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                                        Filter: (b = 0)
-(33 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (t1_3.a = b)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (t1_3.a = ((a + b) / 2))
+         ->  Hash Right Join
+               Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+               ->  Seq Scan on prt1_e_p4 t3_4
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_4.b = t1_4.a)
+                           ->  Seq Scan on prt2_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_p4 t1_4
+                                       Filter: (b = 0)
+(51 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                            QUERY PLAN                             
--------------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p1 t3
+                           ->  Seq Scan on prt1_e_p0 t3
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p1_b on prt2_p1 t2
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
                      Index Cond: (t1.a = b)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p2 t3_1
+                           ->  Seq Scan on prt1_e_p1 t3_1
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
+               ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
                      Index Cond: (t1_1.a = b)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p3 t3_2
+                           ->  Seq Scan on prt1_e_p2 t3_2
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
                      Index Cond: (t1_2.a = b)
-(30 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_e_p3 t3_3
+                           Filter: (c = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                           Index Cond: (a = ((t3_3.a + t3_3.b) / 2))
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (t1_3.a = b)
+         ->  Nested Loop Left Join
+               ->  Hash Right Join
+                     Hash Cond: (t1_4.a = ((t3_4.a + t3_4.b) / 2))
+                     ->  Seq Scan on prt1_p4 t1_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p4 t3_4
+                                 Filter: (c = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (t1_4.a = b)
+(47 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -692,21 +1083,34 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                                                    QUERY PLAN                                                   
 ----------------------------------------------------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   Sort Key: prt1_p0.a, prt2_p0.b, ((prt1_e_p0.a + prt1_e_p0.b))
    ->  Append
          ->  Hash Full Join
-               Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
-               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               Hash Cond: (prt1_p0.a = ((prt1_e_p0.a + prt1_e_p0.b) / 2))
+               Filter: ((prt1_p0.a = (50)) OR (prt2_p0.b = (75)) OR (((prt1_e_p0.a + prt1_e_p0.b) / 2) = (50)))
                ->  Hash Full Join
-                     Hash Cond: (prt1_p1.a = prt2_p1.b)
-                     ->  Seq Scan on prt1_p1
+                     Hash Cond: (prt1_p0.a = prt2_p0.b)
+                     ->  Seq Scan on prt1_p0
                            Filter: (b = 0)
                      ->  Hash
-                           ->  Seq Scan on prt2_p1
+                           ->  Seq Scan on prt2_p0
                                  Filter: (a = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1
+                     ->  Seq Scan on prt1_e_p0
                            Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = prt1_p1.a)
+               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               ->  Seq Scan on prt1_e_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on prt1_p1
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1
+                                       Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
                Filter: ((prt1_p2.a = (50)) OR (prt2_p2.b = (75)) OR (((prt1_e_p2.a + prt1_e_p2.b) / 2) = (50)))
@@ -733,7 +1137,20 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                ->  Hash
                      ->  Seq Scan on prt1_e_p3
                            Filter: (c = 0)
-(42 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = ((prt1_e_p4.a + prt1_e_p4.b) / 2))
+               Filter: ((prt1_p4.a = (50)) OR (prt2_p4.b = (75)) OR (((prt1_e_p4.a + prt1_e_p4.b) / 2) = (50)))
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = prt2_p4.b)
+                     ->  Seq Scan on prt1_p4
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4
+                                 Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4
+                           Filter: (c = 0)
+(68 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
@@ -751,173 +1168,261 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = t1_3.b)
+               Join Filter: (t1.a = t1_5.b)
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Join
-                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
-                           ->  Seq Scan on prt1_e_p1 t2
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t1_3
+                                 ->  Seq Scan on prt2_p0 t1_5
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
                      Index Cond: (a = ((t2.a + t2.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_1.a = t1_4.b)
+               Join Filter: (t1_1.a = t1_6.b)
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Join
-                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
-                           ->  Seq Scan on prt1_e_p2 t2_1
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_6.b)
+                           ->  Seq Scan on prt1_e_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t1_4
+                                 ->  Seq Scan on prt2_p1 t1_6
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
                      Index Cond: (a = ((t2_1.a + t2_1.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_2.a = t1_5.b)
+               Join Filter: (t1_2.a = t1_7.b)
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
+                           ->  Seq Scan on prt2_p2 t1_7
                                  Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
+                           ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t2_2
+                                 Index Cond: (((a + b) / 2) = t1_7.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t1_8.b)
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Nested Loop
+                           ->  Seq Scan on prt2_p3 t1_8
+                                 Filter: (a = 0)
+                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_3
+                                 Index Cond: (((a + b) / 2) = t1_8.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = ((t2_3.a + t2_3.b) / 2))
+                     Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t1_9.b)
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Join
+                           Hash Cond: (((t2_4.a + t2_4.b) / 2) = t1_9.b)
+                           ->  Seq Scan on prt1_e_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t1_9
+                                       Filter: (a = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = ((t2_4.a + t2_4.b) / 2))
+                     Filter: (b = 0)
+(66 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                                  QUERY PLAN                                   
--------------------------------------------------------------------------------
+                                   QUERY PLAN                                    
+---------------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
-                           ->  Seq Scan on prt2_p1 t1_3
+                           Hash Cond: (t1_5.b = ((t1_10.a + t1_10.b) / 2))
+                           ->  Seq Scan on prt2_p0 t1_5
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t1_6
+                                 ->  Seq Scan on prt1_e_p0 t1_10
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
-                     Index Cond: (a = t1_3.b)
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t1_5.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
-                           ->  Seq Scan on prt2_p2 t1_4
+                           Hash Cond: (t1_6.b = ((t1_11.a + t1_11.b) / 2))
+                           ->  Seq Scan on prt2_p1 t1_6
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t1_7
+                                 ->  Seq Scan on prt1_e_p1 t1_11
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
-                     Index Cond: (a = t1_4.b)
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
+                     Index Cond: (a = t1_6.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_7.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_7.b = ((t1_12.a + t1_12.b) / 2))
+                           ->  Seq Scan on prt2_p2 t1_7
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p2 t1_12
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t1_7.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  Unique
                      ->  Sort
-                           Sort Key: t1_5.b
+                           Sort Key: t1_8.b
                            ->  Hash Semi Join
-                                 Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
-                                 ->  Seq Scan on prt2_p3 t1_5
+                                 Hash Cond: (t1_8.b = ((t1_13.a + t1_13.b) / 2))
+                                 ->  Seq Scan on prt2_p3 t1_8
                                  ->  Hash
-                                       ->  Seq Scan on prt1_e_p3 t1_8
+                                       ->  Seq Scan on prt1_e_p3 t1_13
                                              Filter: (c = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t1_5.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t1_8.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_9.b = ((t1_14.a + t1_14.b) / 2))
+                           ->  Seq Scan on prt2_p4 t1_9
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t1_14
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = t1_9.b)
                      Filter: (b = 0)
-(40 rows)
+(64 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 -- test merge joins
 SET enable_hashjoin TO off;
 SET enable_nestloop TO off;
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                           QUERY PLAN                           
-----------------------------------------------------------------
+                            QUERY PLAN                            
+------------------------------------------------------------------
  Merge Append
    Sort Key: t1.a
    ->  Merge Semi Join
-         Merge Cond: (t1.a = t1_3.b)
+         Merge Cond: (t1.a = t1_5.b)
          ->  Sort
                Sort Key: t1.a
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+               Merge Cond: (t1_5.b = (((t1_10.a + t1_10.b) / 2)))
                ->  Sort
-                     Sort Key: t1_3.b
-                     ->  Seq Scan on prt2_p1 t1_3
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on prt2_p0 t1_5
                ->  Sort
-                     Sort Key: (((t1_6.a + t1_6.b) / 2))
-                     ->  Seq Scan on prt1_e_p1 t1_6
+                     Sort Key: (((t1_10.a + t1_10.b) / 2))
+                     ->  Seq Scan on prt1_e_p0 t1_10
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_1.a = t1_4.b)
+         Merge Cond: (t1_1.a = t1_6.b)
          ->  Sort
                Sort Key: t1_1.a
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+               Merge Cond: (t1_6.b = (((t1_11.a + t1_11.b) / 2)))
                ->  Sort
-                     Sort Key: t1_4.b
-                     ->  Seq Scan on prt2_p2 t1_4
+                     Sort Key: t1_6.b
+                     ->  Seq Scan on prt2_p1 t1_6
                ->  Sort
-                     Sort Key: (((t1_7.a + t1_7.b) / 2))
-                     ->  Seq Scan on prt1_e_p2 t1_7
+                     Sort Key: (((t1_11.a + t1_11.b) / 2))
+                     ->  Seq Scan on prt1_e_p1 t1_11
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_2.a = t1_5.b)
+         Merge Cond: (t1_2.a = t1_7.b)
          ->  Sort
                Sort Key: t1_2.a
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+               Merge Cond: (t1_7.b = (((t1_12.a + t1_12.b) / 2)))
                ->  Sort
-                     Sort Key: t1_5.b
-                     ->  Seq Scan on prt2_p3 t1_5
+                     Sort Key: t1_7.b
+                     ->  Seq Scan on prt2_p2 t1_7
                ->  Sort
-                     Sort Key: (((t1_8.a + t1_8.b) / 2))
-                     ->  Seq Scan on prt1_e_p3 t1_8
+                     Sort Key: (((t1_12.a + t1_12.b) / 2))
+                     ->  Seq Scan on prt1_e_p2 t1_12
                            Filter: (c = 0)
-(47 rows)
+   ->  Merge Semi Join
+         Merge Cond: (t1_3.a = t1_8.b)
+         ->  Sort
+               Sort Key: t1_3.a
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_8.b = (((t1_13.a + t1_13.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_8.b
+                     ->  Seq Scan on prt2_p3 t1_8
+               ->  Sort
+                     Sort Key: (((t1_13.a + t1_13.b) / 2))
+                     ->  Seq Scan on prt1_e_p3 t1_13
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_4.a = t1_9.b)
+         ->  Sort
+               Sort Key: t1_4.a
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_9.b = (((t1_14.a + t1_14.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_9.b
+                     ->  Seq Scan on prt2_p4 t1_9
+               ->  Sort
+                     Sort Key: (((t1_14.a + t1_14.b) / 2))
+                     ->  Seq Scan on prt1_e_p4 t1_14
+                           Filter: (c = 0)
+(77 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
@@ -934,14 +1439,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
                            ->  Sort
                                  Sort Key: (((t3.a + t3.b) / 2))
-                                 ->  Seq Scan on prt1_e_p1 t3
+                                 ->  Seq Scan on prt1_e_p0 t3
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1.a
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                ->  Sort
                      Sort Key: t2.b
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Merge Left Join
                Merge Cond: (t1_1.a = t2_1.b)
                ->  Sort
@@ -950,14 +1455,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
                            ->  Sort
                                  Sort Key: (((t3_1.a + t3_1.b) / 2))
-                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                 ->  Seq Scan on prt1_e_p1 t3_1
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_1.a
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                ->  Sort
                      Sort Key: t2_1.b
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Merge Left Join
                Merge Cond: (t1_2.a = t2_2.b)
                ->  Sort
@@ -966,32 +1471,74 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
                            ->  Sort
                                  Sort Key: (((t3_2.a + t3_2.b) / 2))
-                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                 ->  Seq Scan on prt1_e_p2 t3_2
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_2.a
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                ->  Sort
                      Sort Key: t2_2.b
-                     ->  Seq Scan on prt2_p3 t2_2
-(51 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Merge Left Join
+               Merge Cond: (t1_3.a = t2_3.b)
+               ->  Sort
+                     Sort Key: t1_3.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_3.a + t3_3.b) / 2)) = t1_3.a)
+                           ->  Sort
+                                 Sort Key: (((t3_3.a + t3_3.b) / 2))
+                                 ->  Seq Scan on prt1_e_p3 t3_3
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_3.a
+                                 ->  Seq Scan on prt1_p3 t1_3
+               ->  Sort
+                     Sort Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Merge Left Join
+               Merge Cond: (t1_4.a = t2_4.b)
+               ->  Sort
+                     Sort Key: t1_4.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_4.a + t3_4.b) / 2)) = t1_4.a)
+                           ->  Sort
+                                 Sort Key: (((t3_4.a + t3_4.b) / 2))
+                                 ->  Seq Scan on prt1_e_p4 t3_4
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_4.a
+                                 ->  Seq Scan on prt1_p4 t1_4
+               ->  Sort
+                     Sort Key: t2_4.b
+                     ->  Seq Scan on prt2_p4 t2_4
+(83 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- MergeAppend on nullable column
 EXPLAIN (COSTS OFF)
@@ -999,8 +1546,18 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Merge Left Join
+               Merge Cond: (prt1_p0.a = b)
+               ->  Sort
+                     Sort Key: prt1_p0.a
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
+               ->  Sort
+                     Sort Key: b
+                     ->  Result
+                           One-Time Filter: false
          ->  Merge Left Join
                Merge Cond: (prt1_p1.a = b)
                ->  Sort
@@ -1021,21 +1578,26 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                      Sort Key: prt2_p2.b
                      ->  Seq Scan on prt2_p2
                            Filter: (b > 250)
-(23 rows)
+(33 rows)
 
 SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  b  
------+-----
-   0 |    
-  50 |    
- 100 |    
- 150 |    
- 200 |    
- 250 |    
- 300 | 300
- 350 |    
- 400 |    
-(9 rows)
+  a   |  b  
+------+-----
+ -250 |    
+ -200 |    
+ -150 |    
+ -100 |    
+  -50 |    
+    0 |    
+   50 |    
+  100 |    
+  150 |    
+  200 |    
+  250 |    
+  300 | 300
+  350 |    
+  400 |    
+(14 rows)
 
 -- merge join when expression with whole-row reference needs to be sorted;
 -- partitionwise join does not apply
@@ -1049,175 +1611,2412 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
          Sort Key: t1.a, ((((t1.*)::prt1))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
    ->  Sort
          Sort Key: t2.b, ((((t2.*)::prt2))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt2_p1 t2
-                     ->  Seq Scan on prt2_p2 t2_1
-                     ->  Seq Scan on prt2_p3 t2_2
-(16 rows)
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+(20 rows)
 
 SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.b ORDER BY t1.a;
- a  | b  
-----+----
-  0 |  0
-  6 |  6
- 12 | 12
- 18 | 18
- 24 | 24
-(5 rows)
+  a  |  b  
+-----+-----
+ -24 | -24
+ -18 | -18
+ -12 | -12
+  -6 |  -6
+   0 |   0
+   6 |   6
+  12 |  12
+  18 |  18
+  24 |  24
+(9 rows)
 
 RESET enable_hashjoin;
 RESET enable_nestloop;
---
--- partitioned by multiple columns
---
-CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
-CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
-ANALYZE prt1_m;
-CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
-CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
-ANALYZE prt2_m;
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-                                                             QUERY PLAN                                                             
-------------------------------------------------------------------------------------------------------------------------------------
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p4 t2_3
+               ->  Seq Scan on prt2_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_4
+                           Filter: (b = 0)
+(22 rows)
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p3 t1_2
+                           Filter: (b = 0)
+(21 rows)
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -150 | -0150 | 0 | -0150
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+(7 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (t1_3.a = b)
+         ->  Hash Right Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 |   | 
+ -200 | -0200 |   | 
+ -150 | -0150 | 0 | -0150
+ -100 | -0100 |   | 
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+(21 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -200 | 0 | -0200
+ -100 | 0 | -0100
+  -50 | 0 | -0050
+   50 | 0 | 0050
+  100 | 0 | 0100
+  200 | 0 | 0200
+  250 | 0 | 0250
+  350 | 0 | 0350
+  400 | 0 | 0400
+  500 | 0 | 0500
+  550 | 0 | 0550
+  650 | 0 | 0650
+  700 | 0 | 0700
+(14 rows)
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+                             QUERY PLAN                              
+---------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t3.c
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p0_a on prt1_p0 t3
+                           Index Cond: (a = t2.b)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t2.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_1.a = t2_1.b)
+                           ->  Seq Scan on prt1_p1 t3_1
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1 t2_1
+                                       Filter: (a = 0)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p2_a on prt1_p2 t3_2
+                           Index Cond: (a = t2_2.b)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t3_3
+                           Index Cond: (a = t2_3.b)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_4.a = t2_4.b)
+                           ->  Seq Scan on prt1_p4 t3_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t2_4
+                                       Filter: (a = 0)
+(47 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+  a   | a |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.b
+   ->  Hash Right Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p5 t2_5
+                           Filter: (a = 0)
+(24 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: (t1.a = t2.b)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.a, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
+                     Filter: (a = 0)
+(22 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
+               ->  Seq Scan on prt1_m_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p1
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
+               ->  Seq Scan on prt1_m_p2
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p2
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
+               ->  Seq Scan on prt1_m_p3
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p3
+                           Filter: (c = 0)
+(24 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+  a  | c |  b  | c 
+-----+---+-----+---
+   0 | 0 |   0 | 0
+  50 | 0 |     |  
+ 100 | 0 |     |  
+ 150 | 0 | 150 | 0
+ 200 | 0 |     |  
+ 250 | 0 |     |  
+ 300 | 0 | 300 | 0
+ 350 | 0 |     |  
+ 400 | 0 |     |  
+ 450 | 0 | 450 | 0
+ 500 | 0 |     |  
+ 550 | 0 |     |  
+     |   |  75 | 0
+     |   | 225 | 0
+     |   | 375 | 0
+     |   | 525 | 0
+(16 rows)
+
+--
+-- tests for list partitioned tables.
+--
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
+--
+-- list partitioned by expression
+--
+CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
+ANALYZE plt1_e;
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+                                           QUERY PLAN                                           
+------------------------------------------------------------------------------------------------
+ GroupAggregate
+   Group Key: t1.c, t2.c, t3.c
+   ->  Sort
+         Sort Key: t1.c, t3.c
+         ->  Append
+               ->  Hash Join
+                     Hash Cond: ((t1.c)::text = ltrim(t3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2.b = t1.b) AND ((t2.c)::text = (t1.c)::text))
+                           ->  Seq Scan on plt2_p4 t2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p4 t1
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p4 t3
+               ->  Hash Join
+                     Hash Cond: ((t1_1.c)::text = ltrim(t3_1.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t1_1.b = t2_1.b) AND ((t1_1.c)::text = (t2_1.c)::text))
+                           ->  Seq Scan on plt1_p1 t1_1
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p1 t2_1
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p1 t3_1
+               ->  Hash Join
+                     Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t1_2.b = t2_2.b) AND ((t1_2.c)::text = (t2_2.c)::text))
+                           ->  Seq Scan on plt1_p2 t1_2
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p2 t2_2
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p2 t3_2
+               ->  Hash Join
+                     Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2_3.b = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text))
+                           ->  Seq Scan on plt2_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p3 t3_3
+(41 rows)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p5 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_3
+                                 ->  Seq Scan on plt2_p3 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
  Sort
-   Sort Key: prt1_m_p1.a, prt2_m_p1.b
-   ->  Append
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
-               ->  Seq Scan on prt1_m_p1
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p1
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
-               ->  Seq Scan on prt1_m_p2
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p2
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
-               ->  Seq Scan on prt1_m_p3
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p3
-                           Filter: (c = 0)
-(24 rows)
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
 
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
-  50 | 0 |     |  
- 100 | 0 |     |  
- 150 | 0 | 150 | 0
- 200 | 0 |     |  
- 250 | 0 |     |  
- 300 | 0 | 300 | 0
- 350 | 0 |     |  
- 400 | 0 |     |  
- 450 | 0 | 450 | 0
- 500 | 0 |     |  
- 550 | 0 |     |  
-     |   |  75 | 0
-     |   | 225 | 0
-     |   | 375 | 0
-     |   | 525 | 0
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
 (16 rows)
 
---
--- tests for list partitioned tables.
---
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
---
--- list partitioned by expression
---
-CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1_e;
--- test partition matching with N-way join
+-- semi join
 EXPLAIN (COSTS OFF)
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                   QUERY PLAN                                   
---------------------------------------------------------------------------------
- GroupAggregate
-   Group Key: t1.c, t2.c, t3.c
-   ->  Sort
-         Sort Key: t1.c, t3.c
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
          ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.c = ltrim(t3.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1.b = t2.b) AND (t1.c = t2.c))
-                           ->  Seq Scan on plt1_p1 t1
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.c = ltrim(t3_1.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
-                           ->  Seq Scan on plt1_p2 t1_1
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.c = ltrim(t3_2.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
-                           ->  Seq Scan on plt1_p3 t1_2
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p3 t3_2
-(32 rows)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Seq Scan on plt2_p2 t2_2
+                                 ->  Seq Scan on plt2_p3 t2_3
+(22 rows)
 
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  75.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 123.0000000000000000 | 123.0000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 225.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 273.0000000000000000 | 273.0000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 375.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 423.0000000000000000 | 423.0000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 525.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 573.0000000000000000 | 573.0000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1242,22 +4041,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
 --------------------------------------------------
  Hash Left Join
    Hash Cond: (t2.b = a)
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t3.a = t2.b)
-               ->  Seq Scan on prt1_p1 t3
-               ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
-         ->  Hash Join
-               Hash Cond: (t3_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t3_1
-               ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
-         ->  Hash Join
-               Hash Cond: (t3_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t3_2
-               ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
+   ->  Hash Join
+         Hash Cond: (t3.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t3
+               ->  Seq Scan on prt1_p1 t3_1
+               ->  Seq Scan on prt1_p2 t3_2
+               ->  Seq Scan on prt1_p3 t3_3
+               ->  Seq Scan on prt1_p4 t3_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
    ->  Hash
          ->  Result
                One-Time Filter: false
@@ -1272,16 +4071,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 --
 -- tests for hash partitioned tables.
@@ -1357,41 +4162,9 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  273.0000000000000000 | 273.0000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
 (6 rows)
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
- Sort
-   Sort Key: t1.a
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
-               ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
-               ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
-                           Filter: (b = 0)
-(21 rows)
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
@@ -1406,26 +4179,24 @@ SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c
    Sort Key: t1.c
    ->  HashAggregate
          Group Key: t1.c, t2.c
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t2.c = t1.c)
-                     ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p1 t1
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on plt1_p4 t1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_1.c = t1_1.c)
-                     ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_p2 t1_1
+                           ->  Seq Scan on plt1_p1 t1_1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_2.c = t1_2.c)
-                     ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Seq Scan on plt1_p2 t1_2
                                  Filter: ((a % 25) = 0)
-(23 rows)
+                           ->  Seq Scan on plt1_p3 t1_3
+                                 Filter: ((a % 25) = 0)
+(21 rows)
 
 --
 -- multiple levels of partitioning
@@ -1827,64 +4598,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
  Hash Join
    Hash Cond: (t1.a = t2.a)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt4_n_p1 t2
                ->  Seq Scan on prt4_n_p2 t2_1
                ->  Seq Scan on prt4_n_p3 t2_2
-(11 rows)
+(13 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a and t1.a = t3.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                        QUERY PLAN                        
+----------------------------------------------------------
  Hash Join
-   Hash Cond: (t2.a = t1.a)
+   Hash Cond: (t3.b = t1.a)
    ->  Append
-         ->  Seq Scan on prt4_n_p1 t2
-         ->  Seq Scan on prt4_n_p2 t2_1
-         ->  Seq Scan on prt4_n_p3 t2_2
+         ->  Seq Scan on prt2_p0 t3
+         ->  Seq Scan on prt2_p1 t3_1
+         ->  Seq Scan on prt2_p2 t3_2
+         ->  Seq Scan on prt2_p3 t3_3
+         ->  Seq Scan on prt2_p4 t3_4
+         ->  Seq Scan on prt2_p5 t3_5
    ->  Hash
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.a = t3_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_2.b)
-                     ->  Seq Scan on prt1_p3 t1_2
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3 t3_2
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt4_n_p1 t2
+                           ->  Seq Scan on prt4_n_p2 t2_1
+                           ->  Seq Scan on prt4_n_p3 t2_2
 (23 rows)
 
 -- partitionwise join can not be applied if there are no equi-join conditions
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (t1.a < b)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (t1.a < b)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (t1.a < b)
-(12 rows)
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+(16 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partitionwise join
@@ -1970,16 +4747,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
-                     ->  Seq Scan on plt1_p1 t3
-                     ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3
+                     ->  Seq Scan on plt1_p1 t3_1
+                     ->  Seq Scan on plt1_p2 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partitionwise join can not be applied for a join between list and range
 -- partitioned table
@@ -1990,14 +4768,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
  Hash Full Join
    Hash Cond: ((t2.c)::text = (t1.c)::text)
    ->  Append
-         ->  Seq Scan on prt1_p1 t2
-         ->  Seq Scan on prt1_p2 t2_1
-         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p0 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p2 t2_2
+         ->  Seq Scan on prt1_p3 t2_3
+         ->  Seq Scan on prt1_p4 t2_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt1_n_p1 t1
                ->  Seq Scan on prt1_n_p2 t1_1
-(10 rows)
+(12 rows)
 
 -- partitionwise join can not be applied if only one of joining table has
 -- default partition
@@ -2013,16 +4793,23 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
    ->  Hash Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index c1c9859651..0e884835fb 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -10,25 +10,39 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -67,11 +81,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -93,20 +115,30 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 
 EXPLAIN (COSTS OFF)
@@ -169,6 +201,128 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -193,28 +347,79 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
 -- test partition matching with N-way join
@@ -222,6 +427,175 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
@@ -267,27 +641,18 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
 ALTER TABLE plt2 DETACH PARTITION plt2_p3;
 ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
 ANALYZE plt2;
-
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.a % 25 = 0 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
 --
 -- multiple levels of partitioning
 --
-- 
2.16.4

#54Dmitry Dolgov
9erthalion6@gmail.com
In reply to: Dmitry Dolgov (#53)
4 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Thu, Oct 25, 2018 at 11:19 PM Dmitry Dolgov <9erthalion6@gmail.com> wrote:

Since most of my complaints about the patch were related to code readability,
I modified it a bit to show more clearly what I have in mind. I haven't changed
anything about the functionality, just adjusted some parts of it:

* removed some unused arguments (looks like they were added for consistency in
all higher level branches, but not all of them were actually in use)

* replaced representation for partition mapping (instead of int arrays there is
a structure, that allows to replace 0/1 with more meaningful from/to)

* tried to make naming a bit more consistent - so, if a function doesn't
explicitely say anything about outer/inner, we have partmap1/partmap2,
otherwise outermap/innermap. I don't really like this style with
partmap1/partmap2 or index1/index2, but it seems consistent with the rest of
the code in partbounds.c

* removed some state representation flags, e.g. merged - instead, if there is a
situation when we can't merge, functions will return NULL right away

* removed questionable debugging statement

I've noticed, that this patch set is outdated, so here is the rebased version.

Attachments:

0001-Hash-partition-bound-equality-refactoring-v13.patchapplication/octet-stream; name=0001-Hash-partition-bound-equality-refactoring-v13.patchDownload
From c47036b5aa5e46307c2e0a3db94bec86431f0f38 Mon Sep 17 00:00:00 2001
From: erthalion <9erthalion6@gmail.com>
Date: Tue, 11 Sep 2018 21:08:07 +0200
Subject: [PATCH 1/5] Hash partition bound equality refactoring

Separate the code to check whether two given hash bounds are equal into a
separate function, so that it can be called from multiple places. Right now
it's only caller is partition_bounds_equal() but later we will use it for
merging partition bounds.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/partitioning/partbounds.c | 95 ++++++++++++++++++++++-------------
 1 file changed, 61 insertions(+), 34 deletions(-)

diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index eeaab2f4c9..0af3372cdf 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -104,6 +104,9 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 						 Expr **keyCol,
 						 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static bool partition_hbounds_equal(PartitionBoundInfo b1,
+									PartitionBoundInfo b2);
+
 
 /*
  * get_qual_from_partbound
@@ -652,6 +655,63 @@ create_range_bounds(PartitionBoundSpec **boundspecs, int nparts,
 	return boundinfo;
 }
 
+/*
+ * Are two hash partition bound collections logically equal?
+ *
+ * Hash partition bounds store modulus and remainder in datums array which are
+ * always integers irrespective of the number of partition keys and their data
+ * types. Hence we can compare the hash bound collection without any partition
+ * key specific information. Separating this logic in a function which does not
+ * require partition key specific information allows it be called from places
+ * where the partition key specific information is not completely available.
+ */
+static bool
+partition_hbounds_equal(PartitionBoundInfo b1, PartitionBoundInfo b2)
+{
+	int			greatest_modulus = get_hash_partition_greatest_modulus(b1);
+	int			i;
+
+	Assert(b1->strategy == PARTITION_STRATEGY_HASH &&
+		   b2->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * If two hash partitioned tables have different greatest moduli,
+	 * their partition schemes don't match.  For hash partitioned table,
+	 * the greatest modulus is given by the last datum and number of
+	 * partitions is given by ndatums.
+	 */
+	if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
+		return false;
+
+	/*
+	 * We arrange the partitions in the ascending order of their modulus and
+	 * remainders.  Also every modulus is factor of next larger modulus.
+	 * Therefore we can safely store index of a given partition in indexes
+	 * array at remainder of that partition.  Also entries at (remainder + N *
+	 * modulus) positions in indexes array are all same for (modulus,
+	 * remainder) specification for any partition.  Thus datums array from both
+	 * the given bounds are same, if and only if their indexes array will be
+	 * same.  So, it suffices to compare indexes array.
+	 */
+	for (i = 0; i < greatest_modulus; i++)
+		if (b1->indexes[i] != b2->indexes[i])
+			return false;
+
+#ifdef USE_ASSERT_CHECKING
+
+	/*
+	 * Nonetheless make sure that the bounds are indeed same when the indexes
+	 * match.  Hash partition bound stores modulus and remainder at
+	 * b1->datums[i][0] and b1->datums[i][1] position respectively.
+	 */
+	for (i = 0; i < b1->ndatums; i++)
+		Assert((b1->datums[i][0] == b2->datums[i][0] &&
+				b1->datums[i][1] == b2->datums[i][1]));
+#endif
+
+	return true;
+}
+
 /*
  * Are two partition bound collections logically equal?
  *
@@ -680,41 +740,8 @@ partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
 
 	if (b1->strategy == PARTITION_STRATEGY_HASH)
 	{
-		int			greatest_modulus = get_hash_partition_greatest_modulus(b1);
-
-		/*
-		 * If two hash partitioned tables have different greatest moduli,
-		 * their partition schemes don't match.
-		 */
-		if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
+		if (!partition_hbounds_equal(b1, b2))
 			return false;
-
-		/*
-		 * We arrange the partitions in the ascending order of their moduli
-		 * and remainders.  Also every modulus is factor of next larger
-		 * modulus.  Therefore we can safely store index of a given partition
-		 * in indexes array at remainder of that partition.  Also entries at
-		 * (remainder + N * modulus) positions in indexes array are all same
-		 * for (modulus, remainder) specification for any partition.  Thus
-		 * datums array from both the given bounds are same, if and only if
-		 * their indexes array will be same.  So, it suffices to compare
-		 * indexes array.
-		 */
-		for (i = 0; i < greatest_modulus; i++)
-			if (b1->indexes[i] != b2->indexes[i])
-				return false;
-
-#ifdef USE_ASSERT_CHECKING
-
-		/*
-		 * Nonetheless make sure that the bounds are indeed same when the
-		 * indexes match.  Hash partition bound stores modulus and remainder
-		 * at b1->datums[i][0] and b1->datums[i][1] position respectively.
-		 */
-		for (i = 0; i < b1->ndatums; i++)
-			Assert((b1->datums[i][0] == b2->datums[i][0] &&
-					b1->datums[i][1] == b2->datums[i][1]));
-#endif
 	}
 	else
 	{
-- 
2.16.4

0002-Targetlist-of-a-child-join-is-produced-by-translating-v13.patchapplication/octet-stream; name=0002-Targetlist-of-a-child-join-is-produced-by-translating-v13.patchDownload
From 77fa061f1d3d89efc8abb0ec91bfd7611a39641b Mon Sep 17 00:00:00 2001
From: erthalion <9erthalion6@gmail.com>
Date: Tue, 11 Sep 2018 21:09:44 +0200
Subject: [PATCH 2/5] Targetlist of a child-join is produced by translating
 that of parent join

The next patch adds more general partition matching algorithm for
partition-wise join. With that change, the first pair of joining
relations for the parent joinrel may not produce a partition-wise join
because of the restrictions in partition_bounds_merge(), to be added
in the next patch.  Hence the first pair of joining relations, which
is used to build the child join and presented to
build_child_join_rel(), for the child joinrel does not correspond to
the first pair of joining relations for the parent joinrel. The
targetlist built using different pairs have the targetlist entries
arranged in different order. An appendrel expects that all its
children have their targetlists ordered in the same fashion.  Hence
translate the parent's targetlist so that parent and child joinrels
have their targetlists in sync.

Basic partition-wise join commit
f49842d1ee31b976c681322f76025d7732e860f3 modified
build_joinrel_tlist() to build targetlist for child-join. With the
above changes we don't need it anymore.  Similarly, the same commit
added code to set_append_rel_size() to compute attr_needed for a
child-join relation. That change too is not needed with the above
change. This commit reverts those two changes.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/optimizer/util/relnode.c | 20 ++++++++++++++++++++
 1 file changed, 20 insertions(+)

diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 39f5729b91..2f1137c13d 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -802,6 +802,19 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 
 	appinfos = find_appinfos_by_relids(root, joinrel->relids, &nappinfos);
 
+	/*
+	 * The first pair of joining relations for the parent joinrel may not
+	 * produce a partition-wise join because of the restrictions in
+	 * partition_bounds_merge(). Hence the first pair of joining relations,
+	 * which is used to build the child join and presented here, for the child
+	 * joinrel does not correspond to the first pair of joining relations for
+	 * the parent joinrel. The targetlist built using different pairs have the
+	 * targetlist nodes arranged in different order. An appendrel expects that
+	 * all its children have their targetlists ordered in the same fashion.
+	 * Hence translate the parent's targetlist so that parent and child
+	 * joinrels have their targetlists in sync.
+	 */
+	joinrel->reltarget = copy_pathtarget(parent_joinrel->reltarget);
 	/* Set up reltarget struct */
 	build_child_join_reltarget(root, parent_joinrel, joinrel,
 							   nappinfos, appinfos);
@@ -907,6 +920,12 @@ build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 	Relids		relids = joinrel->relids;
 	ListCell   *vars;
 
+	/*
+	 * We only see parent joins. Targetlist of a child-join is computed by
+	 * translating corresponding parent join's targetlist.
+	 */
+	Assert(joinrel->reloptkind == RELOPT_JOINREL);
+
 	foreach(vars, input_rel->reltarget->exprs)
 	{
 		Var		   *var = (Var *) lfirst(vars);
@@ -934,6 +953,7 @@ build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 
 		/* Is it still needed above this joinrel? */
 		ndx = var->varattno - baserel->min_attr;
+
 		if (bms_nonempty_difference(baserel->attr_needed[ndx], relids))
 		{
 			/* Yup, add it to the output */
-- 
2.16.4

0003-Partition-wise-join-for-1-1-1-0-0-1-partition-matching-v13.patchapplication/octet-stream; name=0003-Partition-wise-join-for-1-1-1-0-0-1-partition-matching-v13.patchDownload
From 7428090314f1e25072d2f6841b83f513e4e611f2 Mon Sep 17 00:00:00 2001
From: erthalion <9erthalion6@gmail.com>
Date: Tue, 11 Sep 2018 21:10:46 +0200
Subject: [PATCH 3/5] Partition-wise join for 1:1, 1:0, 0:1 partition matching

Earlier version of partition-wise join implementation allowed
partition-wise join between two relations with exactly same partition
bounds. This commit allows partition-wise join to be applied under
following conditions

1. the partition bounds of joining relations are such that rows from
given partition on one side join can join with rows from maximum one
partition on the other side i.e. bounds of a given partition on one
side match/overlap with those of maximum one partition on the other
side. If the mapping happens to be m:n where m > 1 or n > 1, we have
to gang multiple partition relations together into a single relation.
This means that we have to add simple relations during join
processing, something which is not supported right now.  ALso, in such
a case, different pairs of joining relations can produce different
partition bounds for the same join relation, which again is not
supported right now.

2. For every partition on outer side that can contribute to the result
of an OUTER join, there exists at least one (taken along with item 1,
it means exactly one)  matching partition on the inner side. To
support partition-wise join when the inner matching partition doesn't
exist, we have to add a dummy base relation corresponding to the
non-existent inner partition. We don't have support to add base
relations during join processing.

3. For hash partitioned tables however, we support partition-wise join
only when the bounds exactly match. For hash partitioning it's unusual
to have missing partitions and hence generic partition matching is not
required.

With advanced partition matching, we won't be able to apply
partition-wise join when there are missing matching partitions. So for
a given N-way join, some sub-joins may not be partitioned, while the
overall N-way join is partitioned. We can not try partition-wise join
when one or both of the joining relations are not partitioned in one
of the pairs of joining relation. Skip partition-wise join for that
pair.

An example is A IJ B LJ C where B has an extra partition compared to A
and C. Join B LJ C requires dummy relation to join the extra partition
from B to a missing partition from C, and hence can not use
partition-wise join right now and hence BC is not partitioned.  The
extra partition in B gets eliminated in A IJ B and thus it can use
partition-wise join and is partitioned. Hence (AB)C can use
partition-wise join but not A(BC).

Not every pair of joining relation (including the one presented to
build_joinrel_partition_info()) for the same joinrel can use
partition-wise join or has both the relations partitioned. Hence we
calculate the partition bounds for the join relation in
try_partition_wise_join() instead of doing it here.

The partition bounds produced for the first pair that can use
partition-wise are saved in the RelOptInfo of the joinrel. Partition
bounds produced for subsequent pairs should match that produced by the
first pair.

Support for default partition in list partitioned tables
========================================================

When a list value is present in one of the joining relations and not
the other, and the other relation has default partition, match (join)
the partition containing that list value with the default partition.
If there are multiple matches, we can not proceed with the
partition-wise join similar to the case of matching non-default
partition. If the default partition happens to be on the outer side of
the join, the resulting join partition acts as a default partition as
it will contain all the values from the default partition. If
the partition containing the list value happens to be on the outer
side of the join, the resulting join partition is associated with the
list value, since no other partition key value from the default
partition makes it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join.  Such a
default partition may get joined to a partition from the inner side,
if inner side has a list value that is not present in the outer side.
But if such a join partner does not exist, we won't be able to use
partition-wise join since there is no (dummy) relation on the inner
side to join to.

Support for matching default partition in range partitioned tables
==================================================================

When a range is present in one of the joining relations and not the
other, and the other relation has default partition, match (join) the
partition corresponding range with the default partition. If two
ranges overlap but their ranges don't match exactly, the
non-overlapping portion from one range may join the previous
(non-overlapping portion near the lower bound, if exists), next
(non-overlapping portion near the upper bound, if exists) ranges from
the other relation or default partition from the other relation. This
causes multiple partitions on the other side to be joined with a
single partition on the first side; a case we don't support. Any range
from one side which doesn't overlap with any range on the other side,
may find its join partner in the default partition and the
corresponding range partition will need to be joined with the default
partition.  If the default partition happens to be on the outer side
of the join, the resulting join partition acts as a default partition
as it will contain all the values from the default partition. If the
non-partition corresponding to the range happens to be on the outer
side of the join, the resulting join partition is associated with that
range, since partition key values from the default partition outside
that range won't make it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join.  Such a
default partition may get joined to a non-default partition from the
inner side, if inner side has a range missing in the outer side.  But
if such a join partner does not exist, we won't be able to use
partition-wise join since there is no (dummy) relation on the inner
side to join to.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/optimizer/path/joinrels.c |   99 +-
 src/backend/optimizer/util/relnode.c  |   33 +-
 src/backend/partitioning/partbounds.c | 1724 +++++++++++++++++++++++++++++++++
 src/include/partitioning/partbounds.h |    6 +
 4 files changed, 1817 insertions(+), 45 deletions(-)

diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index d3d21fed5d..875b39e334 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1312,25 +1312,31 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 					   RelOptInfo *joinrel, SpecialJoinInfo *parent_sjinfo,
 					   List *parent_restrictlist)
 {
-	int			nparts;
 	int			cnt_parts;
+	PartitionScheme part_scheme;
+	PartitionBoundInfo join_boundinfo;
+	List	   *parts1;
+	List	   *parts2;
+	ListCell   *lc1;
+	ListCell   *lc2;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1343,32 +1349,67 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	 */
 	Assert(joinrel->part_scheme == rel1->part_scheme &&
 		   joinrel->part_scheme == rel2->part_scheme);
+	part_scheme = joinrel->part_scheme;
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * Get the list of matching partitions to be joined along with the
+	 * partition bounds of the join relation. Because of the restrictions
+	 * imposed by partition matching algorithm, not every pair of joining
+	 * relations for this join will be able to use partition-wise join. But all
+	 * those pairs which can use partition-wise join will produce the same
+	 * partition bounds for the join relation.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	join_boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											part_scheme->partsupfunc,
+											part_scheme->partcollation,
+											rel1->boundinfo, rel1->nparts,
+											rel2->boundinfo, rel2->nparts,
+											parent_sjinfo->jointype,
+											&parts1, &parts2);
+
+	if (join_boundinfo == NULL)
+		return;
+
+	if (joinrel->boundinfo == NULL)
+	{
+		Assert(joinrel->nparts == 0 && joinrel->part_rels == NULL);
+		joinrel->boundinfo = join_boundinfo;
+		joinrel->nparts = list_length(parts1);
+		Assert(joinrel->nparts == list_length(parts2));
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) *
+									joinrel->nparts);
+	}
+	else
+	{
+		Assert(partition_bounds_equal(part_scheme->partnatts,
+									  part_scheme->parttyplen,
+									  part_scheme->parttypbyval,
+									  join_boundinfo, joinrel->boundinfo));
+		/*
+		 * Every pair of joining relations should result in the same number
+		 * of child-joins.
+		 */
+		Assert(joinrel->nparts == list_length(parts1));
+		Assert(joinrel->nparts == list_length(parts2));
+		Assert(joinrel->part_rels);
+	}
 
-	nparts = joinrel->nparts;
+	elog(DEBUG3, "join between relations %s and %s is considered for partition-wise join.",
+		 bmsToString(rel1->relids), bmsToString(rel2->relids));
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	cnt_parts = 0;
+	forboth(lc1, parts1, lc2, parts2)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
+		int			part1 = lfirst_int(lc1);
+		int			part2 = lfirst_int(lc2);
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1376,6 +1417,10 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		Assert(part1 >= 0 && part2 >= 0);
+		child_rel1 = rel1->part_rels[part1];
+		child_rel2 = rel2->part_rels[part2];
+
 		/* We should never try to join two overlapping sets of rels. */
 		Assert(!bms_overlap(child_rel1->relids, child_rel2->relids));
 		child_joinrelids = bms_union(child_rel1->relids, child_rel2->relids);
@@ -1409,12 +1454,24 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 			joinrel->part_rels[cnt_parts] = child_joinrel;
 		}
 
+		/*
+		 * For every pair of joining relations, the set of matching partitions
+		 * would change. However, the base relation partitions constituting
+		 * the given child should remain same for all the joining pairs. Since
+		 * the order in which children are stored in the array of child-joins,
+		 * depends upon partition bounds of the join, which are same for all
+		 * the joining pairs, every joining pair yields the child-joins in the
+		 * same order.
+		 */
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
 
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
+		cnt_parts++;
 	}
+
+	Assert(cnt_parts == joinrel->nparts);
 }
 
 /*
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 2f1137c13d..4f15cd8e8b 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1627,7 +1627,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1640,24 +1640,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1669,17 +1651,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 0af3372cdf..491ebedc4c 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -106,6 +106,63 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 static List *get_range_nulltest(PartitionKey key);
 static bool partition_hbounds_equal(PartitionBoundInfo b1,
 									PartitionBoundInfo b2);
+static PartitionBoundInfo partition_range_bounds_merge(int partnatts,
+							 FmgrInfo *supfuncs, Oid *collations,
+							 PartitionBoundInfo boundinfo1, int nparts1,
+							 PartitionBoundInfo boundinfo2, int nparts2,
+							 JoinType jointype, List **parts1, List **parts2);
+static PartitionBoundInfo partition_list_bounds_merge(int partnatts,
+							FmgrInfo *partsupfunc, Oid *collations,
+							PartitionBoundInfo boundinfo1, int nparts1,
+							PartitionBoundInfo boundinfo2, int nparts2,
+							JoinType jointype, List **parts1, List **parts2);
+static PartitionBoundInfo partition_hash_bounds_merge(int partnatts,
+							FmgrInfo *partsupfunc,
+							Oid *partcollation, PartitionBoundInfo left_bi,
+							int left_nparts, PartitionBoundInfo right_bi,
+							int right_nparts, JoinType jointype, List **left_parts,
+							List **right_parts);
+static void generate_matching_part_pairs(int *mergemap1, int npart1,
+							 int *mergemap2, int nparts2, JoinType jointype,
+							 int nparts, List **parts1, List **parts2);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int map_and_merge_partitions(int *partmap1, int *mergemap1, int index1,
+						 int *partmap2, int *mergemap2, int index2,
+						 int *next_index);
+static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *collations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2);
+static bool partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
+						   Oid *collations, PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, int *ub_cmpval,
+						   int *lb_cmpval);
+static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes);
+static bool merge_default_partitions(PartitionBoundInfo outer_bi,
+						 int *outer_pmap,
+						 int *outer_mmap, PartitionBoundInfo inner_bi,
+						 int *inner_pmap, int *inner_mmap, JoinType jointype,
+						 int *next_index, int *default_index);
+static bool merge_null_partitions(PartitionBoundInfo outer_bi, int *outer_pmap,
+					  int *outer_mmap, PartitionBoundInfo inner_bi,
+					  int *inner_pmap, int *inner_mmap, JoinType jointype,
+					  int *next_index, int *null_index, int *default_index);
+static bool handle_missing_partition(int *missing_side_pmap,
+						 int *missing_side_mmap,
+						 int missing_side_default, int *other_pmap,
+						 int *other_mmap, int other_part,
+						 bool missing_side_outer,
+						 bool missing_side_inner,
+						 int *next_index, int *default_index,
+						 int *merged_index);
 
 
 /*
@@ -2941,3 +2998,1670 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * The function produces the partition bounds for a join between two relations
+ * whose partition bounds are given. The function also returns two lists of
+ * partition indexes one for each of the joining relations. Both the lists
+ * contain the same number of elements. The partition indexes at the same
+ * positions in the lists indicate the pair partitions, one from each side, to
+ * be joined and the position itself corresponds to the index of partition
+ * produced by that child-join in the partitioned join.
+ *
+ * The function returns NULL if we can not find the matching pair of
+ * partitions. This happens if 1. multiple partitions on one side match with
+ * one partition on the other side. 2. a given partition on the outer side
+ * doesn't have a matching partition on the inner side. We can not support the
+ * first case since we don't have a way to represent multiple partitions as a
+ * single relation (RelOptInfo) and then perform join using the ganged
+ * relation. We can not support the second case since the missing inner
+ * partition needs to be represented as an empty relation and we don't have a
+ * way to introduce empty relation during join planning after creating paths
+ * for all the base relations.
+ */
+extern PartitionBoundInfo
+partition_bounds_merge(int partnatts, FmgrInfo *partsupfunc,
+					   Oid *partcollation,
+					   PartitionBoundInfo outer_bi, int outer_nparts,
+					   PartitionBoundInfo inner_bi, int inner_nparts,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds;
+	char		strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_bi->strategy != inner_bi->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	strategy = outer_bi->strategy;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partnatts, partsupfunc,
+														partcollation,
+														outer_bi, outer_nparts,
+														inner_bi, inner_nparts,
+														jointype, outer_parts,
+														inner_parts);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(partnatts,
+														 partsupfunc,
+														 partcollation,
+														 outer_bi,
+														 outer_nparts,
+														 inner_bi,
+														 inner_nparts,
+														 jointype,
+														 outer_parts,
+														 inner_parts);
+			break;
+
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = partition_hash_bounds_merge(partnatts,
+														partsupfunc,
+														partcollation,
+														outer_bi, outer_nparts,
+														inner_bi, inner_nparts,
+														jointype, outer_parts,
+														inner_parts);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	Assert((*outer_parts == NIL || *inner_parts != NIL) &&
+		   (*inner_parts == NIL || *outer_parts != NIL));
+
+	return merged_bounds;
+}
+
+/*
+ * partition_get_range_bounds
+ *
+ * Given the index of lower bound in datums array, return lower and upper
+ * bounds and the index of the partition with that lower bound.
+ */
+static int
+partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
+						   PartitionRangeBound *lower,
+						   PartitionRangeBound *upper)
+{
+	int			part_index;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The lower bound should correspond to a valid partition. */
+	part_index = bi->indexes[lb_index + 1];
+	Assert(part_index >= 0);
+
+	lower->kind = bi->kind[lb_index];
+	lower->datums = bi->datums[lb_index];
+	lower->lower = true;
+	upper->kind = bi->kind[lb_index + 1];
+	upper->datums = bi->datums[lb_index + 1];
+	upper->lower = false;
+
+	return part_index;
+}
+
+/*
+ * partition_range_get_next_lb_index
+ *
+ * Given the index of lower bound in datums array return the
+ * index of lower bound of the next partition. When the given index corresponds
+ * to the last partition, return number of datums (ndatums).
+ */
+static int
+partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
+{
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The partition index corresponding to the upper bound should be valid. */
+	Assert(bi->indexes[lb_index + 1] >= 0);
+
+	/*
+	 * If there are no bounds left beyond the upper bound, we have reached the
+	 * last partition.
+	 */
+	if (lb_index + 2 < bi->ndatums)
+	{
+		/*
+		 * If the bound next to the upper bound corresponds to no partition,
+		 * that's the next lower bound of the next partition. Otherwise, the
+		 * current upper bound is the lower bound of the next partition.
+		 */
+		if (bi->indexes[lb_index + 2] < 0)
+			return lb_index + 2;
+		else
+			return lb_index + 1;
+	}
+	else
+		return bi->ndatums;
+}
+
+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *partcollations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2)
+{
+	return partition_rbound_cmp(partnatts, partsupfunc, partcollations,
+								bound1->datums, bound1->kind, bound1->lower,
+								bound2);
+}
+
+/*
+ * partition_range_cmp
+ *
+ * Compare the bounds of two range partitions. Set ub_cmpval <, = or > 0, if the
+ * first partition's upper bound is lower than, equal to or higher than the
+ * second partition's upper bound resp. Similarly set lb_cmpval <, =  or > 0,
+ * if the first partition's lower bound is lower than, equal to or higher than
+ * the second partition's lower bound resp.
+ *
+ * Return true, if the ranges overlap, otherwise return false.
+ */
+static bool
+partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
+					PartitionRangeBound *lower_bound1,
+					PartitionRangeBound *upper_bound1,
+					PartitionRangeBound *lower_bound2,
+					PartitionRangeBound *upper_bound2, int *ub_cmpval,
+					int *lb_cmpval)
+{
+	bool		overlap;
+
+	/*
+	 * Compare upper bound of the first partition with the lower bound of the
+	 * second and vice-versa. If lower bound is higher than the upper bound,
+	 * the partitions are not overlapping. All other cases indicate overlapping
+	 * partitions.
+	 */
+	if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+								  lower_bound1, upper_bound2) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = 1;
+		*lb_cmpval = 1;
+	}
+	else if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+									   lower_bound2, upper_bound1) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = -1;
+		*lb_cmpval = -1;
+	}
+	else
+	{
+		overlap = true;
+		*ub_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, upper_bound1,
+											   upper_bound2);
+		*lb_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, lower_bound1,
+											   lower_bound2);
+	}
+
+	return overlap;
+}
+
+/*
+ * partition_range_merge
+ *
+ * Merge the partition bounds of given two partitions such that the join
+ * between the given two partitions fits merged bounds.
+ *
+ * "merged_upper" will be set to one of the given upper bounds and
+ * "merged_lower" will be set to one of the given lower bounds.
+ */
+static void
+partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
+					  Oid *partcollations, JoinType jointype,
+					  PartitionRangeBound *left_lb,
+					  PartitionRangeBound *left_ub,
+					  PartitionRangeBound *right_lb,
+					  PartitionRangeBound *right_ub,
+					  PartitionRangeBound **merged_lb,
+					  PartitionRangeBound **merged_ub)
+{
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = left_ub;
+			*merged_lb = left_lb;
+			break;
+
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) < 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) > 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		case JOIN_FULL:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) > 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) < 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+
+	return;
+}
+
+/*
+ * Add the lower bound of the next range to the list of bounds, if the lower
+ * bound is higher or equal to the previous upper bound. If successful return
+ * true, otherwise false.
+ */
+static bool
+partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
+							  Oid *partcollations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		prev_ub.datums = llast(*merged_datums);
+		prev_ub.kind = llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  next_lb_datums, next_lb_kind, false,
+									  &prev_ub);
+	}
+
+	/*
+	 * The lower bound is lower than the last upper bound, thus does not fit
+	 * the bounds created so far and hence can not be merged with the existing
+	 * bounds.
+	 */
+	if (cmpval < 0)
+		return false;
+
+	/*
+	 * Add bounds of the new merged partition. If the next lower bound is
+	 * higher than the last upper bound, add new range with index
+	 * corresponding to the lower bound as -1. If the merged lower bound
+	 * is same as the last merged upper bound, the last upper bound will be
+	 * reused as the lower bound of the next range.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, next_lb_datums);
+		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	return true;
+}
+
+/*
+ * handle_missing_partition
+ *
+ * If a range appears in one of the joining relations but not the other, a row
+ * in the corresponding partition will not have any join partner in the other
+ * relation, unless the other relation has a default partition. If a given list
+ * value is present in one joining relation but not the other, the default
+ * partition on the other side may contain that value.
+ *
+ * In both these cases, such an extra partition forms a joining pair with the
+ * default partition, if any,  on the other side.
+ *
+ * If the default partition happens to be on the outer side of the join, the
+ * resultant partition will act as the default partition of the join relation.
+ * Otherwise the resultant partition will be associated with the range.
+ *
+ * When the default partition is not present in the other relation, the rows in
+ * the extra partition will be included in the bounds of the join result, if it
+ * appears on the outer side of the join, since all rows from the outer side
+ * are included in the join result.
+ *
+ * This function handles all these cases.
+ *
+ * missing_side_pmap, missing_side_mmap and missing_side_default are the
+ * partition map, merge map (See partition_range/list_bounds_merge() for
+ * details) and the index of default partition respectively corresponding the
+ * side with missing partition.
+ *
+ * other_pmap, other_mmap and other_part are the partition map, merge map (See
+ * partition_range/list_bounds_merge() for details) and the index of extra
+ * partition respectively corresponding to the side with the extra partition.
+ *
+ * It returns true if the matching succeeds, otherwise returns false.
+ */
+static bool
+handle_missing_partition(int *missing_side_pmap, int *missing_side_mmap,
+						 int missing_side_default, int *other_pmap,
+						 int *other_mmap, int other_part,
+						 bool missing_side_outer,
+						 bool missing_side_inner,
+						 int *next_index, int *default_index,
+						 int *merged_index)
+{
+	bool		missing_has_default = (missing_side_default != -1);
+
+	if (missing_has_default)
+	{
+		*merged_index = map_and_merge_partitions(missing_side_pmap,
+												 missing_side_mmap,
+												 missing_side_default,
+												 other_pmap, other_mmap,
+												 other_part,
+												 next_index);
+		if (*merged_index < 0)
+			return false;
+
+		if (missing_side_outer)
+		{
+			/*
+			 * Default partition on the outer side forms the default
+			 * partition of the join result.
+			 */
+			if (*default_index < 0)
+				*default_index = *merged_index;
+			else if(*default_index != *merged_index)
+			{
+				/*
+				 * Ended up with default partition on the outer side
+				 * being joined with multiple partitions on the inner
+				 * side. We don't support this case.
+				 */
+				return false;
+			}
+
+			/*
+			 * Since the merged partition acts as a default partition, it
+			 * doesn't need a separate index.
+			 */
+			*merged_index = -1;
+		}
+	}
+	else if (missing_side_inner)
+	{
+		/*
+		 * If this partition has already been mapped (say because we
+		 * found an overlapping range earlier), we know where does it
+		 * fit in the join result. Nothing to do in that case. Else
+		 * create a new merged partition.
+		 */
+		if (other_mmap[other_part] < 0)
+		{
+			other_mmap[other_part] = *next_index;
+			*next_index = *next_index + 1;
+			*merged_index = other_mmap[other_part];
+		}
+	}
+
+	return true;
+}
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations, PartitionBoundInfo outer_bi,
+							 int outer_nparts, PartitionBoundInfo inner_bi,
+							 int inner_nparts, JoinType jointype,
+							 List **outer_parts, List **inner_parts)
+{
+	int		   *outer_pmap;
+	int		   *outer_mmap;
+	int		   *inner_pmap;
+	int		   *inner_mmap;
+	int			cnt1;
+	int			cnt2;
+	int			outer_part;
+	int			inner_part;
+	PartitionBoundInfo merged_bounds = NULL;
+	bool		merged = true;
+	int			outer_lb_index;
+	int			inner_lb_index;
+	int			next_index;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	List	   *merged_kinds = NIL;
+	int			inner_default = inner_bi->default_index;
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+
+	/*
+	 * Allocate and initialize partition maps. We maintain four maps, two maps
+	 * for each joining relation. pmap[i] gives the partition from the other
+	 * relation which would join with ith partition of the given relation.
+	 * Partition i from the given relation will join with partition pmap[i]
+	 * from the other relation to produce partition mmap[i] of the join (merged
+	 * partition).
+	 *
+	 * pmap[i] = -1 indicates that ith partition of a given relation does not
+	 * have a matching partition from the other relation.
+	 *
+	 * mmap[i] = -1 indicates that ith partition of a given relation does not
+	 * contribute to the join result. That can happen only when the given
+	 * relation is the inner relation and it doesn't have a matching partition
+	 * from the outer relation, hence pmap[i] should be -1.
+	 *
+	 * In case of an outer join, every partition of the outer join will appear
+	 * in the join result, and thus has mmap[i] set for all i. But it's not
+	 * necessary that every partition on the outer side will have a matching
+	 * partition on the inner side. In such a case, we end up with pmap[i] = -1
+	 * and mmap[i] != -1.
+	 */
+	outer_pmap = (int *) palloc(sizeof(int) * outer_nparts);
+	outer_mmap = (int *) palloc(sizeof(int) * outer_nparts);
+	inner_pmap = (int *) palloc(sizeof(int) * inner_nparts);
+	inner_mmap = (int *) palloc(sizeof(int) * inner_nparts);
+
+	for (cnt1 = 0; cnt1 < outer_nparts; cnt1++)
+	{
+		outer_pmap[cnt1] = -1;
+		outer_mmap[cnt1] = -1;
+	}
+	for (cnt2 = 0; cnt2 < inner_nparts; cnt2++)
+	{
+		inner_pmap[cnt2] = -1;
+		inner_mmap[cnt2] = -1;
+	}
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = 0;
+	inner_lb_index = 0;
+	next_index = 0;
+	while (outer_lb_index < outer_bi->ndatums ||
+		   inner_lb_index < inner_bi->ndatums)
+	{
+		PartitionRangeBound outer_lb;
+		PartitionRangeBound outer_ub;
+		PartitionRangeBound inner_lb;
+		PartitionRangeBound inner_ub;
+		PartitionRangeBound *merged_lb = NULL;
+		PartitionRangeBound *merged_ub = NULL;
+		int			merged_index = -1;
+		bool		overlap;
+		bool		finished_outer = false;
+		bool		finished_inner = false;
+
+		/* Result of bounds comparison per partition_rbound_cmp(). */
+		int			ub_cmpval;	/* Upper bounds comparison result. */
+		int			lb_cmpval;	/* Lower bounds comparison result. */
+
+		/* Get the range bounds of the next pair of partitions. */
+		if (outer_lb_index < outer_bi->ndatums)
+			outer_part = partition_get_range_bounds(outer_bi, outer_lb_index,
+												&outer_lb, &outer_ub);
+		else
+			finished_outer = true;
+
+		if (inner_lb_index < inner_bi->ndatums)
+			inner_part = partition_get_range_bounds(inner_bi, inner_lb_index,
+												&inner_lb, &inner_ub);
+		else
+			finished_inner = true;
+
+		Assert(!finished_outer || !finished_inner);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (finished_outer)
+		{
+			overlap = false;
+			ub_cmpval = 1;
+			lb_cmpval = 1;
+		}
+		else if (finished_inner)
+		{
+			overlap = false;
+			ub_cmpval = -1;
+			lb_cmpval = -1;
+		}
+		else
+			overlap = partition_range_cmp(partnatts, partsupfuncs, partcollations,
+										  &outer_lb, &outer_ub, &inner_lb,
+										  &inner_ub, &ub_cmpval, &lb_cmpval);
+
+		if (overlap)
+		{
+			/*
+			 * The rows from overlapping portion of ranges on both sides may
+			 * join, hence the corresponding pair of partitions form a joining
+			 * pair. Match them and produce the bounds of the joint partition
+			 * and its index by merging the bounds according to the type of
+			 * join.
+			 */
+			partition_range_merge(partnatts, partsupfuncs, partcollations,
+								  jointype, &outer_lb, &outer_ub, &inner_lb,
+								  &inner_ub, &merged_lb, &merged_ub);
+			merged_index = map_and_merge_partitions(outer_pmap, outer_mmap,
+													outer_part, inner_pmap,
+													inner_mmap, inner_part,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				/* Failed to match the partitions. */
+				merged = false;
+				break;
+			}
+
+			/*
+			 * If the ranges overlap but don't exactly match, a row from
+			 * non-overlapping portion of the range from one side of join may
+			 * find its join partner in the previous or next overlapping
+			 * partition or default partition on the other side , if such a
+			 * partition exists. All those cases, if true, will cause one
+			 * partition from that side to match at least two partitions on the
+			 * other side; a case that we do not support now. Previous
+			 * partition has been delt with in the previous iteration of this
+			 * loop, next partition will be delt in the next iteration. We will
+			 * deal with the default partition here.
+			 */
+			if ((lb_cmpval < 0 && inner_has_default) ||
+				/* Non-overlapping range on the lower side of outer range. */
+				(lb_cmpval > 0 && outer_has_default) ||
+				/* Non-overlapping range on the lower side of inner range. */
+				(ub_cmpval < 0 && outer_has_default) ||
+				/* Non-overlapping range on the upper side of inner range. */
+				(ub_cmpval > 0 && inner_has_default))
+				/* Non-overlapping range on the upper side of outer range. */
+			{
+				merged = false;
+				break;
+			}
+		}
+
+		if (ub_cmpval == 0)
+		{
+			/* Upper bounds of both the ranges match. */
+			Assert(overlap);
+
+			/* Move to the next pair of partitions. */
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the inner side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &outer_lb;
+				merged_ub = &outer_ub;
+
+				/*
+				 * For a FULL join, inner relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the inner relation acts as
+				 * INNER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_inner = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_outer = (jointype == JOIN_FULL);
+				merged = handle_missing_partition(inner_pmap, inner_mmap,
+														inner_default, outer_pmap,
+														outer_mmap, outer_part,
+														missing_side_outer,
+														missing_side_inner,
+														&next_index,
+														&default_index,
+														&merged_index);
+
+				if (!merged)
+					break;
+			}
+
+			/* Move to the next partition on the outer side. */
+			Assert(!finished_outer);
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+		}
+		else
+		{
+			Assert(ub_cmpval > 0);
+
+			/* Upper bound of outer range higher than that of the inner. */
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the outer side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &inner_lb;
+				merged_ub = &inner_ub;
+
+				/*
+				 * For a FULL join, outer relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the outer relation acts as
+				 * OUTER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_outer = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_inner = (jointype == JOIN_FULL);
+				merged = handle_missing_partition(outer_pmap, outer_mmap,
+												  outer_default, inner_pmap,
+												  inner_mmap, inner_part,
+												  missing_side_outer,
+												  missing_side_inner,
+												  &next_index,
+												  &default_index,
+												  &merged_index);
+
+				if (!merged)
+					break;
+			}
+
+			/* Move to the next partition on the inner side. */
+			Assert (!finished_inner);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+
+		Assert(merged);
+
+		if (merged_index < 0)
+		{
+			/* We didn't find a new merged partition. */
+			continue;
+		}
+
+		/*
+		 * We have a valid partition index for the next partition of join. The
+		 * partition should have valid range.
+		 */
+		Assert(merged_lb && merged_ub);
+
+		/* Try merging new lower bound with the last upper bound. */
+		merged = partition_range_merge_next_lb(partnatts, partsupfuncs,
+											   partcollations,
+											   merged_lb->datums,
+											   merged_lb->kind, &merged_datums,
+											   &merged_kinds, &merged_indexes);
+		if (merged)
+		{
+			/* Add upper bound with the merged partition index. */
+			merged_datums = lappend(merged_datums, merged_ub->datums);
+			merged_kinds = lappend(merged_kinds, merged_ub->kind);
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+		}
+		else
+			break;
+	}
+
+	if (merged)
+		merged = merge_default_partitions(outer_bi, outer_pmap, outer_mmap,
+										  inner_bi, inner_pmap, inner_mmap,
+										  jointype, &next_index,
+										  &default_index);
+
+	/* Create PartitionBoundInfo for the join result. */
+	if (merged)
+	{
+		/* Use maps to match partition from the joining relations. */
+		generate_matching_part_pairs(outer_mmap, outer_nparts, inner_mmap,
+									 inner_nparts, jointype, next_index,
+									 outer_parts, inner_parts);
+
+		/* Craft a PartitionBoundInfo to return. */
+		if (*outer_parts && *inner_parts)
+		{
+			Assert(list_length(*outer_parts) == list_length(*inner_parts));
+			Assert(list_length(*outer_parts) == next_index);
+			merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+														  merged_datums,
+														  merged_indexes,
+														  merged_kinds,
+														  -1, default_index);
+		}
+	}
+
+	/* Free any memory we used in this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+	pfree(outer_pmap);
+	pfree(inner_pmap);
+	pfree(outer_mmap);
+	pfree(inner_mmap);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(int partnatts, FmgrInfo *partsupfunc,
+							Oid *partcollation, PartitionBoundInfo outer_bi,
+							int outer_nparts, PartitionBoundInfo inner_bi,
+							int inner_nparts, JoinType jointype,
+							List **outer_parts, List **inner_parts)
+{
+	int		   *outer_pmap;	/* outer to inner partition map */
+	int		   *outer_mmap;	/* outer to merged partition map */
+	int		   *inner_pmap;	/* inner to outer partition map */
+	int		   *inner_mmap;	/* inner to merged partition map */
+	int			cnto;
+	int			cnti;
+	bool		merged = true;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			next_index = 0;
+	int			null_index;
+	int			default_index = -1;
+	PartitionBoundInfo merged_bounds = NULL;
+	int		   *outer_indexes = outer_bi->indexes;
+	int		   *inner_indexes = inner_bi->indexes;
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	/*
+	 * Allocate and initialize partition maps. We maintain four maps, two maps
+	 * for each joining relation. pmap[i] gives the partition from the other
+	 * relation which would join with ith partition of the given relation.
+	 * Partition i from the given relation will join with partition pmap[i]
+	 * from the other relation to produce partition mmap[i] of the join (merged
+	 * partition).
+	 *
+	 * pmap[i] = -1 indicates that ith partition of a given relation does not
+	 * have a matching partition from the other relation.
+	 *
+	 * mmap[i] = -1 indicates that ith partition of a given relation does not
+	 * contribute to the join result. That can happen only when the given
+	 * relation is the inner relation and it doesn't have a matching partition
+	 * from the outer relation, hence pmap[i] should be -1.
+	 *
+	 * In case of an outer join, every partition of the outer join will appear
+	 * in the join result, and thus has mmap[i] set for all i. But it's not
+	 * necessary that every partition on the outer side will have a matching
+	 * partition on the inner side. In such a case, we end up with pmap[i] = -1
+	 * and mmap[i] != -1.
+	 */
+	outer_pmap = (int *) palloc(sizeof(int) * outer_nparts);
+	outer_mmap = (int *) palloc(sizeof(int) * outer_nparts);
+	inner_pmap = (int *) palloc(sizeof(int) * inner_nparts);
+	inner_mmap = (int *) palloc(sizeof(int) * inner_nparts);
+
+	for (cnto = 0; cnto < outer_nparts; cnto++)
+	{
+		outer_pmap[cnto] = -1;
+		outer_mmap[cnto] = -1;
+	}
+	for (cnti = 0; cnti < inner_nparts; cnti++)
+	{
+		inner_pmap[cnti] = -1;
+		inner_mmap[cnti] = -1;
+	}
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			o_index;
+		int			i_index;
+		int			cmpval;
+		int			merged_index = -1;
+		Datum	   *merged_datum;
+		bool		finished_inner;
+		bool		finished_outer;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+		{
+			finished_outer = true;
+			odatums = NULL;
+			o_index = -1;
+		}
+		else
+		{
+			finished_outer = false;
+			odatums = outer_bi->datums[cnto];
+			o_index = outer_indexes[cnto];
+		}
+
+		if (cnti >= inner_bi->ndatums)
+		{
+			finished_inner = true;
+			idatums = NULL;
+			i_index = -1;
+		}
+		else
+		{
+			finished_inner = false;
+			idatums = inner_bi->datums[cnti];
+			i_index = inner_indexes[cnti];
+		}
+
+		/* If we exhausted both the sides, we won't enter the loop. */
+		Assert(!finished_inner || !finished_outer);
+
+		if (finished_outer)
+			cmpval = 1;
+		else if (finished_inner)
+			cmpval = -1;
+		else
+		{
+			/* Every list datum should map to a valid partition index. */
+			Assert(o_index >= 0 && i_index >= 0 &&
+				   odatums != NULL && idatums != NULL);
+
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0], idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			merged_index = map_and_merge_partitions(outer_pmap, outer_mmap,
+													o_index, inner_pmap,
+													inner_mmap, i_index,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				merged = false;
+				break;
+			}
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			/* A datum missing from the inner side. */
+			merged_index = -1;
+			merged_datum = odatums;
+
+			/*
+			 * For a FULL join, inner relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the inner relation acts as
+			 * INNER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_inner = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_outer = (jointype == JOIN_FULL);
+			merged = handle_missing_partition(inner_pmap, inner_mmap,
+													inner_default, outer_pmap,
+													outer_mmap, o_index,
+													missing_side_outer,
+													missing_side_inner,
+													&next_index,
+													&default_index,
+													&merged_index);
+
+			if (!merged)
+				break;
+
+			/* Move to the next datum on the outer side. */
+			Assert(!finished_outer);
+			cnto++;
+		}
+		else
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			Assert(cmpval > 0);
+
+			/* A datum missing from the outer side. */
+			merged_index = -1;
+			merged_datum = idatums;
+
+			/*
+			 * For a FULL join, outer relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the outer relation acts as
+			 * OUTER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_outer = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_inner = (jointype == JOIN_FULL);
+			merged = handle_missing_partition(outer_pmap, outer_mmap,
+											  outer_default, inner_pmap,
+											  inner_mmap, i_index,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index);
+
+			if (!merged)
+				break;
+
+			/* Move to the next datum on the right side. */
+			Assert(!finished_inner);
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	if (merged)
+		merged = merge_null_partitions(outer_bi, outer_pmap, outer_mmap,
+									   inner_bi, inner_pmap, inner_mmap,
+									   jointype, &next_index, &null_index,
+									   &default_index);
+
+	if (merged)
+		merged = merge_default_partitions(outer_bi, outer_pmap, outer_mmap,
+										  inner_bi, inner_pmap, inner_mmap,
+										  jointype, &next_index,
+										  &default_index);
+
+	/* If successful build the output structures. */
+	if (merged)
+	{
+
+		/* Use maps to match partition from the joining relations. */
+		generate_matching_part_pairs(outer_mmap, outer_nparts, inner_mmap,
+									 inner_nparts, jointype, next_index,
+									 outer_parts, inner_parts);
+		/* Craft a PartitionBoundInfo to return. */
+		if (*outer_parts && *inner_parts)
+		{
+			Assert(list_length(*outer_parts) == list_length(*inner_parts));
+			Assert(list_length(*outer_parts) == next_index);
+			merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+														  merged_datums,
+														  merged_indexes, NIL,
+														  null_index, default_index);
+		}
+	}
+
+	/* Free up all extra memory before returning from this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	pfree(outer_pmap);
+	pfree(inner_pmap);
+	pfree(outer_mmap);
+	pfree(inner_mmap);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_bounds_merge()'s arm for hash partitioned tables.
+ *
+ * If the given two hash bounds are same, the function returns the first one
+ * without any change, alongwith the lists of matching partitions. Otherwise it
+ * returns NULL.
+ *
+ * We could try merging the bounds when both the bounds have same greatest
+ * modulii. But there seems to be hardly any requirement for the same.
+ */
+static PartitionBoundInfo
+partition_hash_bounds_merge(int partnatts, FmgrInfo *partsupfunc,
+							Oid *partcollation, PartitionBoundInfo outer_bi,
+							int outer_nparts, PartitionBoundInfo inner_bi,
+							int inner_nparts, JoinType jointype,
+							List **outer_parts, List **inner_parts)
+{
+	int			nparts;
+	int			cnt;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * Hash partitioned table does not have explicit NULL accepting partition
+	 * and also does not have a default partition.
+	 */
+	Assert(!partition_bound_has_default(outer_bi) &&
+		   !partition_bound_has_default(inner_bi));
+	Assert(!partition_bound_accepts_nulls(outer_bi) &&
+		   !partition_bound_accepts_nulls(inner_bi));
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+
+	if (outer_nparts != inner_nparts)
+		return NULL;
+	nparts = outer_nparts;
+
+	if (outer_bi->ndatums != inner_bi->ndatums ||
+		!partition_hbounds_equal(outer_bi, inner_bi))
+		return NULL;
+
+	 /*
+	  * Cook up list of matching partitions. Since bounds are exactly same the
+	  * partitions at the same position from both the relations match.
+	  */
+	for (cnt = 0; cnt < nparts; cnt++)
+	{
+		*outer_parts = lappend_int(*outer_parts, cnt);
+		*inner_parts = lappend_int(*inner_parts, cnt);
+	}
+
+	return outer_bi;
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * If the two given partitions (given by index1 and index2 resp.) are already
+ * mapped to each other return the index of corresponding partition in the
+ * merged set of partitions.  If they do not have a merged partition associated
+ * with them, assign a new merged partition index.  If the partitions are
+ * already mapped and their mapped partitions are different from each other,
+ * they can not be merged, so return -1.
+ *
+ * partmap1[i] gives the partition of relation 2 which matches ith partition of
+ * relation 1. Similarly for partmap2.
+ *
+ * mergemap1[i] gives the partition in the merged set to which ith partition of
+ * relation 1 maps to. Similarly for mergemap2.
+ *
+ * index1 and index2 are the indexes of matching partition from respective
+ * relations.
+ *
+ * *next_index is used and incremented when the given partitions require a new
+ * merged partition.
+ */
+static int
+map_and_merge_partitions(int *partmap1, int *mergemap1, int index1,
+						 int *partmap2, int *mergemap2, int index2,
+						 int *next_index)
+{
+	int			merged_index;
+
+	/*
+	 * If both the partitions are not mapped to each other, update the
+	 * maps.
+	 */
+	if (partmap1[index1] < 0 && partmap2[index2] < 0)
+	{
+		partmap1[index1] = index2;
+		partmap2[index2] = index1;
+	}
+
+	/*
+	 * If the given to partitions map to each other, find the corresponding
+	 * merged partition index .
+	 */
+	if (partmap1[index1] == index2 && partmap2[index2] == index1)
+	{
+		/*
+		 * If both the partitions are mapped to the same merged partition, get
+		 * the index of merged partition.
+		 */
+		if (mergemap1[index1] == mergemap2[index2])
+		{
+			merged_index = mergemap1[index1];
+
+			/*
+			 * If the given two partitions do not have a merged partition
+			 * associated with them, allocate a new merged partition.
+			 */
+			if (merged_index < 0)
+			{
+				merged_index = *next_index;
+				*next_index = *next_index + 1;
+				mergemap1[index1] = merged_index;
+				mergemap2[index2] = merged_index;
+			}
+		}
+
+		/*
+		 * If partition from one relation was mapped to a merged partition but
+		 * not the partition from the other relation, map the same merged
+		 * partition to the partition from other relation, since matching
+		 * partitions map to the same merged partition.
+		 */
+		else if (mergemap1[index1] >= 0 && mergemap2[index2] < 0)
+		{
+			mergemap2[index2] = mergemap1[index1];
+			merged_index = mergemap1[index1];
+		}
+		else if (mergemap1[index1] < 0 && mergemap2[index2] >= 0)
+		{
+			mergemap1[index1] = mergemap2[index2];
+			merged_index = mergemap2[index2];
+		}
+		else
+		{
+			Assert(mergemap1[index1] != mergemap2[index2] &&
+				   mergemap1[index1] >= 0 && mergemap2[index2] >= 0);
+
+			/*
+			 * Both the partitions map to different merged partitions. This
+			 * means that multiple partitions from one relation matches to one
+			 * partition from the other relation. Partition-wise join does not
+			 * handle this case right now, since it requires ganging multiple
+			 * partitions together (into one RelOptInfo).
+			 */
+			merged_index = -1;
+		}
+	}
+	else
+	{
+		/*
+		 * Multiple partitions from one relation map to one partition from the
+		 * other relation. Partition-wise join does not handle this case right
+		 * now, since it requires ganging multiple partitions together (into
+		 * one RelOptInfo).
+		 */
+		merged_index = -1;
+	}
+
+	return merged_index;
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * mergemap1 and mergemap2 map each partition from either side of the join to a
+ * merged partition resp. E.g. mergemap1[i] gives the merged partition to which
+ * ith partition of first relation maps and mergemap2[j] gives the merged
+ * partition to which jth partition of second relation maps. If mergemap1[i] =
+ * mergemap2[j], i and j form the matching pair of partitions.
+ *
+ * Given these maps this function produces the list pairs of partitions which
+ * when joined produce the merged partitions in the order of merged partition
+ * indexes.
+ *
+ * nparts1 and nparts2 are the number of partitions of the joining relations
+ * resp.
+ *
+ * nparts is the number of merged partitions.
+ *
+ * If successful, the pairs of partitions are returned as two separate lists,
+ * parts1 and parts2 resp., one for each side. Otherwise, those lists will be
+ * set to NIL.
+ */
+static void
+generate_matching_part_pairs(int *mergemap1, int nparts1, int *mergemap2,
+							 int nparts2, JoinType jointype, int nparts,
+							 List **parts1, List **parts2)
+{
+	bool		merged = true;
+	int		  **matching_parts;
+	int			cnt1;
+	int			cnt2;
+
+	matching_parts = (int **) palloc(sizeof(int *) * 2);
+	matching_parts[0] = (int *) palloc(sizeof(int) * nparts);
+	matching_parts[1] = (int *) palloc(sizeof(int) * nparts);
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt1 = 0; cnt1 < nparts; cnt1++)
+	{
+		matching_parts[0][cnt1] = -1;
+		matching_parts[1][cnt1] = -1;
+	}
+
+	/* Set pairs of matching partitions. */
+	for (cnt1 = 0; cnt1 < nparts1; cnt1++)
+	{
+		if (mergemap1[cnt1] >= 0)
+		{
+			Assert(mergemap1[cnt1] < nparts);
+			matching_parts[0][mergemap1[cnt1]] = cnt1;
+		}
+	}
+	for (cnt2 = 0; cnt2 < nparts2; cnt2++)
+	{
+		if (mergemap2[cnt2] >= 0)
+		{
+			Assert(mergemap2[cnt2] < nparts);
+			matching_parts[1][mergemap2[cnt2]] = cnt2;
+		}
+	}
+
+	/*
+	 * If we have a partition missing on an inner side, we need to add a dummy
+	 * relation which joins with the outer partition. If the inner relation
+	 * happens to be a base relation, it will require adding a dummy child
+	 * base relation during join processing. Right now, we freeze the base
+	 * relation arrays like PlannerInfo::simple_rte_array after planning for
+	 * base relations. Adding a new (dummy) base relation would require some
+	 * changes to that. So, right now, we do not implement partition-wise join
+	 * in such cases.
+	 */
+	for (cnt1 = 0; cnt1 < nparts; cnt1++)
+	{
+		int			part1 = matching_parts[0][cnt1];
+		int			part2 = matching_parts[1][cnt1];
+
+		/* At least one of the partitions should exist. */
+		Assert(part1 >= 0 || part2 >= 0);
+
+		switch (jointype)
+		{
+			case JOIN_INNER:
+			case JOIN_SEMI:
+
+				/*
+				 * An inner or semi join can not return any row when the
+				 * matching partition on either side is missing. We should
+				 * have eliminated all such cases while merging the bounds.
+				 */
+				Assert(part1 >= 0 && part2 >= 0);
+				break;
+
+			case JOIN_LEFT:
+			case JOIN_ANTI:
+				Assert(part1 >= 0);
+				if (part2 < 0)
+					merged = false;
+				break;
+
+			case JOIN_FULL:
+				if (part1 < 0 || part2 < 0)
+					merged = false;
+				break;
+
+			default:
+				elog(ERROR, "unrecognized join type: %d", (int) jointype);
+		}
+
+		if (!merged)
+			break;
+
+		*parts1 = lappend_int(*parts1, part1);
+		*parts2 = lappend_int(*parts2, part2);
+	}
+
+	pfree(matching_parts[0]);
+	pfree(matching_parts[1]);
+	pfree(matching_parts);
+
+	if (!merged)
+	{
+		list_free(*parts1);
+		list_free(*parts2);
+		*parts1 = NIL;
+		*parts2 = NIL;
+	}
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * Merge default partitions from both sides, if any, and assign the default
+ * partition for the join result, if necessary.
+ *
+ * If both the relations have default partitions, try mapping those to each
+ * other. If the mapping succeeds corresponding merged partition will act as
+ * the default partition of the join result.
+ *
+ * If inner side of the join has default but not the outer side, rows in it
+ * won't appear in the join result. So don't create a default partition. If
+ * outer side of the join has default but not the inner side, rows in it will
+ * appear in the join result, so create a default merged partition.
+ */
+static bool
+merge_default_partitions(PartitionBoundInfo outer_bi, int *outer_pmap,
+						 int *outer_mmap, PartitionBoundInfo inner_bi,
+						 int *inner_pmap, int *inner_mmap, JoinType jointype,
+						 int *next_index, int *default_index)
+{
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	bool		merged = true;
+
+	if (!outer_has_default && !inner_has_default)
+		Assert(*default_index < 0);
+	else if (outer_has_default && !inner_has_default)
+	{
+		if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+			jointype == JOIN_ANTI)
+		{
+			if (outer_mmap[outer_default] < 0)
+			{
+				outer_mmap[outer_default] = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = outer_mmap[outer_default];
+			}
+			else
+				Assert(*default_index == outer_mmap[outer_default]);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else if (!outer_has_default && inner_has_default)
+	{
+		if (jointype == JOIN_FULL)
+		{
+			if (inner_mmap[inner_default] < 0)
+			{
+				inner_mmap[inner_default] = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = inner_mmap[inner_default];
+			}
+			else
+				Assert(*default_index == inner_mmap[inner_default]);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		*default_index = map_and_merge_partitions(outer_pmap, outer_mmap,
+												 outer_default, inner_pmap,
+												 inner_mmap, inner_default,
+												 next_index);
+
+		if (*default_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
+
+/*
+ * merge_null_partitions
+ *
+ * Merge NULL partitions, i.e. a partition that can hold NULL values for a list
+ * partitioned table, if any. Find the index of merged partition to which the
+ * NULL values would belong in the join result. If one joining relation has a
+ * NULL partition but not the other, try matching it with the default partition
+ * from the other relation since the default partition may have rows with NULL
+ * partition key. We can eliminate a NULL partition when it appears only on the
+ * inner side of the join and the outer side doesn't have a default partition.
+ *
+ * When the equality operator used for join is strict, two NULL values will not
+ * be considered as equal, and thus a NULL partition can be eliminated for an
+ * inner join. But we don't check the strictness operator here.
+ */
+static bool
+merge_null_partitions(PartitionBoundInfo outer_bi, int *outer_pmap,
+					  int *outer_mmap, PartitionBoundInfo inner_bi,
+					  int *inner_pmap, int *inner_mmap, JoinType jointype,
+					  int *next_index, int *null_index, int *default_index)
+{
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	int			outer_ni = outer_bi->null_index;
+	int			inner_ni = inner_bi->null_index;
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	bool		merged = true;
+
+	if (!outer_has_null && !inner_has_null)
+		*null_index = -1;
+	else if (outer_has_null && !inner_has_null)
+	{
+		int			merged_index;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, inner relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the inner relation acts as
+		 * INNER relation. For INNER and SEMI join OUTER and INNER
+		 * differentiation is immaterial.
+		 */
+		missing_side_inner = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_outer = (jointype == JOIN_FULL);
+
+		merged = handle_missing_partition(inner_pmap, inner_mmap,
+										  inner_default, outer_pmap,
+										  outer_mmap, outer_ni,
+										  missing_side_outer,
+										  missing_side_inner, next_index,
+										  default_index, &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join to which the outer null partition maps
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = outer_mmap[outer_ni];
+	}
+	else if (!outer_has_null && inner_has_null)
+	{
+		int			merged_index;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, outer relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the outer relation acts as OUTER
+		 * relation. For INNER and SEMI join OUTER and INNER differentiation is
+		 * immaterial.
+		 */
+		missing_side_outer = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_inner = (jointype == JOIN_FULL);
+		merged = handle_missing_partition(outer_pmap, outer_mmap,
+										  outer_default, inner_pmap,
+										  inner_mmap, inner_ni,
+										  missing_side_outer,
+										  missing_side_inner,
+										  next_index, default_index,
+										  &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join, to which the outer side null partition maps,
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = inner_mmap[inner_ni];
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		*null_index = map_and_merge_partitions(outer_pmap, outer_mmap,
+											  outer_ni, inner_pmap,
+											  inner_mmap, inner_ni,
+											  next_index);
+		if (*null_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index 36fb584e23..5cc5697da7 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -107,5 +107,11 @@ extern int partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 							  int nvalues, Datum *values, bool *is_equal);
 extern int partition_hash_bsearch(PartitionBoundInfo boundinfo,
 					   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   PartitionBoundInfo outer_bi, int outer_nparts,
+					   PartitionBoundInfo inner_bi, int inner_nparts,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
-- 
2.16.4

0004-Tests-for-0-1-1-1-and-1-0-partition-matching-v13.patchapplication/octet-stream; name=0004-Tests-for-0-1-1-1-and-1-0-partition-matching-v13.patchDownload
From c40ff3fd808166a6121be9980868d3b957d29ec5 Mon Sep 17 00:00:00 2001
From: erthalion <9erthalion6@gmail.com>
Date: Tue, 11 Sep 2018 21:11:55 +0200
Subject: [PATCH 4/5] Tests for 0:1, 1:1 and 1:0 partition matching

Rajkumar Raghuvanshi and Ashutosh Bapat.
---
 src/test/regress/expected/partition_join.out | 4131 +++++++++++++++++++++-----
 src/test/regress/sql/partition_join.sql      |  427 ++-
 2 files changed, 3855 insertions(+), 703 deletions(-)

diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index c55de5d476..f19611c853 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -8,59 +8,86 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-(4 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+(7 rows)
 
 -- left outer join, with whole-row reference; partitionwise join does not apply
 EXPLAIN (COSTS OFF)
@@ -72,35 +99,50 @@ SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER
    ->  Hash Right Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(22 rows)
 
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-      t1      |      t2      
---------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
- (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
- (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
- (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
- (550,0,0550) | 
-(12 rows)
+       t1       |       t2       
+----------------+----------------
+ (-250,0,-0250) | 
+ (-200,0,-0200) | 
+ (-150,0,-0150) | (0,-150,-0150)
+ (-100,0,-0100) | 
+ (-50,0,-0050)  | 
+ (0,0,0000)     | (0,0,0000)
+ (50,0,0050)    | 
+ (100,0,0100)   | 
+ (150,0,0150)   | (0,150,0150)
+ (200,0,0200)   | 
+ (250,0,0250)   | 
+ (300,0,0300)   | (0,300,0300)
+ (350,0,0350)   | 
+ (400,0,0400)   | 
+ (450,0,0450)   | (0,450,0450)
+ (500,0,0500)   | 
+ (550,0,0550)   | 
+ (600,0,0600)   | (0,600,0600)
+ (650,0,0650)   | 
+ (700,0,0700)   | 
+ (750,0,0750)   | (0,750,0750)
+(21 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
@@ -112,35 +154,53 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
    ->  Append
          ->  Hash Right Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Right Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t2_2.b)
-(20 rows)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+      |       | -225 | -0225
+      |       |  -75 | -0075
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(14 rows)
 
 -- full outer join, with placeholder vars
 EXPLAIN (COSTS OFF)
@@ -148,8 +208,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                             QUERY PLAN                            
 ------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               Filter: (((50) = prt1_p0.a) OR ((75) = prt2_p0.b))
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
@@ -174,7 +242,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               Filter: (((50) = prt1_p4.a) OR ((75) = prt2_p4.b))
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
@@ -211,8 +287,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Hash Left Join
+               Hash Cond: (prt1_p0.a = b)
+               ->  Seq Scan on prt1_p0
+                     Filter: ((a < 450) AND (b = 0))
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
          ->  Hash Left Join
                Hash Cond: (prt1_p1.a = b)
                ->  Seq Scan on prt1_p1
@@ -227,29 +310,42 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                ->  Hash
                      ->  Seq Scan on prt1_p2
                            Filter: ((a < 450) AND (b = 0))
-(17 rows)
+(24 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-(9 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+(14 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
                          QUERY PLAN                         
 ------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = b)
+               Filter: ((prt1_p0.b = 0) OR (a = 0))
+               ->  Seq Scan on prt1_p0
+                     Filter: (a < 450)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = b)
                Filter: ((prt1_p1.b = 0) OR (a = 0))
@@ -274,64 +370,153 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                ->  Hash
                      ->  Result
                            One-Time Filter: false
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt2_p4.b = a)
+               Filter: ((b = 0) OR (prt2_p4.a = 0))
+               ->  Seq Scan on prt2_p4
+                     Filter: (b > 250)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
-(12 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+      |       | 375 | 0375
+      |       | 450 | 0450
+      |       | 525 | 0525
+      |       | 600 | 0600
+      |       | 675 | 0675
+      |       | 750 | 0750
+(20 rows)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Semi Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Semi Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                            Filter: (a = 0)
-         ->  Nested Loop Semi Join
-               Join Filter: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
-               ->  Materialize
-                     ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
-(24 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt1_p3 t2_3
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+                           Filter: (b = 0)
+(37 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
 
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
@@ -342,27 +527,82 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
    ->  Append
          ->  Hash Anti Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Hash Anti Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Hash Anti Join
                Hash Cond: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
-(17 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(27 rows)
 
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
-  sum  |         avg          | sum  |         avg         
--------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+  sum  |         avg          | sum  |        avg         
+-------+----------------------+------+--------------------
+ 95550 | 273.0000000000000000 | 2200 | 6.2857142857142857
 (1 row)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Append
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p0 t1
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+               Index Cond: (a = t1.b)
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.b = t2_1.a)
+         ->  Seq Scan on prt2_p1 t1_1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p1 t2_1
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p2 t1_2
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+               Index Cond: (a = t1_2.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p3 t1_3
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+               Index Cond: (a = t1_3.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p4 t1_4
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+               Index Cond: (a = t1_4.b)
+(27 rows)
+
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+  b   |   c   
+------+-------
+ -225 | -0225
+  -75 | -0075
+   75 | 0075
+  225 | 0225
+  375 | 0375
+  525 | 0525
+  675 | 0675
+(7 rows)
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -374,49 +614,74 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
+                     ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
                            Index Cond: (a = t1.a)
-                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t3
                            Index Cond: (b = t2.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_1
+                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
                            Index Cond: (a = t1_1.a)
-                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3_1
                            Index Cond: (b = t2_1.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_2
+                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
                            Index Cond: (a = t1_2.a)
-                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_2
                            Index Cond: (b = t2_2.a)
-(27 rows)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                           Index Cond: (a = t1_3.a)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_3
+                           Index Cond: (b = t2_3.a)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                           Index Cond: (a = t1_4.a)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t3_4
+                           Index Cond: (b = t2_4.a)
+(43 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   | t2a | t3a | least 
------+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
- 100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
- 250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
- 400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
- 550 | 0 | 0550 |     |     |      
-(12 rows)
+  a   | b |   c   | t2a  | t3a | least 
+------+---+-------+------+-----+-------
+ -250 | 0 | -0250 |      |     |      
+ -200 | 0 | -0200 |      |     |      
+ -150 | 0 | -0150 | -150 |   0 |  -150
+ -100 | 0 | -0100 |      |     |      
+  -50 | 0 | -0050 |      |     |      
+    0 | 0 | 0000  |    0 |   0 |     0
+   50 | 0 | 0050  |      |     |      
+  100 | 0 | 0100  |      |     |      
+  150 | 0 | 0150  |  150 |   0 |   150
+  200 | 0 | 0200  |      |     |      
+  250 | 0 | 0250  |      |     |      
+  300 | 0 | 0300  |  300 |   0 |   300
+  350 | 0 | 0350  |      |     |      
+  400 | 0 | 0400  |      |     |      
+  450 | 0 | 0450  |  450 |   0 |   450
+  500 | 0 | 0500  |      |     |      
+  550 | 0 | 0550  |      |     |      
+  600 | 0 | 0600  |  600 |   0 |   600
+  650 | 0 | 0650  |      |     |      
+  700 | 0 | 0700  |      |     |      
+  750 | 0 | 0750  |  750 |   0 |   750
+(21 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
@@ -430,64 +695,95 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
          Hash Cond: ((t1.c)::text = (t2.c)::text)
          Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Seq Scan on prt1_p1 t1
-               ->  Seq Scan on prt1_p2 t1_1
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
          ->  Hash
                ->  Append
                      ->  Hash Join
                            Hash Cond: (t2.a = t3.b)
-                           ->  Seq Scan on prt1_p1 t2
+                           ->  Seq Scan on prt1_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t3
+                                 ->  Seq Scan on prt2_p0 t3
                      ->  Hash Join
                            Hash Cond: (t2_1.a = t3_1.b)
-                           ->  Seq Scan on prt1_p2 t2_1
+                           ->  Seq Scan on prt1_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t3_1
+                                 ->  Seq Scan on prt2_p1 t3_1
                      ->  Hash Join
                            Hash Cond: (t2_2.a = t3_2.b)
-                           ->  Seq Scan on prt1_p3 t2_2
+                           ->  Seq Scan on prt1_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p3 t3_2
-(26 rows)
+                                 ->  Seq Scan on prt2_p2 t3_2
+                     ->  Hash Join
+                           Hash Cond: (t2_3.a = t3_3.b)
+                           ->  Seq Scan on prt1_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t3_3
+                     ->  Hash Join
+                           Hash Cond: (t2_4.a = t3_4.b)
+                           ->  Seq Scan on prt1_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t3_4
+(38 rows)
 
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, t2.b t2b, t2.c t2c, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
-  a  | t2a | t2c  
------+-----+------
-   0 |   0 | 0000
-  50 |     | 
- 100 |     | 
- 150 | 150 | 0150
- 200 |     | 
- 250 |     | 
- 300 | 300 | 0300
- 350 |     | 
- 400 |     | 
- 450 | 450 | 0450
- 500 |     | 
- 550 |     | 
-(12 rows)
+  a   | t2a  |  t2c  
+------+------+-------
+ -250 |      | 
+ -200 |      | 
+ -150 | -150 | -0150
+ -100 |      | 
+  -50 |      | 
+    0 |    0 | 0000
+   50 |      | 
+  100 |      | 
+  150 |  150 | 0150
+  200 |      | 
+  250 |      | 
+  300 |  300 | 0300
+  350 |      | 
+  400 |      | 
+  450 |  450 | 0450
+  500 |      | 
+  550 |      | 
+  600 |  600 | 0600
+  650 |      | 
+  700 |      | 
+  750 |  750 | 0750
+(21 rows)
 
 --
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
@@ -498,32 +794,49 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
    ->  Append
          ->  Hash Join
                Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
-               ->  Seq Scan on prt2_e_p1 t2
+               ->  Seq Scan on prt2_e_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 t1
+                     ->  Seq Scan on prt1_e_p0 t1
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
-               ->  Seq Scan on prt2_e_p2 t2_1
+               Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+               ->  Seq Scan on prt1_e_p1 t1_1
+                     Filter: (c = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 t1_1
-                           Filter: (c = 0)
+                     ->  Seq Scan on prt2_e_p1 t2_1
          ->  Hash Join
                Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
-               ->  Seq Scan on prt2_e_p3 t2_2
+               ->  Seq Scan on prt2_e_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 t1_2
+                     ->  Seq Scan on prt1_e_p2 t1_2
                            Filter: (c = 0)
-(21 rows)
+         ->  Hash Join
+               Hash Cond: (((t2_3.b + t2_3.a) / 2) = ((t1_3.a + t1_3.b) / 2))
+               ->  Seq Scan on prt2_e_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p3 t1_3
+                           Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (((t2_4.b + t2_4.a) / 2) = ((t1_4.a + t1_4.b) / 2))
+               ->  Seq Scan on prt2_e_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4 t1_4
+                           Filter: (c = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
- 150 | 0 | 150 | 0
- 300 | 0 | 300 | 0
- 450 | 0 | 450 | 0
-(4 rows)
+  a   | c |  b   | c 
+------+---+------+---
+ -250 | 0 | -250 | 0
+ -100 | 0 | -100 | 0
+    0 | 0 |    0 | 0
+    0 | 0 |    0 | 0
+  150 | 0 |  150 | 0
+  300 | 0 |  300 | 0
+  450 | 0 |  450 | 0
+  600 | 0 |  600 | 0
+  750 | 0 |  750 | 0
+(9 rows)
 
 --
 -- N-way join
@@ -536,154 +849,232 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = ((t3.a + t3.b) / 2))
+               Join Filter: (t1.a = t2.b)
                ->  Hash Join
-                     Hash Cond: (t2.b = t1.a)
-                     ->  Seq Scan on prt2_p1 t2
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on prt1_e_p0 t3
                      ->  Hash
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                     Index Cond: (((a + b) / 2) = t2.b)
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
+                     Index Cond: (b = ((t3.a + t3.b) / 2))
          ->  Nested Loop
                Join Filter: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_1.b = t1_1.a)
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
+               ->  Index Scan using iprt1_e_p4_ab2 on prt1_e_p1 t3_1
                      Index Cond: (((a + b) / 2) = t2_1.b)
          ->  Nested Loop
                Join Filter: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_2.b = t1_2.a)
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
+               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_2
                      Index Cond: (((a + b) / 2) = t2_2.b)
-(33 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = ((t3_3.a + t3_3.b) / 2))
+               ->  Nested Loop
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t2_3.b)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t2_4.b)
+               ->  Hash Join
+                     Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                     ->  Seq Scan on prt1_e_p4 t3_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_p4 t1_4
+                                 Filter: (b = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = ((t3_4.a + t3_4.b) / 2))
+(52 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
-(4 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(8 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                          QUERY PLAN                          
---------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Hash Right Join
                Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
-               ->  Seq Scan on prt1_e_p1 t3
+               ->  Seq Scan on prt1_e_p0 t3
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2.b = t1.a)
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_1.a)
-               ->  Seq Scan on prt1_e_p2 t3_1
+               ->  Seq Scan on prt1_e_p1 t3_1
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_1.b = t1_1.a)
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_2.a)
-               ->  Seq Scan on prt1_e_p3 t3_2
+               ->  Seq Scan on prt1_e_p2 t3_2
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_2.b = t1_2.a)
-                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                                        Filter: (b = 0)
-(33 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (t1_3.a = b)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (t1_3.a = ((a + b) / 2))
+         ->  Hash Right Join
+               Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+               ->  Seq Scan on prt1_e_p4 t3_4
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_4.b = t1_4.a)
+                           ->  Seq Scan on prt2_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_p4 t1_4
+                                       Filter: (b = 0)
+(51 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                            QUERY PLAN                             
--------------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p1 t3
+                           ->  Seq Scan on prt1_e_p0 t3
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p1_b on prt2_p1 t2
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
                      Index Cond: (t1.a = b)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p2 t3_1
+                           ->  Seq Scan on prt1_e_p1 t3_1
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
+               ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
                      Index Cond: (t1_1.a = b)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p3 t3_2
+                           ->  Seq Scan on prt1_e_p2 t3_2
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
                      Index Cond: (t1_2.a = b)
-(30 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_e_p3 t3_3
+                           Filter: (c = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                           Index Cond: (a = ((t3_3.a + t3_3.b) / 2))
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (t1_3.a = b)
+         ->  Nested Loop Left Join
+               ->  Hash Right Join
+                     Hash Cond: (t1_4.a = ((t3_4.a + t3_4.b) / 2))
+                     ->  Seq Scan on prt1_p4 t1_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p4 t3_4
+                                 Filter: (c = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (t1_4.a = b)
+(47 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -692,21 +1083,34 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                                                    QUERY PLAN                                                   
 ----------------------------------------------------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   Sort Key: prt1_p0.a, prt2_p0.b, ((prt1_e_p0.a + prt1_e_p0.b))
    ->  Append
          ->  Hash Full Join
-               Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
-               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               Hash Cond: (prt1_p0.a = ((prt1_e_p0.a + prt1_e_p0.b) / 2))
+               Filter: ((prt1_p0.a = (50)) OR (prt2_p0.b = (75)) OR (((prt1_e_p0.a + prt1_e_p0.b) / 2) = (50)))
                ->  Hash Full Join
-                     Hash Cond: (prt1_p1.a = prt2_p1.b)
-                     ->  Seq Scan on prt1_p1
+                     Hash Cond: (prt1_p0.a = prt2_p0.b)
+                     ->  Seq Scan on prt1_p0
                            Filter: (b = 0)
                      ->  Hash
-                           ->  Seq Scan on prt2_p1
+                           ->  Seq Scan on prt2_p0
                                  Filter: (a = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1
+                     ->  Seq Scan on prt1_e_p0
                            Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = prt1_p1.a)
+               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               ->  Seq Scan on prt1_e_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on prt1_p1
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1
+                                       Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
                Filter: ((prt1_p2.a = (50)) OR (prt2_p2.b = (75)) OR (((prt1_e_p2.a + prt1_e_p2.b) / 2) = (50)))
@@ -733,7 +1137,20 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                ->  Hash
                      ->  Seq Scan on prt1_e_p3
                            Filter: (c = 0)
-(42 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = ((prt1_e_p4.a + prt1_e_p4.b) / 2))
+               Filter: ((prt1_p4.a = (50)) OR (prt2_p4.b = (75)) OR (((prt1_e_p4.a + prt1_e_p4.b) / 2) = (50)))
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = prt2_p4.b)
+                     ->  Seq Scan on prt1_p4
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4
+                                 Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4
+                           Filter: (c = 0)
+(68 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
@@ -751,172 +1168,260 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = t1_3.b)
+               Join Filter: (t1.a = t1_5.b)
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Join
-                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
-                           ->  Seq Scan on prt1_e_p1 t2
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t1_3
+                                 ->  Seq Scan on prt2_p0 t1_5
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
                      Index Cond: (a = ((t2.a + t2.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_1.a = t1_4.b)
+               Join Filter: (t1_1.a = t1_6.b)
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Join
-                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
-                           ->  Seq Scan on prt1_e_p2 t2_1
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_6.b)
+                           ->  Seq Scan on prt1_e_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t1_4
+                                 ->  Seq Scan on prt2_p1 t1_6
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
                      Index Cond: (a = ((t2_1.a + t2_1.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_2.a = t1_5.b)
+               Join Filter: (t1_2.a = t1_7.b)
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
+                           ->  Seq Scan on prt2_p2 t1_7
                                  Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
+                           ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t2_2
+                                 Index Cond: (((a + b) / 2) = t1_7.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t1_8.b)
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Nested Loop
+                           ->  Seq Scan on prt2_p3 t1_8
+                                 Filter: (a = 0)
+                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_3
+                                 Index Cond: (((a + b) / 2) = t1_8.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = ((t2_3.a + t2_3.b) / 2))
+                     Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t1_9.b)
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Join
+                           Hash Cond: (((t2_4.a + t2_4.b) / 2) = t1_9.b)
+                           ->  Seq Scan on prt1_e_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t1_9
+                                       Filter: (a = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = ((t2_4.a + t2_4.b) / 2))
+                     Filter: (b = 0)
+(66 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                               QUERY PLAN                                
--------------------------------------------------------------------------
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
-                           ->  Seq Scan on prt2_p1 t1_3
+                           Hash Cond: (t1_5.b = ((t1_10.a + t1_10.b) / 2))
+                           ->  Seq Scan on prt2_p0 t1_5
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t1_6
+                                 ->  Seq Scan on prt1_e_p0 t1_10
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
-                     Index Cond: (a = t1_3.b)
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t1_5.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
-                           ->  Seq Scan on prt2_p2 t1_4
+                           Hash Cond: (t1_6.b = ((t1_11.a + t1_11.b) / 2))
+                           ->  Seq Scan on prt2_p1 t1_6
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t1_7
+                                 ->  Seq Scan on prt1_e_p1 t1_11
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
-                     Index Cond: (a = t1_4.b)
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
+                     Index Cond: (a = t1_6.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
-                           ->  Seq Scan on prt2_p3 t1_5
+                           Hash Cond: (t1_7.b = ((t1_12.a + t1_12.b) / 2))
+                           ->  Seq Scan on prt2_p2 t1_7
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p3 t1_8
+                                 ->  Seq Scan on prt1_e_p2 t1_12
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t1_5.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t1_7.b)
                      Filter: (b = 0)
-(39 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_8.b = ((t1_13.a + t1_13.b) / 2))
+                           ->  Seq Scan on prt2_p3 t1_8
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p3 t1_13
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t1_8.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_9.b = ((t1_14.a + t1_14.b) / 2))
+                           ->  Seq Scan on prt2_p4 t1_9
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t1_14
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = t1_9.b)
+                     Filter: (b = 0)
+(63 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 -- test merge joins
 SET enable_hashjoin TO off;
 SET enable_nestloop TO off;
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                           QUERY PLAN                           
-----------------------------------------------------------------
+                            QUERY PLAN                            
+------------------------------------------------------------------
  Merge Append
    Sort Key: t1.a
    ->  Merge Semi Join
-         Merge Cond: (t1.a = t1_3.b)
+         Merge Cond: (t1.a = t1_5.b)
          ->  Sort
                Sort Key: t1.a
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+               Merge Cond: (t1_5.b = (((t1_10.a + t1_10.b) / 2)))
                ->  Sort
-                     Sort Key: t1_3.b
-                     ->  Seq Scan on prt2_p1 t1_3
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on prt2_p0 t1_5
                ->  Sort
-                     Sort Key: (((t1_6.a + t1_6.b) / 2))
-                     ->  Seq Scan on prt1_e_p1 t1_6
+                     Sort Key: (((t1_10.a + t1_10.b) / 2))
+                     ->  Seq Scan on prt1_e_p0 t1_10
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_1.a = t1_4.b)
+         Merge Cond: (t1_1.a = t1_6.b)
          ->  Sort
                Sort Key: t1_1.a
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+               Merge Cond: (t1_6.b = (((t1_11.a + t1_11.b) / 2)))
                ->  Sort
-                     Sort Key: t1_4.b
-                     ->  Seq Scan on prt2_p2 t1_4
+                     Sort Key: t1_6.b
+                     ->  Seq Scan on prt2_p1 t1_6
                ->  Sort
-                     Sort Key: (((t1_7.a + t1_7.b) / 2))
-                     ->  Seq Scan on prt1_e_p2 t1_7
+                     Sort Key: (((t1_11.a + t1_11.b) / 2))
+                     ->  Seq Scan on prt1_e_p1 t1_11
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_2.a = t1_5.b)
+         Merge Cond: (t1_2.a = t1_7.b)
          ->  Sort
                Sort Key: t1_2.a
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+               Merge Cond: (t1_7.b = (((t1_12.a + t1_12.b) / 2)))
                ->  Sort
-                     Sort Key: t1_5.b
-                     ->  Seq Scan on prt2_p3 t1_5
+                     Sort Key: t1_7.b
+                     ->  Seq Scan on prt2_p2 t1_7
                ->  Sort
-                     Sort Key: (((t1_8.a + t1_8.b) / 2))
-                     ->  Seq Scan on prt1_e_p3 t1_8
+                     Sort Key: (((t1_12.a + t1_12.b) / 2))
+                     ->  Seq Scan on prt1_e_p2 t1_12
                            Filter: (c = 0)
-(47 rows)
+   ->  Merge Semi Join
+         Merge Cond: (t1_3.a = t1_8.b)
+         ->  Sort
+               Sort Key: t1_3.a
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_8.b = (((t1_13.a + t1_13.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_8.b
+                     ->  Seq Scan on prt2_p3 t1_8
+               ->  Sort
+                     Sort Key: (((t1_13.a + t1_13.b) / 2))
+                     ->  Seq Scan on prt1_e_p3 t1_13
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_4.a = t1_9.b)
+         ->  Sort
+               Sort Key: t1_4.a
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_9.b = (((t1_14.a + t1_14.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_9.b
+                     ->  Seq Scan on prt2_p4 t1_9
+               ->  Sort
+                     Sort Key: (((t1_14.a + t1_14.b) / 2))
+                     ->  Seq Scan on prt1_e_p4 t1_14
+                           Filter: (c = 0)
+(77 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
@@ -933,14 +1438,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
                            ->  Sort
                                  Sort Key: (((t3.a + t3.b) / 2))
-                                 ->  Seq Scan on prt1_e_p1 t3
+                                 ->  Seq Scan on prt1_e_p0 t3
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1.a
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                ->  Sort
                      Sort Key: t2.b
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Merge Left Join
                Merge Cond: (t1_1.a = t2_1.b)
                ->  Sort
@@ -949,14 +1454,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
                            ->  Sort
                                  Sort Key: (((t3_1.a + t3_1.b) / 2))
-                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                 ->  Seq Scan on prt1_e_p1 t3_1
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_1.a
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                ->  Sort
                      Sort Key: t2_1.b
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Merge Left Join
                Merge Cond: (t1_2.a = t2_2.b)
                ->  Sort
@@ -965,32 +1470,74 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
                            ->  Sort
                                  Sort Key: (((t3_2.a + t3_2.b) / 2))
-                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                 ->  Seq Scan on prt1_e_p2 t3_2
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_2.a
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                ->  Sort
                      Sort Key: t2_2.b
-                     ->  Seq Scan on prt2_p3 t2_2
-(51 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Merge Left Join
+               Merge Cond: (t1_3.a = t2_3.b)
+               ->  Sort
+                     Sort Key: t1_3.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_3.a + t3_3.b) / 2)) = t1_3.a)
+                           ->  Sort
+                                 Sort Key: (((t3_3.a + t3_3.b) / 2))
+                                 ->  Seq Scan on prt1_e_p3 t3_3
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_3.a
+                                 ->  Seq Scan on prt1_p3 t1_3
+               ->  Sort
+                     Sort Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Merge Left Join
+               Merge Cond: (t1_4.a = t2_4.b)
+               ->  Sort
+                     Sort Key: t1_4.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_4.a + t3_4.b) / 2)) = t1_4.a)
+                           ->  Sort
+                                 Sort Key: (((t3_4.a + t3_4.b) / 2))
+                                 ->  Seq Scan on prt1_e_p4 t3_4
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_4.a
+                                 ->  Seq Scan on prt1_p4 t1_4
+               ->  Sort
+                     Sort Key: t2_4.b
+                     ->  Seq Scan on prt2_p4 t2_4
+(83 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- MergeAppend on nullable column
 EXPLAIN (COSTS OFF)
@@ -998,8 +1545,18 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Merge Left Join
+               Merge Cond: (prt1_p0.a = b)
+               ->  Sort
+                     Sort Key: prt1_p0.a
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
+               ->  Sort
+                     Sort Key: b
+                     ->  Result
+                           One-Time Filter: false
          ->  Merge Left Join
                Merge Cond: (prt1_p1.a = b)
                ->  Sort
@@ -1020,21 +1577,26 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                      Sort Key: prt2_p2.b
                      ->  Seq Scan on prt2_p2
                            Filter: (b > 250)
-(23 rows)
+(33 rows)
 
 SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  b  
------+-----
-   0 |    
-  50 |    
- 100 |    
- 150 |    
- 200 |    
- 250 |    
- 300 | 300
- 350 |    
- 400 |    
-(9 rows)
+  a   |  b  
+------+-----
+ -250 |    
+ -200 |    
+ -150 |    
+ -100 |    
+  -50 |    
+    0 |    
+   50 |    
+  100 |    
+  150 |    
+  200 |    
+  250 |    
+  300 | 300
+  350 |    
+  400 |    
+(14 rows)
 
 -- merge join when expression with whole-row reference needs to be sorted;
 -- partitionwise join does not apply
@@ -1048,175 +1610,2412 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
          Sort Key: t1.a, ((((t1.*)::prt1))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
    ->  Sort
          Sort Key: t2.b, ((((t2.*)::prt2))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt2_p1 t2
-                     ->  Seq Scan on prt2_p2 t2_1
-                     ->  Seq Scan on prt2_p3 t2_2
-(16 rows)
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+(20 rows)
 
 SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.b ORDER BY t1.a;
- a  | b  
-----+----
-  0 |  0
-  6 |  6
- 12 | 12
- 18 | 18
- 24 | 24
-(5 rows)
+  a  |  b  
+-----+-----
+ -24 | -24
+ -18 | -18
+ -12 | -12
+  -6 |  -6
+   0 |   0
+   6 |   6
+  12 |  12
+  18 |  18
+  24 |  24
+(9 rows)
 
 RESET enable_hashjoin;
 RESET enable_nestloop;
---
--- partitioned by multiple columns
---
-CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
-CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
-ANALYZE prt1_m;
-CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
-CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
-ANALYZE prt2_m;
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-                                                             QUERY PLAN                                                             
-------------------------------------------------------------------------------------------------------------------------------------
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p4 t2_3
+               ->  Seq Scan on prt2_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_4
+                           Filter: (b = 0)
+(22 rows)
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p3 t1_2
+                           Filter: (b = 0)
+(21 rows)
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -150 | -0150 | 0 | -0150
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+(7 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (t1_3.a = b)
+         ->  Hash Right Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 |   | 
+ -200 | -0200 |   | 
+ -150 | -0150 | 0 | -0150
+ -100 | -0100 |   | 
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+(21 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -200 | 0 | -0200
+ -100 | 0 | -0100
+  -50 | 0 | -0050
+   50 | 0 | 0050
+  100 | 0 | 0100
+  200 | 0 | 0200
+  250 | 0 | 0250
+  350 | 0 | 0350
+  400 | 0 | 0400
+  500 | 0 | 0500
+  550 | 0 | 0550
+  650 | 0 | 0650
+  700 | 0 | 0700
+(14 rows)
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+                             QUERY PLAN                              
+---------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t3.c
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p0_a on prt1_p0 t3
+                           Index Cond: (a = t2.b)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t2.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_1.a = t2_1.b)
+                           ->  Seq Scan on prt1_p1 t3_1
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1 t2_1
+                                       Filter: (a = 0)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p2_a on prt1_p2 t3_2
+                           Index Cond: (a = t2_2.b)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t3_3
+                           Index Cond: (a = t2_3.b)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_4.a = t2_4.b)
+                           ->  Seq Scan on prt1_p4 t3_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t2_4
+                                       Filter: (a = 0)
+(47 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+  a   | a |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.b
+   ->  Hash Right Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p5 t2_5
+                           Filter: (a = 0)
+(24 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: (t1.a = t2.b)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.a, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
+                     Filter: (a = 0)
+(22 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
+               ->  Seq Scan on prt1_m_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p1
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
+               ->  Seq Scan on prt1_m_p2
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p2
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
+               ->  Seq Scan on prt1_m_p3
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p3
+                           Filter: (c = 0)
+(24 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+  a  | c |  b  | c 
+-----+---+-----+---
+   0 | 0 |   0 | 0
+  50 | 0 |     |  
+ 100 | 0 |     |  
+ 150 | 0 | 150 | 0
+ 200 | 0 |     |  
+ 250 | 0 |     |  
+ 300 | 0 | 300 | 0
+ 350 | 0 |     |  
+ 400 | 0 |     |  
+ 450 | 0 | 450 | 0
+ 500 | 0 |     |  
+ 550 | 0 |     |  
+     |   |  75 | 0
+     |   | 225 | 0
+     |   | 375 | 0
+     |   | 525 | 0
+(16 rows)
+
+--
+-- tests for list partitioned tables.
+--
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
+--
+-- list partitioned by expression
+--
+CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
+ANALYZE plt1_e;
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+                                           QUERY PLAN                                           
+------------------------------------------------------------------------------------------------
+ GroupAggregate
+   Group Key: t1.c, t2.c, t3.c
+   ->  Sort
+         Sort Key: t1.c, t3.c
+         ->  Append
+               ->  Hash Join
+                     Hash Cond: ((t1.c)::text = ltrim(t3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2.b = t1.b) AND ((t2.c)::text = (t1.c)::text))
+                           ->  Seq Scan on plt2_p4 t2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p4 t1
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p4 t3
+               ->  Hash Join
+                     Hash Cond: ((t1_1.c)::text = ltrim(t3_1.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t1_1.b = t2_1.b) AND ((t1_1.c)::text = (t2_1.c)::text))
+                           ->  Seq Scan on plt1_p1 t1_1
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p1 t2_1
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p1 t3_1
+               ->  Hash Join
+                     Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t1_2.b = t2_2.b) AND ((t1_2.c)::text = (t2_2.c)::text))
+                           ->  Seq Scan on plt1_p2 t1_2
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p2 t2_2
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p2 t3_2
+               ->  Hash Join
+                     Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2_3.b = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text))
+                           ->  Seq Scan on plt2_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p3 t3_3
+(41 rows)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p5 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_3
+                                 ->  Seq Scan on plt2_p3 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
  Sort
-   Sort Key: prt1_m_p1.a, prt2_m_p1.b
-   ->  Append
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
-               ->  Seq Scan on prt1_m_p1
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p1
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
-               ->  Seq Scan on prt1_m_p2
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p2
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
-               ->  Seq Scan on prt1_m_p3
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p3
-                           Filter: (c = 0)
-(24 rows)
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
 
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
-  50 | 0 |     |  
- 100 | 0 |     |  
- 150 | 0 | 150 | 0
- 200 | 0 |     |  
- 250 | 0 |     |  
- 300 | 0 | 300 | 0
- 350 | 0 |     |  
- 400 | 0 |     |  
- 450 | 0 | 450 | 0
- 500 | 0 |     |  
- 550 | 0 |     |  
-     |   |  75 | 0
-     |   | 225 | 0
-     |   | 375 | 0
-     |   | 525 | 0
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
 (16 rows)
 
---
--- tests for list partitioned tables.
---
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
---
--- list partitioned by expression
---
-CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1_e;
--- test partition matching with N-way join
+-- semi join
 EXPLAIN (COSTS OFF)
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                   QUERY PLAN                                   
---------------------------------------------------------------------------------
- GroupAggregate
-   Group Key: t1.c, t2.c, t3.c
-   ->  Sort
-         Sort Key: t1.c, t3.c
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
          ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.c = ltrim(t3.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1.b = t2.b) AND (t1.c = t2.c))
-                           ->  Seq Scan on plt1_p1 t1
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.c = ltrim(t3_1.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
-                           ->  Seq Scan on plt1_p2 t1_1
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.c = ltrim(t3_2.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
-                           ->  Seq Scan on plt1_p3 t1_2
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p3 t3_2
-(32 rows)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Seq Scan on plt2_p2 t2_2
+                                 ->  Seq Scan on plt2_p3 t2_3
+(22 rows)
 
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  75.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 123.0000000000000000 | 123.0000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 225.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 273.0000000000000000 | 273.0000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 375.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 423.0000000000000000 | 423.0000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 525.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 573.0000000000000000 | 573.0000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1241,22 +4040,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
 --------------------------------------------------
  Hash Left Join
    Hash Cond: (t2.b = a)
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t3.a = t2.b)
-               ->  Seq Scan on prt1_p1 t3
-               ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
-         ->  Hash Join
-               Hash Cond: (t3_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t3_1
-               ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
-         ->  Hash Join
-               Hash Cond: (t3_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t3_2
-               ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
+   ->  Hash Join
+         Hash Cond: (t3.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t3
+               ->  Seq Scan on prt1_p1 t3_1
+               ->  Seq Scan on prt1_p2 t3_2
+               ->  Seq Scan on prt1_p3 t3_3
+               ->  Seq Scan on prt1_p4 t3_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
    ->  Hash
          ->  Result
                One-Time Filter: false
@@ -1271,16 +4070,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 --
 -- tests for hash partitioned tables.
@@ -1356,41 +4161,9 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  273.0000000000000000 | 273.0000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
 (6 rows)
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
- Sort
-   Sort Key: t1.a
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
-               ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
-               ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
-                           Filter: (b = 0)
-(21 rows)
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
@@ -1405,26 +4178,24 @@ SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c
    Sort Key: t1.c
    ->  HashAggregate
          Group Key: t1.c, t2.c
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t2.c = t1.c)
-                     ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p1 t1
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on plt1_p4 t1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_1.c = t1_1.c)
-                     ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_p2 t1_1
+                           ->  Seq Scan on plt1_p1 t1_1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_2.c = t1_2.c)
-                     ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Seq Scan on plt1_p2 t1_2
                                  Filter: ((a % 25) = 0)
-(23 rows)
+                           ->  Seq Scan on plt1_p3 t1_3
+                                 Filter: ((a % 25) = 0)
+(21 rows)
 
 --
 -- multiple levels of partitioning
@@ -1826,64 +4597,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
  Hash Join
    Hash Cond: (t1.a = t2.a)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt4_n_p1 t2
                ->  Seq Scan on prt4_n_p2 t2_1
                ->  Seq Scan on prt4_n_p3 t2_2
-(11 rows)
+(13 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a and t1.a = t3.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                        QUERY PLAN                        
+----------------------------------------------------------
  Hash Join
-   Hash Cond: (t2.a = t1.a)
+   Hash Cond: (t3.b = t1.a)
    ->  Append
-         ->  Seq Scan on prt4_n_p1 t2
-         ->  Seq Scan on prt4_n_p2 t2_1
-         ->  Seq Scan on prt4_n_p3 t2_2
+         ->  Seq Scan on prt2_p0 t3
+         ->  Seq Scan on prt2_p1 t3_1
+         ->  Seq Scan on prt2_p2 t3_2
+         ->  Seq Scan on prt2_p3 t3_3
+         ->  Seq Scan on prt2_p4 t3_4
+         ->  Seq Scan on prt2_p5 t3_5
    ->  Hash
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.a = t3_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_2.b)
-                     ->  Seq Scan on prt1_p3 t1_2
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3 t3_2
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt4_n_p1 t2
+                           ->  Seq Scan on prt4_n_p2 t2_1
+                           ->  Seq Scan on prt4_n_p3 t2_2
 (23 rows)
 
 -- partitionwise join can not be applied if there are no equi-join conditions
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (t1.a < b)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (t1.a < b)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (t1.a < b)
-(12 rows)
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+(16 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partitionwise join
@@ -1969,16 +4746,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
-                     ->  Seq Scan on plt1_p1 t3
-                     ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3
+                     ->  Seq Scan on plt1_p1 t3_1
+                     ->  Seq Scan on plt1_p2 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partitionwise join can not be applied for a join between list and range
 -- partitioned table
@@ -1989,14 +4767,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
  Hash Full Join
    Hash Cond: ((t2.c)::text = (t1.c)::text)
    ->  Append
-         ->  Seq Scan on prt1_p1 t2
-         ->  Seq Scan on prt1_p2 t2_1
-         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p0 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p2 t2_2
+         ->  Seq Scan on prt1_p3 t2_3
+         ->  Seq Scan on prt1_p4 t2_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt1_n_p1 t1
                ->  Seq Scan on prt1_n_p2 t1_1
-(10 rows)
+(12 rows)
 
 -- partitionwise join can not be applied if only one of joining table has
 -- default partition
@@ -2012,16 +4792,23 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
    ->  Hash Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index c1c9859651..0e884835fb 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -10,25 +10,39 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -67,11 +81,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -93,20 +115,30 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 
 EXPLAIN (COSTS OFF)
@@ -169,6 +201,128 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -193,28 +347,79 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
 -- test partition matching with N-way join
@@ -222,6 +427,175 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
@@ -267,27 +641,18 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
 ALTER TABLE plt2 DETACH PARTITION plt2_p3;
 ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
 ANALYZE plt2;
-
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.a % 25 = 0 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
 --
 -- multiple levels of partitioning
 --
-- 
2.16.4

#55Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Dmitry Dolgov (#54)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Mon, Nov 26, 2018 at 9:03 AM Dmitry Dolgov <9erthalion6@gmail.com> wrote:

I've noticed, that this patch set is outdated, so here is the rebased version.

This turned red on cfbot because I turned on -Werror:

partbounds.c: In function ‘partition_bounds_merge’:
partbounds.c:3619:43: error: ‘outer_part’ may be used uninitialized in
this function [-Werror=maybe-uninitialized]
merged_index = map_and_merge_partitions(outer_pmap, outer_mmap,
^
partbounds.c:3475:6: note: ‘outer_part’ was declared here
int outer_part;
^
cc1: all warnings being treated as errors

--
Thomas Munro
http://www.enterprisedb.com

#56Dmitry Dolgov
9erthalion6@gmail.com
In reply to: Thomas Munro (#55)
4 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Sun, Nov 25, 2018 at 10:07 PM Thomas Munro <thomas.munro@enterprisedb.com> wrote:

This turned red on cfbot because I turned on -Werror:

partbounds.c: In function ‘partition_bounds_merge’:
partbounds.c:3619:43: error: ‘outer_part’ may be used uninitialized in
this function [-Werror=maybe-uninitialized]
merged_index = map_and_merge_partitions(outer_pmap, outer_mmap,
^
partbounds.c:3475:6: note: ‘outer_part’ was declared here
int outer_part;
^
cc1: all warnings being treated as errors

Thanks, I've messed this up too - rebased a wrong branch, the correct one
doesn't have this code already. Sorry for that, and here is the proper version
of this patch.

Attachments:

0002-Targetlist-of-a-child-join-is-produced-by-translating-v14.patchapplication/octet-stream; name=0002-Targetlist-of-a-child-join-is-produced-by-translating-v14.patchDownload
From bd68b5832f5d0812767223a332eac26c1cf0513b Mon Sep 17 00:00:00 2001
From: erthalion <9erthalion6@gmail.com>
Date: Tue, 11 Sep 2018 21:09:44 +0200
Subject: [PATCH 2/5] Targetlist of a child-join is produced by translating
 that of parent join

The next patch adds more general partition matching algorithm for
partition-wise join. With that change, the first pair of joining
relations for the parent joinrel may not produce a partition-wise join
because of the restrictions in partition_bounds_merge(), to be added
in the next patch.  Hence the first pair of joining relations, which
is used to build the child join and presented to
build_child_join_rel(), for the child joinrel does not correspond to
the first pair of joining relations for the parent joinrel. The
targetlist built using different pairs have the targetlist entries
arranged in different order. An appendrel expects that all its
children have their targetlists ordered in the same fashion.  Hence
translate the parent's targetlist so that parent and child joinrels
have their targetlists in sync.

Basic partition-wise join commit
f49842d1ee31b976c681322f76025d7732e860f3 modified
build_joinrel_tlist() to build targetlist for child-join. With the
above changes we don't need it anymore.  Similarly, the same commit
added code to set_append_rel_size() to compute attr_needed for a
child-join relation. That change too is not needed with the above
change. This commit reverts those two changes.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/optimizer/util/relnode.c | 20 ++++++++++++++++++++
 1 file changed, 20 insertions(+)

diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 39f5729b91..2f1137c13d 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -802,6 +802,19 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 
 	appinfos = find_appinfos_by_relids(root, joinrel->relids, &nappinfos);
 
+	/*
+	 * The first pair of joining relations for the parent joinrel may not
+	 * produce a partition-wise join because of the restrictions in
+	 * partition_bounds_merge(). Hence the first pair of joining relations,
+	 * which is used to build the child join and presented here, for the child
+	 * joinrel does not correspond to the first pair of joining relations for
+	 * the parent joinrel. The targetlist built using different pairs have the
+	 * targetlist nodes arranged in different order. An appendrel expects that
+	 * all its children have their targetlists ordered in the same fashion.
+	 * Hence translate the parent's targetlist so that parent and child
+	 * joinrels have their targetlists in sync.
+	 */
+	joinrel->reltarget = copy_pathtarget(parent_joinrel->reltarget);
 	/* Set up reltarget struct */
 	build_child_join_reltarget(root, parent_joinrel, joinrel,
 							   nappinfos, appinfos);
@@ -907,6 +920,12 @@ build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 	Relids		relids = joinrel->relids;
 	ListCell   *vars;
 
+	/*
+	 * We only see parent joins. Targetlist of a child-join is computed by
+	 * translating corresponding parent join's targetlist.
+	 */
+	Assert(joinrel->reloptkind == RELOPT_JOINREL);
+
 	foreach(vars, input_rel->reltarget->exprs)
 	{
 		Var		   *var = (Var *) lfirst(vars);
@@ -934,6 +953,7 @@ build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 
 		/* Is it still needed above this joinrel? */
 		ndx = var->varattno - baserel->min_attr;
+
 		if (bms_nonempty_difference(baserel->attr_needed[ndx], relids))
 		{
 			/* Yup, add it to the output */
-- 
2.16.4

0003-Partition-wise-join-for-1-1-1-0-0-1-partition-matching-v14.patchapplication/octet-stream; name=0003-Partition-wise-join-for-1-1-1-0-0-1-partition-matching-v14.patchDownload
From 7adc97a2c1849d85cdce9dac2047d27347396924 Mon Sep 17 00:00:00 2001
From: erthalion <9erthalion6@gmail.com>
Date: Tue, 11 Sep 2018 21:10:46 +0200
Subject: [PATCH 3/5] Partition-wise join for 1:1, 1:0, 0:1 partition matching.

Earlier version of partition-wise join implementation allowed
partition-wise join between two relations with exactly same partition
bounds. This commit allows partition-wise join to be applied under
following conditions

1. the partition bounds of joining relations are such that rows from
given partition on one side join can join with rows from maximum one
partition on the other side i.e. bounds of a given partition on one
side match/overlap with those of maximum one partition on the other
side. If the mapping happens to be m:n where m > 1 or n > 1, we have
to gang multiple partition relations together into a single relation.
This means that we have to add simple relations during join
processing, something which is not supported right now.  ALso, in such
a case, different pairs of joining relations can produce different
partition bounds for the same join relation, which again is not
supported right now.

2. For every partition on outer side that can contribute to the result
of an OUTER join, there exists at least one (taken along with item 1,
it means exactly one)  matching partition on the inner side. To
support partition-wise join when the inner matching partition doesn't
exist, we have to add a dummy base relation corresponding to the
non-existent inner partition. We don't have support to add base
relations during join processing.

3. For hash partitioned tables however, we support partition-wise join
only when the bounds exactly match. For hash partitioning it's unusual
to have missing partitions and hence generic partition matching is not
required.

With advanced partition matching, we won't be able to apply
partition-wise join when there are missing matching partitions. So for
a given N-way join, some sub-joins may not be partitioned, while the
overall N-way join is partitioned. We can not try partition-wise join
when one or both of the joining relations are not partitioned in one
of the pairs of joining relation. Skip partition-wise join for that
pair.

An example is A IJ B LJ C where B has an extra partition compared to A
and C. Join B LJ C requires dummy relation to join the extra partition
from B to a missing partition from C, and hence can not use
partition-wise join right now and hence BC is not partitioned.  The
extra partition in B gets eliminated in A IJ B and thus it can use
partition-wise join and is partitioned. Hence (AB)C can use
partition-wise join but not A(BC).

Not every pair of joining relation (including the one presented to
build_joinrel_partition_info()) for the same joinrel can use
partition-wise join or has both the relations partitioned. Hence we
calculate the partition bounds for the join relation in
try_partition_wise_join() instead of doing it here.

The partition bounds produced for the first pair that can use
partition-wise are saved in the RelOptInfo of the joinrel. Partition
bounds produced for subsequent pairs should match that produced by the
first pair.

Support for default partition in list partitioned tables
========================================================

When a list value is present in one of the joining relations and not
the other, and the other relation has default partition, match (join)
the partition containing that list value with the default partition.
If there are multiple matches, we can not proceed with the
partition-wise join similar to the case of matching non-default
partition. If the default partition happens to be on the outer side of
the join, the resulting join partition acts as a default partition as
it will contain all the values from the default partition. If
the partition containing the list value happens to be on the outer
side of the join, the resulting join partition is associated with the
list value, since no other partition key value from the default
partition makes it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join.  Such a
default partition may get joined to a partition from the inner side,
if inner side has a list value that is not present in the outer side.
But if such a join partner does not exist, we won't be able to use
partition-wise join since there is no (dummy) relation on the inner
side to join to.

Support for matching default partition in range partitioned tables
==================================================================

When a range is present in one of the joining relations and not the
other, and the other relation has default partition, match (join) the
partition corresponding range with the default partition. If two
ranges overlap but their ranges don't match exactly, the
non-overlapping portion from one range may join the previous
(non-overlapping portion near the lower bound, if exists), next
(non-overlapping portion near the upper bound, if exists) ranges from
the other relation or default partition from the other relation. This
causes multiple partitions on the other side to be joined with a
single partition on the first side; a case we don't support. Any range
from one side which doesn't overlap with any range on the other side,
may find its join partner in the default partition and the
corresponding range partition will need to be joined with the default
partition.  If the default partition happens to be on the outer side
of the join, the resulting join partition acts as a default partition
as it will contain all the values from the default partition. If the
non-partition corresponding to the range happens to be on the outer
side of the join, the resulting join partition is associated with that
range, since partition key values from the default partition outside
that range won't make it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join.  Such a
default partition may get joined to a non-default partition from the
inner side, if inner side has a range missing in the outer side.  But
if such a join partner does not exist, we won't be able to use
partition-wise join since there is no (dummy) relation on the inner
side to join to.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/optimizer/path/joinrels.c |   95 +-
 src/backend/optimizer/util/relnode.c  |   33 +-
 src/backend/partitioning/partbounds.c | 1676 ++++++++++++++++++++++++++++++++-
 src/include/partitioning/partbounds.h |    5 +
 4 files changed, 1763 insertions(+), 46 deletions(-)

diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index d3d21fed5d..e476079143 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1312,25 +1312,31 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 					   RelOptInfo *joinrel, SpecialJoinInfo *parent_sjinfo,
 					   List *parent_restrictlist)
 {
-	int			nparts;
 	int			cnt_parts;
+	PartitionScheme part_scheme;
+	PartitionBoundInfo join_boundinfo;
+	List	   *parts1;
+	List	   *parts2;
+	ListCell   *lc1;
+	ListCell   *lc2;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1343,32 +1349,63 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	 */
 	Assert(joinrel->part_scheme == rel1->part_scheme &&
 		   joinrel->part_scheme == rel2->part_scheme);
+	part_scheme = joinrel->part_scheme;
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * Get the list of matching partitions to be joined along with the
+	 * partition bounds of the join relation. Because of the restrictions
+	 * imposed by partition matching algorithm, not every pair of joining
+	 * relations for this join will be able to use partition-wise join. But all
+	 * those pairs which can use partition-wise join will produce the same
+	 * partition bounds for the join relation.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	join_boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											part_scheme->partsupfunc,
+											part_scheme->partcollation,
+											rel1, rel2,
+											parent_sjinfo->jointype,
+											&parts1, &parts2);
+
+	if (join_boundinfo == NULL)
+		return;
 
-	nparts = joinrel->nparts;
+	if (joinrel->boundinfo == NULL)
+	{
+		Assert(joinrel->nparts == 0 && joinrel->part_rels == NULL);
+		joinrel->boundinfo = join_boundinfo;
+		joinrel->nparts = list_length(parts1);
+		Assert(joinrel->nparts == list_length(parts2));
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) *
+									joinrel->nparts);
+	}
+	else
+	{
+		Assert(partition_bounds_equal(part_scheme->partnatts,
+									  part_scheme->parttyplen,
+									  part_scheme->parttypbyval,
+									  join_boundinfo, joinrel->boundinfo));
+		/*
+		 * Every pair of joining relations should result in the same number
+		 * of child-joins.
+		 */
+		Assert(joinrel->nparts == list_length(parts1));
+		Assert(joinrel->nparts == list_length(parts2));
+		Assert(joinrel->part_rels);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	cnt_parts = 0;
+	forboth(lc1, parts1, lc2, parts2)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
+		int			part1 = lfirst_int(lc1);
+		int			part2 = lfirst_int(lc2);
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1376,6 +1413,10 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		Assert(part1 >= 0 && part2 >= 0);
+		child_rel1 = rel1->part_rels[part1];
+		child_rel2 = rel2->part_rels[part2];
+
 		/* We should never try to join two overlapping sets of rels. */
 		Assert(!bms_overlap(child_rel1->relids, child_rel2->relids));
 		child_joinrelids = bms_union(child_rel1->relids, child_rel2->relids);
@@ -1409,12 +1450,24 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 			joinrel->part_rels[cnt_parts] = child_joinrel;
 		}
 
+		/*
+		 * For every pair of joining relations, the set of matching partitions
+		 * would change. However, the base relation partitions constituting
+		 * the given child should remain same for all the joining pairs. Since
+		 * the order in which children are stored in the array of child-joins,
+		 * depends upon partition bounds of the join, which are same for all
+		 * the joining pairs, every joining pair yields the child-joins in the
+		 * same order.
+		 */
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
 
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
+		cnt_parts++;
 	}
+
+	Assert(cnt_parts == joinrel->nparts);
 }
 
 /*
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 2f1137c13d..4f15cd8e8b 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1627,7 +1627,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1640,24 +1640,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1669,17 +1651,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 0af3372cdf..771520379d 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -65,6 +65,12 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+typedef struct PartitionMap
+{
+	int from;
+	int to;
+} PartitionMap;
+
 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);
@@ -106,7 +112,58 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 static List *get_range_nulltest(PartitionKey key);
 static bool partition_hbounds_equal(PartitionBoundInfo b1,
 									PartitionBoundInfo b2);
-
+static PartitionBoundInfo partition_range_bounds_merge(
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 List **outer_parts, List **inner_parts,
+							 JoinType jointype, int partnatts,
+							 FmgrInfo *supfuncs, Oid *collations);
+static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype);
+static PartitionBoundInfo partition_hash_bounds_merge(RelOptInfo *outer_rel,
+							RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype);
+static void generate_matching_part_pairs(PartitionMap *outer_maps,
+										 PartitionMap *inner_maps,
+										 int nparts1, int nparts2,
+										 JoinType jointype, int nparts,
+										 List **parts1, List **parts2);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int map_and_merge_partitions(PartitionMap *outer_maps,
+										PartitionMap *inner_maps,
+										int index1, int index2, int *next_index);
+static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *collations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2);
+static bool partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
+						   Oid *collations, PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, int *ub_cmpval,
+						   int *lb_cmpval);
+static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes);
+static bool merge_default_partitions(PartitionBoundInfo outer_bi,
+						 			 PartitionBoundInfo inner_bi,
+						 			 PartitionMap *outer_maps,
+						 			 PartitionMap *inner_maps,
+						 			 JoinType jointype,
+									 int *next_index, int *default_index);
+static bool merge_null_partitions(PartitionBoundInfo outer_bi,
+								  PartitionBoundInfo inner_bi,
+								  PartitionMap *outer_maps,
+								  PartitionMap *inner_maps,
+								  JoinType jointype,
+								  int *next_index, int *null_index,
+								  int *default_index);
 
 /*
  * get_qual_from_partbound
@@ -2941,3 +2998,1620 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * The function produces the partition bounds for a join between two relations
+ * whose partition bounds are given. The function also returns two lists of
+ * partition indexes one for each of the joining relations. Both the lists
+ * contain the same number of elements. The partition indexes at the same
+ * positions in the lists indicate the pair partitions, one from each side, to
+ * be joined and the position itself corresponds to the index of partition
+ * produced by that child-join in the partitioned join.
+ *
+ * The function returns NULL if we can not find the matching pair of
+ * partitions. This happens if 1. multiple partitions on one side match with
+ * one partition on the other side. 2. a given partition on the outer side
+ * doesn't have a matching partition on the inner side. We can not support the
+ * first case since we don't have a way to represent multiple partitions as a
+ * single relation (RelOptInfo) and then perform join using the ganged
+ * relation. We can not support the second case since the missing inner
+ * partition needs to be represented as an empty relation and we don't have a
+ * way to introduce empty relation during join planning after creating paths
+ * for all the base relations.
+ */
+extern PartitionBoundInfo
+partition_bounds_merge(int partnatts, FmgrInfo *partsupfunc,
+					   Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
+{
+	PartitionBoundInfo 	merged_bounds;
+	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
+					   	inner_binfo = inner_rel->boundinfo;
+	char				strategy = outer_binfo->strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partsupfunc,
+														partcollation,
+														outer_rel, inner_rel,
+														outer_parts, inner_parts,
+														jointype);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(outer_rel, inner_rel,
+														 outer_parts, inner_parts,
+														 jointype, partnatts,
+														 partsupfunc,
+														 partcollation);
+			break;
+
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = partition_hash_bounds_merge(outer_rel, inner_rel,
+														outer_parts, inner_parts,
+														jointype);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	Assert((*outer_parts == NIL || *inner_parts != NIL) &&
+		   (*inner_parts == NIL || *outer_parts != NIL));
+
+	return merged_bounds;
+}
+
+/*
+ * partition_get_range_bounds
+ *
+ * Given the index of lower bound in datums array, return lower and upper
+ * bounds and the index of the partition with that lower bound.
+ */
+static int
+partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
+						   PartitionRangeBound *lower,
+						   PartitionRangeBound *upper)
+{
+	int			part_index;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The lower bound should correspond to a valid partition. */
+	part_index = bi->indexes[lb_index + 1];
+	Assert(part_index >= 0);
+
+	lower->kind = bi->kind[lb_index];
+	lower->datums = bi->datums[lb_index];
+	lower->lower = true;
+	upper->kind = bi->kind[lb_index + 1];
+	upper->datums = bi->datums[lb_index + 1];
+	upper->lower = false;
+
+	return part_index;
+}
+
+/*
+ * partition_range_get_next_lb_index
+ *
+ * Given the index of lower bound in datums array return the
+ * index of lower bound of the next partition. When the given index corresponds
+ * to the last partition, return number of datums (ndatums).
+ */
+static int
+partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
+{
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The partition index corresponding to the upper bound should be valid. */
+	Assert(bi->indexes[lb_index + 1] >= 0);
+
+	/*
+	 * If there are no bounds left beyond the upper bound, we have reached the
+	 * last partition.
+	 */
+	if (lb_index + 2 < bi->ndatums)
+	{
+		/*
+		 * If the bound next to the upper bound corresponds to no partition,
+		 * that's the next lower bound of the next partition. Otherwise, the
+		 * current upper bound is the lower bound of the next partition.
+		 */
+		if (bi->indexes[lb_index + 2] < 0)
+			return lb_index + 2;
+		else
+			return lb_index + 1;
+	}
+	else
+		return bi->ndatums;
+}
+
+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *partcollations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2)
+{
+	return partition_rbound_cmp(partnatts, partsupfunc, partcollations,
+								bound1->datums, bound1->kind, bound1->lower,
+								bound2);
+}
+
+/*
+ * partition_range_cmp
+ *
+ * Compare the bounds of two range partitions. Set ub_cmpval <, = or > 0, if the
+ * first partition's upper bound is lower than, equal to or higher than the
+ * second partition's upper bound resp. Similarly set lb_cmpval <, =  or > 0,
+ * if the first partition's lower bound is lower than, equal to or higher than
+ * the second partition's lower bound resp.
+ *
+ * Return true, if the ranges overlap, otherwise return false.
+ */
+static bool
+partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
+					PartitionRangeBound *lower_bound1,
+					PartitionRangeBound *upper_bound1,
+					PartitionRangeBound *lower_bound2,
+					PartitionRangeBound *upper_bound2, int *ub_cmpval,
+					int *lb_cmpval)
+{
+	bool		overlap;
+
+	/*
+	 * Compare upper bound of the first partition with the lower bound of the
+	 * second and vice-versa. If lower bound is higher than the upper bound,
+	 * the partitions are not overlapping. All other cases indicate overlapping
+	 * partitions.
+	 */
+	if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+								  lower_bound1, upper_bound2) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = 1;
+		*lb_cmpval = 1;
+	}
+	else if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+									   lower_bound2, upper_bound1) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = -1;
+		*lb_cmpval = -1;
+	}
+	else
+	{
+		overlap = true;
+		*ub_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, upper_bound1,
+											   upper_bound2);
+		*lb_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, lower_bound1,
+											   lower_bound2);
+	}
+
+	return overlap;
+}
+
+/*
+ * partition_range_merge
+ *
+ * Merge the partition bounds of given two partitions such that the join
+ * between the given two partitions fits merged bounds.
+ *
+ * "merged_upper" will be set to one of the given upper bounds and
+ * "merged_lower" will be set to one of the given lower bounds.
+ */
+static void
+partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
+					  Oid *partcollations, JoinType jointype,
+					  PartitionRangeBound *left_lb,
+					  PartitionRangeBound *left_ub,
+					  PartitionRangeBound *right_lb,
+					  PartitionRangeBound *right_ub,
+					  PartitionRangeBound **merged_lb,
+					  PartitionRangeBound **merged_ub)
+{
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = left_ub;
+			*merged_lb = left_lb;
+			break;
+
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) < 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) > 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		case JOIN_FULL:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) > 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) < 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+
+	return;
+}
+
+/*
+ * Add the lower bound of the next range to the list of bounds, if the lower
+ * bound is higher or equal to the previous upper bound. If successful return
+ * true, otherwise false.
+ */
+static bool
+partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
+							  Oid *partcollations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		prev_ub.datums = llast(*merged_datums);
+		prev_ub.kind = llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  next_lb_datums, next_lb_kind, false,
+									  &prev_ub);
+	}
+
+	/*
+	 * The lower bound is lower than the last upper bound, thus does not fit
+	 * the bounds created so far and hence can not be merged with the existing
+	 * bounds.
+	 */
+	if (cmpval < 0)
+		return false;
+
+	/*
+	 * Add bounds of the new merged partition. If the next lower bound is
+	 * higher than the last upper bound, add new range with index
+	 * corresponding to the lower bound as -1. If the merged lower bound
+	 * is same as the last merged upper bound, the last upper bound will be
+	 * reused as the lower bound of the next range.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, next_lb_datums);
+		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	return true;
+}
+
+/*
+ * handle_missing_partition
+ *
+ * If a range appears in one of the joining relations but not the other, a row
+ * in the corresponding partition will not have any join partner in the other
+ * relation, unless the other relation has a default partition. If a given list
+ * value is present in one joining relation but not the other, the default
+ * partition on the other side may contain that value.
+ *
+ * In both these cases, such an extra partition forms a joining pair with the
+ * default partition, if any,  on the other side.
+ *
+ * If the default partition happens to be on the outer side of the join, the
+ * resultant partition will act as the default partition of the join relation.
+ * Otherwise the resultant partition will be associated with the range.
+ *
+ * When the default partition is not present in the other relation, the rows in
+ * the extra partition will be included in the bounds of the join result, if it
+ * appears on the outer side of the join, since all rows from the outer side
+ * are included in the join result.
+ *
+ * This function handles all these cases.
+ *
+ * maps_with_missing and missing_side_default are the partition maps (See
+ * partition_range/list_bounds_merge() for details) and the index of default
+ * partition respectively corresponding the side with missing partition.
+ *
+ * maps_with_extra and extra_part are the partition maps (See
+ * partition_range/list_bounds_merge() for details) and the index of extra
+ * partition respectively corresponding to the side with the extra partition.
+ *
+ * It returns true if the matching succeeds, otherwise returns false.
+ */
+static bool
+handle_missing_partition(PartitionMap *maps_with_missing,
+						 PartitionMap *maps_with_extra,
+						 int missing_side_default,
+						 int extra_part,
+						 bool missing_side_outer,
+						 bool missing_side_inner,
+						 int *next_index, int *default_index,
+						 int *merged_index)
+{
+	bool missing_has_default = (missing_side_default != -1);
+
+	if (missing_has_default)
+	{
+		*merged_index = map_and_merge_partitions(maps_with_missing,
+												 maps_with_extra,
+												 missing_side_default,
+												 extra_part,
+												 next_index);
+		if (*merged_index < 0)
+			return false;
+
+		if (missing_side_outer)
+		{
+			/*
+			 * Default partition on the outer side forms the default
+			 * partition of the join result.
+			 */
+			if (*default_index < 0)
+				*default_index = *merged_index;
+			else if(*default_index != *merged_index)
+			{
+				/*
+				 * Ended up with default partition on the outer side
+				 * being joined with multiple partitions on the inner
+				 * side. We don't support this case.
+				 */
+				return false;
+			}
+
+			/*
+			 * Since the merged partition acts as a default partition, it
+			 * doesn't need a separate index.
+			 */
+			*merged_index = -1;
+		}
+	}
+	else if (missing_side_inner)
+	{
+		/*
+		 * If this partition has already been mapped (say because we
+		 * found an overlapping range earlier), we know where does it
+		 * fit in the join result. Nothing to do in that case. Else
+		 * create a new merged partition.
+		 */
+		PartitionMap *extra_map = &maps_with_extra[extra_part];
+		if (extra_map->to < 0)
+		{
+			extra_map->to = *next_index;
+			*next_index = *next_index + 1;
+			*merged_index = extra_map->to;
+		}
+	}
+
+	return true;
+}
+
+static PartitionMap*
+init_partition_map(RelOptInfo *rel)
+{
+	int i, nparts = rel->nparts;
+	PartitionMap *map;
+
+	map = (PartitionMap *) palloc(sizeof(PartitionMap) * nparts);
+
+	for (i = 0; i < nparts; i++)
+	{
+		map[i].from = -1;
+		map[i].to = -1;
+	}
+
+	return map;
+}
+
+/*
+ * Allocate and initialize partition maps. We maintain four maps, two maps
+ * for each joining relation. pmap[i] gives the partition from the other
+ * relation which would join with ith partition of the given relation.
+ * Partition i from the given relation will join with partition pmap[i]
+ * from the other relation to produce partition mmap[i] of the join (merged
+ * partition).
+ *
+ * pmap[i] = -1 indicates that ith partition of a given relation does not
+ * have a matching partition from the other relation.
+ *
+ * mmap[i] = -1 indicates that ith partition of a given relation does not
+ * contribute to the join result. That can happen only when the given
+ * relation is the inner relation and it doesn't have a matching partition
+ * from the outer relation, hence pmap[i] should be -1.
+ *
+ * In case of an outer join, every partition of the outer join will appear
+ * in the join result, and thus has mmap[i] set for all i. But it's not
+ * necessary that every partition on the outer side will have a matching
+ * partition on the inner side. In such a case, we end up with pmap[i] = -1
+ * and mmap[i] != -1.
+ */
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 List **outer_parts, List **inner_parts,
+							 JoinType jointype, int partnatts,
+							 FmgrInfo *partsupfuncs, Oid *partcollations)
+
+{
+	PartitionMap *outer_maps = NULL;
+	PartitionMap *inner_maps = NULL;
+	int			outer_part = 0;
+	int			inner_part = 0;
+	PartitionBoundInfo merged_bounds = NULL;
+	int			outer_lb_index;
+	int			inner_lb_index;
+	int			next_index;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	List	   *merged_kinds = NIL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			inner_default = inner_bi->default_index;
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	outer_maps = init_partition_map(outer_rel);
+	inner_maps = init_partition_map(inner_rel);
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = 0;
+	inner_lb_index = 0;
+	next_index = 0;
+	while (outer_lb_index < outer_bi->ndatums ||
+		   inner_lb_index < inner_bi->ndatums)
+	{
+		PartitionRangeBound outer_lb, outer_ub,
+							inner_lb, inner_ub,
+							*merged_lb = NULL,
+							*merged_ub = NULL;
+
+		int			merged_index = -1;
+		bool		overlap;
+		bool		finished_outer = false;
+		bool		finished_inner = false;
+
+		/* Result of bounds comparison per partition_rbound_cmp(). */
+		int			ub_cmpval;	/* Upper bounds comparison result. */
+		int			lb_cmpval;	/* Lower bounds comparison result. */
+
+		/* Get the range bounds of the next pair of partitions. */
+		if (outer_lb_index < outer_bi->ndatums)
+			outer_part = partition_get_range_bounds(outer_bi, outer_lb_index,
+												&outer_lb, &outer_ub);
+		else
+			finished_outer = true;
+
+		if (inner_lb_index < inner_bi->ndatums)
+			inner_part = partition_get_range_bounds(inner_bi, inner_lb_index,
+												&inner_lb, &inner_ub);
+		else
+			finished_inner = true;
+
+		Assert(!finished_outer || !finished_inner);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (finished_outer)
+		{
+			overlap = false;
+			ub_cmpval = 1;
+			lb_cmpval = 1;
+		}
+		else if (finished_inner)
+		{
+			overlap = false;
+			ub_cmpval = -1;
+			lb_cmpval = -1;
+		}
+		else
+			overlap = partition_range_cmp(partnatts, partsupfuncs, partcollations,
+										  &outer_lb, &outer_ub, &inner_lb,
+										  &inner_ub, &ub_cmpval, &lb_cmpval);
+
+		if (overlap)
+		{
+			/*
+			 * The rows from overlapping portion of ranges on both sides may
+			 * join, hence the corresponding pair of partitions form a joining
+			 * pair. Match them and produce the bounds of the joint partition
+			 * and its index by merging the bounds according to the type of
+			 * join.
+			 */
+			partition_range_merge(partnatts, partsupfuncs, partcollations,
+								  jointype, &outer_lb, &outer_ub, &inner_lb,
+								  &inner_ub, &merged_lb, &merged_ub);
+
+			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+													outer_part, inner_part,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				/* Failed to match the partitions. */
+				return NULL;
+			}
+
+			/*
+			 * If the ranges overlap but don't exactly match, a row from
+			 * non-overlapping portion of the range from one side of join may
+			 * find its join partner in the previous or next overlapping
+			 * partition or default partition on the other side , if such a
+			 * partition exists. All those cases, if true, will cause one
+			 * partition from that side to match at least two partitions on the
+			 * other side; a case that we do not support now. Previous
+			 * partition has been delt with in the previous iteration of this
+			 * loop, next partition will be delt in the next iteration. We will
+			 * deal with the default partition here.
+			 */
+			if ((lb_cmpval < 0 && inner_has_default) ||
+				/* Non-overlapping range on the lower side of outer range. */
+				(lb_cmpval > 0 && outer_has_default) ||
+				/* Non-overlapping range on the lower side of inner range. */
+				(ub_cmpval < 0 && outer_has_default) ||
+				/* Non-overlapping range on the upper side of inner range. */
+				(ub_cmpval > 0 && inner_has_default))
+				/* Non-overlapping range on the upper side of outer range. */
+				return NULL;
+		}
+
+		if (ub_cmpval == 0)
+		{
+			/* Upper bounds of both the ranges match. */
+			Assert(overlap);
+
+			/* Move to the next pair of partitions. */
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the inner side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &outer_lb;
+				merged_ub = &outer_ub;
+
+				/*
+				 * For a FULL join, inner relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the inner relation acts as
+				 * INNER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_inner = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_outer = (jointype == JOIN_FULL);
+				if (!handle_missing_partition(inner_maps,
+											  outer_maps,
+											  inner_default,
+											  outer_part,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the outer side. */
+			Assert(!finished_outer);
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+		}
+		else
+		{
+			Assert(ub_cmpval > 0);
+
+			/* Upper bound of outer range higher than that of the inner. */
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the outer side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &inner_lb;
+				merged_ub = &inner_ub;
+
+				/*
+				 * For a FULL join, outer relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the outer relation acts as
+				 * OUTER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_outer = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_inner = (jointype == JOIN_FULL);
+
+				if (!handle_missing_partition(outer_maps,
+											  inner_maps,
+											  outer_default,
+											  inner_part,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the inner side. */
+			Assert (!finished_inner);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+
+		if (merged_index < 0)
+		{
+			/* We didn't find a new merged partition. */
+			continue;
+		}
+
+		/*
+		 * We have a valid partition index for the next partition of join. The
+		 * partition should have valid range.
+		 */
+		Assert(merged_lb && merged_ub);
+
+		/* Try merging new lower bound with the last upper bound. */
+		if (!partition_range_merge_next_lb(partnatts, partsupfuncs,
+										   partcollations,
+										   merged_lb->datums,
+										   merged_lb->kind, &merged_datums,
+										   &merged_kinds, &merged_indexes))
+			return NULL;
+
+		/* Add upper bound with the merged partition index. */
+		merged_datums = lappend(merged_datums, merged_ub->datums);
+		merged_kinds = lappend(merged_kinds, merged_ub->kind);
+		merged_indexes = lappend_int(merged_indexes, merged_index);
+	}
+
+	if (!merge_default_partitions(outer_bi, inner_bi,
+										  outer_maps, inner_maps,
+										  jointype, &next_index,
+										  &default_index))
+		return NULL;
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_maps, inner_maps,
+								 outer_nparts, inner_nparts,
+								 jointype, next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes,
+													  merged_kinds,
+													  -1, default_index);
+	}
+
+	/* Free any memory we used in this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype)
+{
+	PartitionMap *outer_maps = NULL;
+	PartitionMap *inner_maps = NULL;
+	int			cnto;
+	int			cnti;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			next_index = 0;
+	int			null_index;
+	int			default_index = -1;
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			      *outer_indexes = outer_bi->indexes;
+	int			      *inner_indexes = inner_bi->indexes;
+	int				   outer_default = outer_bi->default_index;
+	int				   inner_default = inner_bi->default_index;
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	outer_maps = init_partition_map(outer_rel);
+	inner_maps = init_partition_map(inner_rel);
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			o_index;
+		int			i_index;
+		int			cmpval;
+		int			merged_index = -1;
+		Datum	   *merged_datum;
+		bool		finished_inner;
+		bool		finished_outer;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+		{
+			finished_outer = true;
+			odatums = NULL;
+			o_index = -1;
+		}
+		else
+		{
+			finished_outer = false;
+			odatums = outer_bi->datums[cnto];
+			o_index = outer_indexes[cnto];
+		}
+
+		if (cnti >= inner_bi->ndatums)
+		{
+			finished_inner = true;
+			idatums = NULL;
+			i_index = -1;
+		}
+		else
+		{
+			finished_inner = false;
+			idatums = inner_bi->datums[cnti];
+			i_index = inner_indexes[cnti];
+		}
+
+		/* If we exhausted both the sides, we won't enter the loop. */
+		Assert(!finished_inner || !finished_outer);
+
+		if (finished_outer)
+			cmpval = 1;
+		else if (finished_inner)
+			cmpval = -1;
+		else
+		{
+			/* Every list datum should map to a valid partition index. */
+			Assert(o_index >= 0 && i_index >= 0 &&
+				   odatums != NULL && idatums != NULL);
+
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0], idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+													o_index, i_index,
+													&next_index);
+
+			if (merged_index < 0)
+				return NULL;
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			/* A datum missing from the inner side. */
+			merged_index = -1;
+			merged_datum = odatums;
+
+			/*
+			 * For a FULL join, inner relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the inner relation acts as
+			 * INNER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_inner = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_outer = (jointype == JOIN_FULL);
+
+			if (!handle_missing_partition(inner_maps,
+										  outer_maps,
+										  inner_default,
+										  o_index,
+										  missing_side_outer,
+										  missing_side_inner,
+										  &next_index,
+										  &default_index,
+										  &merged_index))
+				return NULL;
+
+			/* Move to the next datum on the outer side. */
+			Assert(!finished_outer);
+			cnto++;
+		}
+		else
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			Assert(cmpval > 0);
+
+			/* A datum missing from the outer side. */
+			merged_index = -1;
+			merged_datum = idatums;
+
+			/*
+			 * For a FULL join, outer relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the outer relation acts as
+			 * OUTER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_outer = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_inner = (jointype == JOIN_FULL);
+
+			if (!handle_missing_partition(outer_maps,
+										  inner_maps,
+										  outer_default,
+										  i_index,
+										  missing_side_outer,
+										  missing_side_inner,
+										  &next_index,
+										  &default_index,
+										  &merged_index))
+				return NULL;
+
+			/* Move to the next datum on the right side. */
+			Assert(!finished_inner);
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	if (!merge_null_partitions(outer_bi, inner_bi,
+							   outer_maps, inner_maps,
+							   jointype, &next_index, &null_index,
+							   &default_index))
+		return NULL;
+
+	if (!merge_default_partitions(outer_bi, inner_bi,
+								  outer_maps, inner_maps,
+								  jointype, &next_index,
+								  &default_index))
+		return NULL;
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_maps, inner_maps,
+								 outer_nparts, inner_nparts,
+								 jointype, next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes, NIL,
+													  null_index, default_index);
+	}
+
+	/* Free up all extra memory before returning from this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_bounds_merge()'s arm for hash partitioned tables.
+ *
+ * If the given two hash bounds are same, the function returns the first one
+ * without any change, alongwith the lists of matching partitions. Otherwise it
+ * returns NULL.
+ *
+ * We could try merging the bounds when both the bounds have same greatest
+ * modulii. But there seems to be hardly any requirement for the same.
+ */
+static PartitionBoundInfo
+partition_hash_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype)
+{
+	int			nparts;
+	int			cnt;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * Hash partitioned table does not have explicit NULL accepting partition
+	 * and also does not have a default partition.
+	 */
+	Assert(!partition_bound_has_default(outer_bi) &&
+		   !partition_bound_has_default(inner_bi));
+	Assert(!partition_bound_accepts_nulls(outer_bi) &&
+		   !partition_bound_accepts_nulls(inner_bi));
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+
+	if (outer_nparts != inner_nparts)
+		return NULL;
+	nparts = outer_nparts;
+
+	if (outer_bi->ndatums != inner_bi->ndatums ||
+		!partition_hbounds_equal(outer_bi, inner_bi))
+		return NULL;
+
+	 /*
+	  * Cook up list of matching partitions. Since bounds are exactly same the
+	  * partitions at the same position from both the relations match.
+	  */
+	for (cnt = 0; cnt < nparts; cnt++)
+	{
+		*outer_parts = lappend_int(*outer_parts, cnt);
+		*inner_parts = lappend_int(*inner_parts, cnt);
+	}
+
+	return outer_bi;
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * If the two given partitions (given by index1 and index2 resp.) are
+ * already mapped to each other return the index of corresponding partition in
+ * the merged set of partitions.  If they do not have a merged partition
+ * associated with them, assign a new merged partition index.  If the
+ * partitions are already mapped and their mapped partitions are different from
+ * each other, they can not be merged, so return -1.
+ *
+ * partmaps1[i] gives the mapping of partitions for both relations. It
+ * describes which partition of relation 2 matches ith partition of relation 1,
+ * and which partition in the merged set matches ith partition of relation 1
+ * maps to. Similarly for partmap2.
+ *
+ * index1 and index2 are the indexes of matching partition from respective
+ * relations.
+ *
+ * *next_index is used and incremented when the given partitions require a new
+ * merged partition.
+ */
+
+static int
+map_and_merge_partitions(PartitionMap *partmaps1, PartitionMap *partmaps2,
+						 int index1, int index2, int *next_index)
+{
+	PartitionMap 	*partmap1 = &partmaps1[index1];
+	PartitionMap 	*partmap2 = &partmaps2[index2];
+	int				merged_index;
+
+	/*
+	 * If both the partitions are not mapped to each other, update the
+	 * maps.
+	 */
+	if (partmap1->from < 0 && partmap2->from < 0)
+	{
+		partmap1->from = index2;
+		partmap2->from = index1;
+	}
+
+	/*
+	 * If the given to partitions map to each other, find the corresponding
+	 * merged partition index .
+	 */
+	if (partmap1->from == index2 && partmap2->from == index1)
+	{
+		/*
+		 * If both the partitions are mapped to the same merged partition, get
+		 * the index of merged partition.
+		 */
+		if (partmap1->to == partmap2->to)
+		{
+			merged_index = partmap1->to;
+
+			/*
+			 * If the given two partitions do not have a merged partition
+			 * associated with them, allocate a new merged partition.
+			 */
+			if (merged_index < 0)
+			{
+				merged_index = *next_index;
+				*next_index = *next_index + 1;
+				partmap1->to = merged_index;
+				partmap2->to = merged_index;
+			}
+		}
+
+		/*
+		 * If partition from one relation was mapped to a merged partition but
+		 * not the partition from the other relation, map the same merged
+		 * partition to the partition from other relation, since matching
+		 * partitions map to the same merged partition.
+		 */
+		else if (partmap1->to >= 0 && partmap2->to < 0)
+		{
+			partmap2->to = partmap1->to;
+			merged_index = partmap1->to;
+		}
+		else if (partmap1->to < 0 && partmap2->to >= 0)
+		{
+			partmap1->to = partmap2->to;
+			merged_index = partmap2->to;
+		}
+		else
+		{
+			Assert(partmap1->to != partmap2->to &&
+				   partmap1->to >= 0 && partmap2->to >= 0);
+
+			/*
+			 * Both the partitions map to different merged partitions. This
+			 * means that multiple partitions from one relation matches to one
+			 * partition from the other relation. Partition-wise join does not
+			 * handle this case right now, since it requires ganging multiple
+			 * partitions together (into one RelOptInfo).
+			 */
+			merged_index = -1;
+		}
+	}
+	else
+	{
+		/*
+		 * Multiple partitions from one relation map to one partition from the
+		 * other relation. Partition-wise join does not handle this case right
+		 * now, since it requires ganging multiple partitions together (into
+		 * one RelOptInfo).
+		 */
+		merged_index = -1;
+	}
+
+	return merged_index;
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * partmaps1 map each partition from either side of the join to a merged
+ * partition resp. E.g. partmaps1[i].to gives the merged partition to which ith
+ * partition of first relation maps. Similarly for partmap2. If
+ * partmaps1[i].to == partmaps2[j].to, i and j form the matching pair of
+ * partitions.
+ *
+ * Given these maps this function produces the list pairs of partitions which
+ * when joined produce the merged partitions in the order of merged partition
+ * indexes.
+ *
+ * nparts1 and nparts2 are the number of partitions of the joining relations
+ * resp.
+ *
+ * nparts is the number of merged partitions.
+ *
+ * If successful, the pairs of partitions are returned as two separate lists,
+ * parts1 and parts2 resp., one for each side. Otherwise, those lists will be
+ * set to NIL.
+ */
+static void
+generate_matching_part_pairs(PartitionMap *partmaps1, PartitionMap *partmaps2,
+							 int nparts1, int nparts2,
+							 JoinType jointype, int nparts,
+							 List **matched_parts1, List **matched_parts2)
+{
+	bool	merged = true;
+	int		*matching1 = (int *) palloc(sizeof(int) * nparts),
+			*matching2 = (int *) palloc(sizeof(int) * nparts);
+	int 	i;
+
+	*matched_parts1 = NIL;
+	*matched_parts2 = NIL;
+
+	/* Set pairs of matching partitions. */
+	for (i = 0; i < nparts; i++)
+	{
+		if (i >= nparts1)
+			matching1[i] = -1;
+		else
+		{
+			PartitionMap outer_map = partmaps1[i];
+
+			if (outer_map.to >= 0)
+			{
+				Assert(outer_map.to < nparts);
+				matching1[outer_map.to] = i;
+			}
+		}
+
+		if (i >= nparts2)
+			matching2[i] = -1;
+		else
+		{
+			PartitionMap inner_map = partmaps2[i];
+
+			if (inner_map.to >= 0)
+			{
+				Assert(inner_map.to < nparts);
+				matching2[inner_map.to] = i;
+			}
+		}
+	}
+
+	/*
+	 * If we have a partition missing on an inner side, we need to add a dummy
+	 * relation which joins with the outer partition. If the inner relation
+	 * happens to be a base relation, it will require adding a dummy child
+	 * base relation during join processing. Right now, we freeze the base
+	 * relation arrays like PlannerInfo::simple_rte_array after planning for
+	 * base relations. Adding a new (dummy) base relation would require some
+	 * changes to that. So, right now, we do not implement partition-wise join
+	 * in such cases.
+	 */
+	for (i = 0; i < nparts; i++)
+	{
+		int			part1 = matching1[i];
+		int			part2 = matching2[i];
+
+		/* At least one of the partitions should exist. */
+		Assert(part1 >= 0 || part2 >= 0);
+
+		switch (jointype)
+		{
+			case JOIN_INNER:
+			case JOIN_SEMI:
+
+				/*
+				 * An inner or semi join can not return any row when the
+				 * matching partition on either side is missing. We should
+				 * have eliminated all such cases while merging the bounds.
+				 */
+				Assert(part1 >= 0 && part2 >= 0);
+				break;
+
+			case JOIN_LEFT:
+			case JOIN_ANTI:
+				Assert(part1 >= 0);
+				if (part2 < 0)
+					merged = false;
+				break;
+
+			case JOIN_FULL:
+				if (part1 < 0 || part2 < 0)
+					merged = false;
+				break;
+
+			default:
+				elog(ERROR, "unrecognized join type: %d", (int) jointype);
+		}
+
+		if (!merged)
+			break;
+
+		*matched_parts1 = lappend_int(*matched_parts1, part1);
+		*matched_parts2 = lappend_int(*matched_parts2, part2);
+	}
+
+	pfree(matching1);
+	pfree(matching2);
+
+	if (!merged)
+	{
+		list_free(*matched_parts1);
+		list_free(*matched_parts2);
+		*matched_parts1 = NIL;
+		*matched_parts2 = NIL;
+	}
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * Merge default partitions from both sides, if any, and assign the default
+ * partition for the join result, if necessary.
+ *
+ * If both the relations have default partitions, try mapping those to each
+ * other. If the mapping succeeds corresponding merged partition will act as
+ * the default partition of the join result.
+ *
+ * If inner side of the join has default but not the outer side, rows in it
+ * won't appear in the join result. So don't create a default partition. If
+ * outer side of the join has default but not the inner side, rows in it will
+ * appear in the join result, so create a default merged partition.
+ */
+static bool
+merge_default_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+						 PartitionMap *outer_maps, PartitionMap *inner_maps,
+						 JoinType jointype, int *next_index, int *default_index)
+{
+	int				outer_default = outer_bi->default_index;
+	int				inner_default = inner_bi->default_index;
+	bool			outer_has_default = partition_bound_has_default(outer_bi);
+	bool			inner_has_default = partition_bound_has_default(inner_bi);
+	bool			merged = true;
+	PartitionMap 	*outer_default_map = NULL;
+	PartitionMap 	*inner_default_map = NULL;
+
+	if (outer_has_default)
+		outer_default_map = &outer_maps[outer_default];
+
+	if (inner_has_default)
+		inner_default_map = &inner_maps[inner_default];
+
+	if (!outer_has_default && !inner_has_default)
+		Assert(*default_index < 0);
+	else if (outer_default_map != NULL && inner_default_map == NULL)
+	{
+		if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+			jointype == JOIN_ANTI)
+		{
+			if (outer_default_map->to < 0)
+			{
+				outer_default_map->to = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = outer_default_map->to;
+			}
+			else
+				Assert(*default_index == outer_default_map->to);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else if (outer_default_map == NULL && inner_default_map != NULL)
+	{
+		if (jointype == JOIN_FULL)
+		{
+			if (inner_default_map->to < 0)
+			{
+				inner_default_map->to = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = inner_default_map->to;
+			}
+			else
+				Assert(*default_index == inner_default_map->to);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		*default_index = map_and_merge_partitions(outer_maps, inner_maps,
+												  outer_default, inner_default,
+												  next_index);
+
+		if (*default_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
+
+/*
+ * merge_null_partitions
+ *
+ * Merge NULL partitions, i.e. a partition that can hold NULL values for a list
+ * partitioned table, if any. Find the index of merged partition to which the
+ * NULL values would belong in the join result. If one joining relation has a
+ * NULL partition but not the other, try matching it with the default partition
+ * from the other relation since the default partition may have rows with NULL
+ * partition key. We can eliminate a NULL partition when it appears only on the
+ * inner side of the join and the outer side doesn't have a default partition.
+ *
+ * When the equality operator used for join is strict, two NULL values will not
+ * be considered as equal, and thus a NULL partition can be eliminated for an
+ * inner join. But we don't check the strictness operator here.
+ */
+static bool
+merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_maps, PartitionMap *inner_maps,
+					  JoinType jointype, int *next_index,
+					  int *null_index, int *default_index)
+{
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	int			outer_ni = outer_bi->null_index;
+	int			inner_ni = inner_bi->null_index;
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	bool		merged = true;
+
+	if (!outer_has_null && !inner_has_null)
+		*null_index = -1;
+	else if (outer_has_null && !inner_has_null)
+	{
+		int			merged_index;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, inner relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the inner relation acts as
+		 * INNER relation. For INNER and SEMI join OUTER and INNER
+		 * differentiation is immaterial.
+		 */
+		missing_side_inner = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_outer = (jointype == JOIN_FULL);
+
+		merged = handle_missing_partition(inner_maps,
+										  outer_maps,
+										  inner_default,
+										  outer_ni,
+										  missing_side_outer,
+										  missing_side_inner, next_index,
+										  default_index, &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join to which the outer null partition maps
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = outer_maps[outer_ni].to;
+	}
+	else if (!outer_has_null && inner_has_null)
+	{
+		int			merged_index;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, outer relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the outer relation acts as OUTER
+		 * relation. For INNER and SEMI join OUTER and INNER differentiation is
+		 * immaterial.
+		 */
+		missing_side_outer = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_inner = (jointype == JOIN_FULL);
+		merged = handle_missing_partition(outer_maps,
+										  inner_maps,
+										  outer_default,
+										  inner_ni,
+										  missing_side_outer,
+										  missing_side_inner,
+										  next_index, default_index,
+										  &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join, to which the outer side null partition maps,
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = inner_maps[inner_ni].to;
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		*null_index = map_and_merge_partitions(outer_maps,
+											   inner_maps,
+											   outer_ni,
+											   inner_ni,
+											   next_index);
+		if (*null_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index 36fb584e23..789d089ddc 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -107,5 +107,10 @@ extern int partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 							  int nvalues, Datum *values, bool *is_equal);
 extern int partition_hash_bsearch(PartitionBoundInfo boundinfo,
 					   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
-- 
2.16.4

0001-Hash-partition-bound-equality-refactoring-v14.patchapplication/octet-stream; name=0001-Hash-partition-bound-equality-refactoring-v14.patchDownload
From bab6680b6b5f8dff042672fc6e08b9a9cc580353 Mon Sep 17 00:00:00 2001
From: erthalion <9erthalion6@gmail.com>
Date: Tue, 11 Sep 2018 21:08:07 +0200
Subject: [PATCH 1/5] Hash partition bound equality refactoring.

Separate the code to check whether two given hash bounds are equal into a
separate function, so that it can be called from multiple places. Right now
it's only caller is partition_bounds_equal() but later we will use it for
merging partition bounds.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/partitioning/partbounds.c | 95 ++++++++++++++++++++++-------------
 1 file changed, 61 insertions(+), 34 deletions(-)

diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index eeaab2f4c9..0af3372cdf 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -104,6 +104,9 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 						 Expr **keyCol,
 						 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static bool partition_hbounds_equal(PartitionBoundInfo b1,
+									PartitionBoundInfo b2);
+
 
 /*
  * get_qual_from_partbound
@@ -652,6 +655,63 @@ create_range_bounds(PartitionBoundSpec **boundspecs, int nparts,
 	return boundinfo;
 }
 
+/*
+ * Are two hash partition bound collections logically equal?
+ *
+ * Hash partition bounds store modulus and remainder in datums array which are
+ * always integers irrespective of the number of partition keys and their data
+ * types. Hence we can compare the hash bound collection without any partition
+ * key specific information. Separating this logic in a function which does not
+ * require partition key specific information allows it be called from places
+ * where the partition key specific information is not completely available.
+ */
+static bool
+partition_hbounds_equal(PartitionBoundInfo b1, PartitionBoundInfo b2)
+{
+	int			greatest_modulus = get_hash_partition_greatest_modulus(b1);
+	int			i;
+
+	Assert(b1->strategy == PARTITION_STRATEGY_HASH &&
+		   b2->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * If two hash partitioned tables have different greatest moduli,
+	 * their partition schemes don't match.  For hash partitioned table,
+	 * the greatest modulus is given by the last datum and number of
+	 * partitions is given by ndatums.
+	 */
+	if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
+		return false;
+
+	/*
+	 * We arrange the partitions in the ascending order of their modulus and
+	 * remainders.  Also every modulus is factor of next larger modulus.
+	 * Therefore we can safely store index of a given partition in indexes
+	 * array at remainder of that partition.  Also entries at (remainder + N *
+	 * modulus) positions in indexes array are all same for (modulus,
+	 * remainder) specification for any partition.  Thus datums array from both
+	 * the given bounds are same, if and only if their indexes array will be
+	 * same.  So, it suffices to compare indexes array.
+	 */
+	for (i = 0; i < greatest_modulus; i++)
+		if (b1->indexes[i] != b2->indexes[i])
+			return false;
+
+#ifdef USE_ASSERT_CHECKING
+
+	/*
+	 * Nonetheless make sure that the bounds are indeed same when the indexes
+	 * match.  Hash partition bound stores modulus and remainder at
+	 * b1->datums[i][0] and b1->datums[i][1] position respectively.
+	 */
+	for (i = 0; i < b1->ndatums; i++)
+		Assert((b1->datums[i][0] == b2->datums[i][0] &&
+				b1->datums[i][1] == b2->datums[i][1]));
+#endif
+
+	return true;
+}
+
 /*
  * Are two partition bound collections logically equal?
  *
@@ -680,41 +740,8 @@ partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
 
 	if (b1->strategy == PARTITION_STRATEGY_HASH)
 	{
-		int			greatest_modulus = get_hash_partition_greatest_modulus(b1);
-
-		/*
-		 * If two hash partitioned tables have different greatest moduli,
-		 * their partition schemes don't match.
-		 */
-		if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
+		if (!partition_hbounds_equal(b1, b2))
 			return false;
-
-		/*
-		 * We arrange the partitions in the ascending order of their moduli
-		 * and remainders.  Also every modulus is factor of next larger
-		 * modulus.  Therefore we can safely store index of a given partition
-		 * in indexes array at remainder of that partition.  Also entries at
-		 * (remainder + N * modulus) positions in indexes array are all same
-		 * for (modulus, remainder) specification for any partition.  Thus
-		 * datums array from both the given bounds are same, if and only if
-		 * their indexes array will be same.  So, it suffices to compare
-		 * indexes array.
-		 */
-		for (i = 0; i < greatest_modulus; i++)
-			if (b1->indexes[i] != b2->indexes[i])
-				return false;
-
-#ifdef USE_ASSERT_CHECKING
-
-		/*
-		 * Nonetheless make sure that the bounds are indeed same when the
-		 * indexes match.  Hash partition bound stores modulus and remainder
-		 * at b1->datums[i][0] and b1->datums[i][1] position respectively.
-		 */
-		for (i = 0; i < b1->ndatums; i++)
-			Assert((b1->datums[i][0] == b2->datums[i][0] &&
-					b1->datums[i][1] == b2->datums[i][1]));
-#endif
 	}
 	else
 	{
-- 
2.16.4

0004-Tests-for-0-1-1-1-and-1-0-partition-matching-v14.patchapplication/octet-stream; name=0004-Tests-for-0-1-1-1-and-1-0-partition-matching-v14.patchDownload
From 79133983393b6583dd153e1180f146b285c33901 Mon Sep 17 00:00:00 2001
From: erthalion <9erthalion6@gmail.com>
Date: Tue, 11 Sep 2018 21:11:55 +0200
Subject: [PATCH 4/5] Tests for 0:1, 1:1 and 1:0 partition matching

Rajkumar Raghuvanshi and Ashutosh Bapat.
---
 src/test/regress/expected/partition_join.out | 4101 +++++++++++++++++++++-----
 src/test/regress/sql/partition_join.sql      |  427 ++-
 2 files changed, 3834 insertions(+), 694 deletions(-)

diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index c55de5d476..3be5ebe29d 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -8,59 +8,86 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-(4 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+(7 rows)
 
 -- left outer join, with whole-row reference; partitionwise join does not apply
 EXPLAIN (COSTS OFF)
@@ -72,35 +99,50 @@ SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER
    ->  Hash Right Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(22 rows)
 
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-      t1      |      t2      
---------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
- (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
- (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
- (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
- (550,0,0550) | 
-(12 rows)
+       t1       |       t2       
+----------------+----------------
+ (-250,0,-0250) | 
+ (-200,0,-0200) | 
+ (-150,0,-0150) | (0,-150,-0150)
+ (-100,0,-0100) | 
+ (-50,0,-0050)  | 
+ (0,0,0000)     | (0,0,0000)
+ (50,0,0050)    | 
+ (100,0,0100)   | 
+ (150,0,0150)   | (0,150,0150)
+ (200,0,0200)   | 
+ (250,0,0250)   | 
+ (300,0,0300)   | (0,300,0300)
+ (350,0,0350)   | 
+ (400,0,0400)   | 
+ (450,0,0450)   | (0,450,0450)
+ (500,0,0500)   | 
+ (550,0,0550)   | 
+ (600,0,0600)   | (0,600,0600)
+ (650,0,0650)   | 
+ (700,0,0700)   | 
+ (750,0,0750)   | (0,750,0750)
+(21 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
@@ -112,35 +154,53 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
    ->  Append
          ->  Hash Right Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Right Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t2_2.b)
-(20 rows)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+      |       | -225 | -0225
+      |       |  -75 | -0075
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(14 rows)
 
 -- full outer join, with placeholder vars
 EXPLAIN (COSTS OFF)
@@ -148,8 +208,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                             QUERY PLAN                            
 ------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               Filter: (((50) = prt1_p0.a) OR ((75) = prt2_p0.b))
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
@@ -174,7 +242,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               Filter: (((50) = prt1_p4.a) OR ((75) = prt2_p4.b))
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
@@ -211,8 +287,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Hash Left Join
+               Hash Cond: (prt1_p0.a = b)
+               ->  Seq Scan on prt1_p0
+                     Filter: ((a < 450) AND (b = 0))
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
          ->  Hash Left Join
                Hash Cond: (prt1_p1.a = b)
                ->  Seq Scan on prt1_p1
@@ -227,29 +310,42 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                ->  Hash
                      ->  Seq Scan on prt1_p2
                            Filter: ((a < 450) AND (b = 0))
-(17 rows)
+(24 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-(9 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+(14 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
                          QUERY PLAN                         
 ------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = b)
+               Filter: ((prt1_p0.b = 0) OR (a = 0))
+               ->  Seq Scan on prt1_p0
+                     Filter: (a < 450)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = b)
                Filter: ((prt1_p1.b = 0) OR (a = 0))
@@ -274,64 +370,153 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                ->  Hash
                      ->  Result
                            One-Time Filter: false
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt2_p4.b = a)
+               Filter: ((b = 0) OR (prt2_p4.a = 0))
+               ->  Seq Scan on prt2_p4
+                     Filter: (b > 250)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
-(12 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+      |       | 375 | 0375
+      |       | 450 | 0450
+      |       | 525 | 0525
+      |       | 600 | 0600
+      |       | 675 | 0675
+      |       | 750 | 0750
+(20 rows)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Semi Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Semi Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                            Filter: (a = 0)
-         ->  Nested Loop Semi Join
-               Join Filter: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
-               ->  Materialize
-                     ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
-(24 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt1_p3 t2_3
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+                           Filter: (b = 0)
+(37 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
 
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
@@ -342,27 +527,82 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
    ->  Append
          ->  Hash Anti Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Hash Anti Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Hash Anti Join
                Hash Cond: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
-(17 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(27 rows)
 
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
-  sum  |         avg          | sum  |         avg         
--------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+  sum  |         avg          | sum  |        avg         
+-------+----------------------+------+--------------------
+ 95550 | 273.0000000000000000 | 2200 | 6.2857142857142857
 (1 row)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Append
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p0 t1
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+               Index Cond: (a = t1.b)
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.b = t2_1.a)
+         ->  Seq Scan on prt2_p1 t1_1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p1 t2_1
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p2 t1_2
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+               Index Cond: (a = t1_2.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p3 t1_3
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+               Index Cond: (a = t1_3.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p4 t1_4
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+               Index Cond: (a = t1_4.b)
+(27 rows)
+
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+  b   |   c   
+------+-------
+ -225 | -0225
+  -75 | -0075
+   75 | 0075
+  225 | 0225
+  375 | 0375
+  525 | 0525
+  675 | 0675
+(7 rows)
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -374,49 +614,74 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
+                     ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
                            Index Cond: (a = t1.a)
-                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t3
                            Index Cond: (b = t2.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_1
+                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
                            Index Cond: (a = t1_1.a)
-                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3_1
                            Index Cond: (b = t2_1.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_2
+                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
                            Index Cond: (a = t1_2.a)
-                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_2
                            Index Cond: (b = t2_2.a)
-(27 rows)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                           Index Cond: (a = t1_3.a)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_3
+                           Index Cond: (b = t2_3.a)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                           Index Cond: (a = t1_4.a)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t3_4
+                           Index Cond: (b = t2_4.a)
+(43 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   | t2a | t3a | least 
------+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
- 100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
- 250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
- 400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
- 550 | 0 | 0550 |     |     |      
-(12 rows)
+  a   | b |   c   | t2a  | t3a | least 
+------+---+-------+------+-----+-------
+ -250 | 0 | -0250 |      |     |      
+ -200 | 0 | -0200 |      |     |      
+ -150 | 0 | -0150 | -150 |   0 |  -150
+ -100 | 0 | -0100 |      |     |      
+  -50 | 0 | -0050 |      |     |      
+    0 | 0 | 0000  |    0 |   0 |     0
+   50 | 0 | 0050  |      |     |      
+  100 | 0 | 0100  |      |     |      
+  150 | 0 | 0150  |  150 |   0 |   150
+  200 | 0 | 0200  |      |     |      
+  250 | 0 | 0250  |      |     |      
+  300 | 0 | 0300  |  300 |   0 |   300
+  350 | 0 | 0350  |      |     |      
+  400 | 0 | 0400  |      |     |      
+  450 | 0 | 0450  |  450 |   0 |   450
+  500 | 0 | 0500  |      |     |      
+  550 | 0 | 0550  |      |     |      
+  600 | 0 | 0600  |  600 |   0 |   600
+  650 | 0 | 0650  |      |     |      
+  700 | 0 | 0700  |      |     |      
+  750 | 0 | 0750  |  750 |   0 |   750
+(21 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
@@ -430,64 +695,95 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
          Hash Cond: ((t1.c)::text = (t2.c)::text)
          Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Seq Scan on prt1_p1 t1
-               ->  Seq Scan on prt1_p2 t1_1
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
          ->  Hash
                ->  Append
                      ->  Hash Join
                            Hash Cond: (t2.a = t3.b)
-                           ->  Seq Scan on prt1_p1 t2
+                           ->  Seq Scan on prt1_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t3
+                                 ->  Seq Scan on prt2_p0 t3
                      ->  Hash Join
                            Hash Cond: (t2_1.a = t3_1.b)
-                           ->  Seq Scan on prt1_p2 t2_1
+                           ->  Seq Scan on prt1_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t3_1
+                                 ->  Seq Scan on prt2_p1 t3_1
                      ->  Hash Join
                            Hash Cond: (t2_2.a = t3_2.b)
-                           ->  Seq Scan on prt1_p3 t2_2
+                           ->  Seq Scan on prt1_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p3 t3_2
-(26 rows)
+                                 ->  Seq Scan on prt2_p2 t3_2
+                     ->  Hash Join
+                           Hash Cond: (t2_3.a = t3_3.b)
+                           ->  Seq Scan on prt1_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t3_3
+                     ->  Hash Join
+                           Hash Cond: (t2_4.a = t3_4.b)
+                           ->  Seq Scan on prt1_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t3_4
+(38 rows)
 
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, t2.b t2b, t2.c t2c, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
-  a  | t2a | t2c  
------+-----+------
-   0 |   0 | 0000
-  50 |     | 
- 100 |     | 
- 150 | 150 | 0150
- 200 |     | 
- 250 |     | 
- 300 | 300 | 0300
- 350 |     | 
- 400 |     | 
- 450 | 450 | 0450
- 500 |     | 
- 550 |     | 
-(12 rows)
+  a   | t2a  |  t2c  
+------+------+-------
+ -250 |      | 
+ -200 |      | 
+ -150 | -150 | -0150
+ -100 |      | 
+  -50 |      | 
+    0 |    0 | 0000
+   50 |      | 
+  100 |      | 
+  150 |  150 | 0150
+  200 |      | 
+  250 |      | 
+  300 |  300 | 0300
+  350 |      | 
+  400 |      | 
+  450 |  450 | 0450
+  500 |      | 
+  550 |      | 
+  600 |  600 | 0600
+  650 |      | 
+  700 |      | 
+  750 |  750 | 0750
+(21 rows)
 
 --
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
@@ -498,32 +794,49 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
    ->  Append
          ->  Hash Join
                Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
-               ->  Seq Scan on prt2_e_p1 t2
+               ->  Seq Scan on prt2_e_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 t1
+                     ->  Seq Scan on prt1_e_p0 t1
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
-               ->  Seq Scan on prt2_e_p2 t2_1
+               Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+               ->  Seq Scan on prt1_e_p1 t1_1
+                     Filter: (c = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 t1_1
-                           Filter: (c = 0)
+                     ->  Seq Scan on prt2_e_p1 t2_1
          ->  Hash Join
                Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
-               ->  Seq Scan on prt2_e_p3 t2_2
+               ->  Seq Scan on prt2_e_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 t1_2
+                     ->  Seq Scan on prt1_e_p2 t1_2
                            Filter: (c = 0)
-(21 rows)
+         ->  Hash Join
+               Hash Cond: (((t2_3.b + t2_3.a) / 2) = ((t1_3.a + t1_3.b) / 2))
+               ->  Seq Scan on prt2_e_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p3 t1_3
+                           Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (((t2_4.b + t2_4.a) / 2) = ((t1_4.a + t1_4.b) / 2))
+               ->  Seq Scan on prt2_e_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4 t1_4
+                           Filter: (c = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
- 150 | 0 | 150 | 0
- 300 | 0 | 300 | 0
- 450 | 0 | 450 | 0
-(4 rows)
+  a   | c |  b   | c 
+------+---+------+---
+ -250 | 0 | -250 | 0
+ -100 | 0 | -100 | 0
+    0 | 0 |    0 | 0
+    0 | 0 |    0 | 0
+  150 | 0 |  150 | 0
+  300 | 0 |  300 | 0
+  450 | 0 |  450 | 0
+  600 | 0 |  600 | 0
+  750 | 0 |  750 | 0
+(9 rows)
 
 --
 -- N-way join
@@ -536,154 +849,232 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = ((t3.a + t3.b) / 2))
+               Join Filter: (t1.a = t2.b)
                ->  Hash Join
-                     Hash Cond: (t2.b = t1.a)
-                     ->  Seq Scan on prt2_p1 t2
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on prt1_e_p0 t3
                      ->  Hash
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                     Index Cond: (((a + b) / 2) = t2.b)
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
+                     Index Cond: (b = ((t3.a + t3.b) / 2))
          ->  Nested Loop
                Join Filter: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_1.b = t1_1.a)
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
+               ->  Index Scan using iprt1_e_p4_ab2 on prt1_e_p1 t3_1
                      Index Cond: (((a + b) / 2) = t2_1.b)
          ->  Nested Loop
                Join Filter: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_2.b = t1_2.a)
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
+               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_2
                      Index Cond: (((a + b) / 2) = t2_2.b)
-(33 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = ((t3_3.a + t3_3.b) / 2))
+               ->  Nested Loop
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t2_3.b)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t2_4.b)
+               ->  Hash Join
+                     Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                     ->  Seq Scan on prt1_e_p4 t3_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_p4 t1_4
+                                 Filter: (b = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = ((t3_4.a + t3_4.b) / 2))
+(52 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
-(4 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(8 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                          QUERY PLAN                          
---------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Hash Right Join
                Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
-               ->  Seq Scan on prt1_e_p1 t3
+               ->  Seq Scan on prt1_e_p0 t3
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2.b = t1.a)
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_1.a)
-               ->  Seq Scan on prt1_e_p2 t3_1
+               ->  Seq Scan on prt1_e_p1 t3_1
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_1.b = t1_1.a)
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_2.a)
-               ->  Seq Scan on prt1_e_p3 t3_2
+               ->  Seq Scan on prt1_e_p2 t3_2
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_2.b = t1_2.a)
-                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                                        Filter: (b = 0)
-(33 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (t1_3.a = b)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (t1_3.a = ((a + b) / 2))
+         ->  Hash Right Join
+               Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+               ->  Seq Scan on prt1_e_p4 t3_4
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_4.b = t1_4.a)
+                           ->  Seq Scan on prt2_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_p4 t1_4
+                                       Filter: (b = 0)
+(51 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                            QUERY PLAN                             
--------------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p1 t3
+                           ->  Seq Scan on prt1_e_p0 t3
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p1_b on prt2_p1 t2
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
                      Index Cond: (t1.a = b)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p2 t3_1
+                           ->  Seq Scan on prt1_e_p1 t3_1
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
+               ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
                      Index Cond: (t1_1.a = b)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p3 t3_2
+                           ->  Seq Scan on prt1_e_p2 t3_2
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
                      Index Cond: (t1_2.a = b)
-(30 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_e_p3 t3_3
+                           Filter: (c = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                           Index Cond: (a = ((t3_3.a + t3_3.b) / 2))
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (t1_3.a = b)
+         ->  Nested Loop Left Join
+               ->  Hash Right Join
+                     Hash Cond: (t1_4.a = ((t3_4.a + t3_4.b) / 2))
+                     ->  Seq Scan on prt1_p4 t1_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p4 t3_4
+                                 Filter: (c = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (t1_4.a = b)
+(47 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -692,21 +1083,34 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                                                    QUERY PLAN                                                   
 ----------------------------------------------------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   Sort Key: prt1_p0.a, prt2_p0.b, ((prt1_e_p0.a + prt1_e_p0.b))
    ->  Append
          ->  Hash Full Join
-               Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
-               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               Hash Cond: (prt1_p0.a = ((prt1_e_p0.a + prt1_e_p0.b) / 2))
+               Filter: ((prt1_p0.a = (50)) OR (prt2_p0.b = (75)) OR (((prt1_e_p0.a + prt1_e_p0.b) / 2) = (50)))
                ->  Hash Full Join
-                     Hash Cond: (prt1_p1.a = prt2_p1.b)
-                     ->  Seq Scan on prt1_p1
+                     Hash Cond: (prt1_p0.a = prt2_p0.b)
+                     ->  Seq Scan on prt1_p0
                            Filter: (b = 0)
                      ->  Hash
-                           ->  Seq Scan on prt2_p1
+                           ->  Seq Scan on prt2_p0
                                  Filter: (a = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1
+                     ->  Seq Scan on prt1_e_p0
                            Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = prt1_p1.a)
+               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               ->  Seq Scan on prt1_e_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on prt1_p1
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1
+                                       Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
                Filter: ((prt1_p2.a = (50)) OR (prt2_p2.b = (75)) OR (((prt1_e_p2.a + prt1_e_p2.b) / 2) = (50)))
@@ -733,7 +1137,20 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                ->  Hash
                      ->  Seq Scan on prt1_e_p3
                            Filter: (c = 0)
-(42 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = ((prt1_e_p4.a + prt1_e_p4.b) / 2))
+               Filter: ((prt1_p4.a = (50)) OR (prt2_p4.b = (75)) OR (((prt1_e_p4.a + prt1_e_p4.b) / 2) = (50)))
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = prt2_p4.b)
+                     ->  Seq Scan on prt1_p4
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4
+                                 Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4
+                           Filter: (c = 0)
+(68 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
@@ -751,53 +1168,81 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = t1_3.b)
+               Join Filter: (t1.a = t1_5.b)
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Join
-                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
-                           ->  Seq Scan on prt1_e_p1 t2
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t1_3
+                                 ->  Seq Scan on prt2_p0 t1_5
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
                      Index Cond: (a = ((t2.a + t2.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_1.a = t1_4.b)
+               Join Filter: (t1_1.a = t1_6.b)
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Join
-                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
-                           ->  Seq Scan on prt1_e_p2 t2_1
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_6.b)
+                           ->  Seq Scan on prt1_e_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t1_4
+                                 ->  Seq Scan on prt2_p1 t1_6
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
                      Index Cond: (a = ((t2_1.a + t2_1.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_2.a = t1_5.b)
+               Join Filter: (t1_2.a = t1_7.b)
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
+                           ->  Seq Scan on prt2_p2 t1_7
                                  Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
+                           ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t2_2
+                                 Index Cond: (((a + b) / 2) = t1_7.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t1_8.b)
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Nested Loop
+                           ->  Seq Scan on prt2_p3 t1_8
+                                 Filter: (a = 0)
+                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_3
+                                 Index Cond: (((a + b) / 2) = t1_8.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = ((t2_3.a + t2_3.b) / 2))
+                     Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t1_9.b)
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Join
+                           Hash Cond: (((t2_4.a + t2_4.b) / 2) = t1_9.b)
+                           ->  Seq Scan on prt1_e_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t1_9
+                                       Filter: (a = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = ((t2_4.a + t2_4.b) / 2))
+                     Filter: (b = 0)
+(66 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
@@ -808,27 +1253,39 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (
    ->  Append
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
-                           ->  Seq Scan on prt2_p1 t1_3
+                           Hash Cond: (t1_5.b = ((t1_10.a + t1_10.b) / 2))
+                           ->  Seq Scan on prt2_p0 t1_5
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t1_6
+                                 ->  Seq Scan on prt1_e_p0 t1_10
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
-                     Index Cond: (a = t1_3.b)
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t1_5.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
-                           ->  Seq Scan on prt2_p2 t1_4
+                           Hash Cond: (t1_6.b = ((t1_11.a + t1_11.b) / 2))
+                           ->  Seq Scan on prt2_p1 t1_6
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t1_7
+                                 ->  Seq Scan on prt1_e_p1 t1_11
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
-                     Index Cond: (a = t1_4.b)
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
+                     Index Cond: (a = t1_6.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_7.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_7.b = ((t1_12.a + t1_12.b) / 2))
+                           ->  Seq Scan on prt2_p2 t1_7
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p2 t1_12
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t1_7.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
@@ -845,78 +1302,114 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (
 (39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 -- test merge joins
 SET enable_hashjoin TO off;
 SET enable_nestloop TO off;
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                           QUERY PLAN                           
-----------------------------------------------------------------
+                            QUERY PLAN                            
+------------------------------------------------------------------
  Merge Append
    Sort Key: t1.a
    ->  Merge Semi Join
-         Merge Cond: (t1.a = t1_3.b)
+         Merge Cond: (t1.a = t1_5.b)
          ->  Sort
                Sort Key: t1.a
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+               Merge Cond: (t1_5.b = (((t1_10.a + t1_10.b) / 2)))
                ->  Sort
-                     Sort Key: t1_3.b
-                     ->  Seq Scan on prt2_p1 t1_3
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on prt2_p0 t1_5
                ->  Sort
-                     Sort Key: (((t1_6.a + t1_6.b) / 2))
-                     ->  Seq Scan on prt1_e_p1 t1_6
+                     Sort Key: (((t1_10.a + t1_10.b) / 2))
+                     ->  Seq Scan on prt1_e_p0 t1_10
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_1.a = t1_4.b)
+         Merge Cond: (t1_1.a = t1_6.b)
          ->  Sort
                Sort Key: t1_1.a
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+               Merge Cond: (t1_6.b = (((t1_11.a + t1_11.b) / 2)))
                ->  Sort
-                     Sort Key: t1_4.b
-                     ->  Seq Scan on prt2_p2 t1_4
+                     Sort Key: t1_6.b
+                     ->  Seq Scan on prt2_p1 t1_6
                ->  Sort
-                     Sort Key: (((t1_7.a + t1_7.b) / 2))
-                     ->  Seq Scan on prt1_e_p2 t1_7
+                     Sort Key: (((t1_11.a + t1_11.b) / 2))
+                     ->  Seq Scan on prt1_e_p1 t1_11
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_2.a = t1_5.b)
+         Merge Cond: (t1_2.a = t1_7.b)
          ->  Sort
                Sort Key: t1_2.a
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+               Merge Cond: (t1_7.b = (((t1_12.a + t1_12.b) / 2)))
                ->  Sort
-                     Sort Key: t1_5.b
-                     ->  Seq Scan on prt2_p3 t1_5
+                     Sort Key: t1_7.b
+                     ->  Seq Scan on prt2_p2 t1_7
                ->  Sort
-                     Sort Key: (((t1_8.a + t1_8.b) / 2))
-                     ->  Seq Scan on prt1_e_p3 t1_8
+                     Sort Key: (((t1_12.a + t1_12.b) / 2))
+                     ->  Seq Scan on prt1_e_p2 t1_12
                            Filter: (c = 0)
-(47 rows)
+   ->  Merge Semi Join
+         Merge Cond: (t1_3.a = t1_8.b)
+         ->  Sort
+               Sort Key: t1_3.a
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_8.b = (((t1_13.a + t1_13.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_8.b
+                     ->  Seq Scan on prt2_p3 t1_8
+               ->  Sort
+                     Sort Key: (((t1_13.a + t1_13.b) / 2))
+                     ->  Seq Scan on prt1_e_p3 t1_13
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_4.a = t1_9.b)
+         ->  Sort
+               Sort Key: t1_4.a
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_9.b = (((t1_14.a + t1_14.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_9.b
+                     ->  Seq Scan on prt2_p4 t1_9
+               ->  Sort
+                     Sort Key: (((t1_14.a + t1_14.b) / 2))
+                     ->  Seq Scan on prt1_e_p4 t1_14
+                           Filter: (c = 0)
+(77 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
@@ -933,14 +1426,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
                            ->  Sort
                                  Sort Key: (((t3.a + t3.b) / 2))
-                                 ->  Seq Scan on prt1_e_p1 t3
+                                 ->  Seq Scan on prt1_e_p0 t3
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1.a
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                ->  Sort
                      Sort Key: t2.b
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Merge Left Join
                Merge Cond: (t1_1.a = t2_1.b)
                ->  Sort
@@ -949,14 +1442,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
                            ->  Sort
                                  Sort Key: (((t3_1.a + t3_1.b) / 2))
-                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                 ->  Seq Scan on prt1_e_p1 t3_1
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_1.a
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                ->  Sort
                      Sort Key: t2_1.b
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Merge Left Join
                Merge Cond: (t1_2.a = t2_2.b)
                ->  Sort
@@ -965,32 +1458,74 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
                            ->  Sort
                                  Sort Key: (((t3_2.a + t3_2.b) / 2))
-                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                 ->  Seq Scan on prt1_e_p2 t3_2
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_2.a
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                ->  Sort
                      Sort Key: t2_2.b
-                     ->  Seq Scan on prt2_p3 t2_2
-(51 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Merge Left Join
+               Merge Cond: (t1_3.a = t2_3.b)
+               ->  Sort
+                     Sort Key: t1_3.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_3.a + t3_3.b) / 2)) = t1_3.a)
+                           ->  Sort
+                                 Sort Key: (((t3_3.a + t3_3.b) / 2))
+                                 ->  Seq Scan on prt1_e_p3 t3_3
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_3.a
+                                 ->  Seq Scan on prt1_p3 t1_3
+               ->  Sort
+                     Sort Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Merge Left Join
+               Merge Cond: (t1_4.a = t2_4.b)
+               ->  Sort
+                     Sort Key: t1_4.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_4.a + t3_4.b) / 2)) = t1_4.a)
+                           ->  Sort
+                                 Sort Key: (((t3_4.a + t3_4.b) / 2))
+                                 ->  Seq Scan on prt1_e_p4 t3_4
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_4.a
+                                 ->  Seq Scan on prt1_p4 t1_4
+               ->  Sort
+                     Sort Key: t2_4.b
+                     ->  Seq Scan on prt2_p4 t2_4
+(83 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- MergeAppend on nullable column
 EXPLAIN (COSTS OFF)
@@ -998,8 +1533,18 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Merge Left Join
+               Merge Cond: (prt1_p0.a = b)
+               ->  Sort
+                     Sort Key: prt1_p0.a
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
+               ->  Sort
+                     Sort Key: b
+                     ->  Result
+                           One-Time Filter: false
          ->  Merge Left Join
                Merge Cond: (prt1_p1.a = b)
                ->  Sort
@@ -1020,21 +1565,26 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                      Sort Key: prt2_p2.b
                      ->  Seq Scan on prt2_p2
                            Filter: (b > 250)
-(23 rows)
+(33 rows)
 
 SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  b  
------+-----
-   0 |    
-  50 |    
- 100 |    
- 150 |    
- 200 |    
- 250 |    
- 300 | 300
- 350 |    
- 400 |    
-(9 rows)
+  a   |  b  
+------+-----
+ -250 |    
+ -200 |    
+ -150 |    
+ -100 |    
+  -50 |    
+    0 |    
+   50 |    
+  100 |    
+  150 |    
+  200 |    
+  250 |    
+  300 | 300
+  350 |    
+  400 |    
+(14 rows)
 
 -- merge join when expression with whole-row reference needs to be sorted;
 -- partitionwise join does not apply
@@ -1048,175 +1598,2412 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
          Sort Key: t1.a, ((((t1.*)::prt1))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
    ->  Sort
          Sort Key: t2.b, ((((t2.*)::prt2))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt2_p1 t2
-                     ->  Seq Scan on prt2_p2 t2_1
-                     ->  Seq Scan on prt2_p3 t2_2
-(16 rows)
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+(20 rows)
 
 SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.b ORDER BY t1.a;
- a  | b  
-----+----
-  0 |  0
-  6 |  6
- 12 | 12
- 18 | 18
- 24 | 24
-(5 rows)
+  a  |  b  
+-----+-----
+ -24 | -24
+ -18 | -18
+ -12 | -12
+  -6 |  -6
+   0 |   0
+   6 |   6
+  12 |  12
+  18 |  18
+  24 |  24
+(9 rows)
 
 RESET enable_hashjoin;
 RESET enable_nestloop;
---
--- partitioned by multiple columns
---
-CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
-CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
-ANALYZE prt1_m;
-CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
-CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
-ANALYZE prt2_m;
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-                                                             QUERY PLAN                                                             
-------------------------------------------------------------------------------------------------------------------------------------
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p4 t2_3
+               ->  Seq Scan on prt2_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_4
+                           Filter: (b = 0)
+(22 rows)
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p3 t1_2
+                           Filter: (b = 0)
+(21 rows)
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -150 | -0150 | 0 | -0150
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+(7 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (t1_3.a = b)
+         ->  Hash Right Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 |   | 
+ -200 | -0200 |   | 
+ -150 | -0150 | 0 | -0150
+ -100 | -0100 |   | 
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+(21 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -200 | 0 | -0200
+ -100 | 0 | -0100
+  -50 | 0 | -0050
+   50 | 0 | 0050
+  100 | 0 | 0100
+  200 | 0 | 0200
+  250 | 0 | 0250
+  350 | 0 | 0350
+  400 | 0 | 0400
+  500 | 0 | 0500
+  550 | 0 | 0550
+  650 | 0 | 0650
+  700 | 0 | 0700
+(14 rows)
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+                             QUERY PLAN                              
+---------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t3.c
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p0_a on prt1_p0 t3
+                           Index Cond: (a = t2.b)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t2.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_1.a = t2_1.b)
+                           ->  Seq Scan on prt1_p1 t3_1
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1 t2_1
+                                       Filter: (a = 0)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p2_a on prt1_p2 t3_2
+                           Index Cond: (a = t2_2.b)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t3_3
+                           Index Cond: (a = t2_3.b)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_4.a = t2_4.b)
+                           ->  Seq Scan on prt1_p4 t3_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t2_4
+                                       Filter: (a = 0)
+(47 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+  a   | a |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.b
+   ->  Hash Right Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p5 t2_5
+                           Filter: (a = 0)
+(24 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: (t1.a = t2.b)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.a, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
+                     Filter: (a = 0)
+(22 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
+               ->  Seq Scan on prt1_m_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p1
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
+               ->  Seq Scan on prt1_m_p2
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p2
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
+               ->  Seq Scan on prt1_m_p3
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p3
+                           Filter: (c = 0)
+(24 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+  a  | c |  b  | c 
+-----+---+-----+---
+   0 | 0 |   0 | 0
+  50 | 0 |     |  
+ 100 | 0 |     |  
+ 150 | 0 | 150 | 0
+ 200 | 0 |     |  
+ 250 | 0 |     |  
+ 300 | 0 | 300 | 0
+ 350 | 0 |     |  
+ 400 | 0 |     |  
+ 450 | 0 | 450 | 0
+ 500 | 0 |     |  
+ 550 | 0 |     |  
+     |   |  75 | 0
+     |   | 225 | 0
+     |   | 375 | 0
+     |   | 525 | 0
+(16 rows)
+
+--
+-- tests for list partitioned tables.
+--
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
+--
+-- list partitioned by expression
+--
+CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
+ANALYZE plt1_e;
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+                                           QUERY PLAN                                           
+------------------------------------------------------------------------------------------------
+ GroupAggregate
+   Group Key: t1.c, t2.c, t3.c
+   ->  Sort
+         Sort Key: t1.c, t3.c
+         ->  Append
+               ->  Hash Join
+                     Hash Cond: ((t1.c)::text = ltrim(t3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2.b = t1.b) AND ((t2.c)::text = (t1.c)::text))
+                           ->  Seq Scan on plt2_p4 t2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p4 t1
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p4 t3
+               ->  Hash Join
+                     Hash Cond: ((t1_1.c)::text = ltrim(t3_1.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t1_1.b = t2_1.b) AND ((t1_1.c)::text = (t2_1.c)::text))
+                           ->  Seq Scan on plt1_p1 t1_1
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p1 t2_1
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p1 t3_1
+               ->  Hash Join
+                     Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t1_2.b = t2_2.b) AND ((t1_2.c)::text = (t2_2.c)::text))
+                           ->  Seq Scan on plt1_p2 t1_2
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p2 t2_2
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p2 t3_2
+               ->  Hash Join
+                     Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2_3.b = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text))
+                           ->  Seq Scan on plt2_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p3 t3_3
+(41 rows)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p5 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_3
+                                 ->  Seq Scan on plt2_p3 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
  Sort
-   Sort Key: prt1_m_p1.a, prt2_m_p1.b
-   ->  Append
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
-               ->  Seq Scan on prt1_m_p1
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p1
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
-               ->  Seq Scan on prt1_m_p2
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p2
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
-               ->  Seq Scan on prt1_m_p3
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p3
-                           Filter: (c = 0)
-(24 rows)
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
 
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
-  50 | 0 |     |  
- 100 | 0 |     |  
- 150 | 0 | 150 | 0
- 200 | 0 |     |  
- 250 | 0 |     |  
- 300 | 0 | 300 | 0
- 350 | 0 |     |  
- 400 | 0 |     |  
- 450 | 0 | 450 | 0
- 500 | 0 |     |  
- 550 | 0 |     |  
-     |   |  75 | 0
-     |   | 225 | 0
-     |   | 375 | 0
-     |   | 525 | 0
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
 (16 rows)
 
---
--- tests for list partitioned tables.
---
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
---
--- list partitioned by expression
---
-CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1_e;
--- test partition matching with N-way join
+-- semi join
 EXPLAIN (COSTS OFF)
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                   QUERY PLAN                                   
---------------------------------------------------------------------------------
- GroupAggregate
-   Group Key: t1.c, t2.c, t3.c
-   ->  Sort
-         Sort Key: t1.c, t3.c
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
          ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.c = ltrim(t3.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1.b = t2.b) AND (t1.c = t2.c))
-                           ->  Seq Scan on plt1_p1 t1
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.c = ltrim(t3_1.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
-                           ->  Seq Scan on plt1_p2 t1_1
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.c = ltrim(t3_2.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
-                           ->  Seq Scan on plt1_p3 t1_2
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p3 t3_2
-(32 rows)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Seq Scan on plt2_p2 t2_2
+                                 ->  Seq Scan on plt2_p3 t2_3
+(22 rows)
 
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  75.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 123.0000000000000000 | 123.0000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 225.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 273.0000000000000000 | 273.0000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 375.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 423.0000000000000000 | 423.0000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 525.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 573.0000000000000000 | 573.0000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1241,22 +4028,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
 --------------------------------------------------
  Hash Left Join
    Hash Cond: (t2.b = a)
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t3.a = t2.b)
-               ->  Seq Scan on prt1_p1 t3
-               ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
-         ->  Hash Join
-               Hash Cond: (t3_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t3_1
-               ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
-         ->  Hash Join
-               Hash Cond: (t3_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t3_2
-               ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
+   ->  Hash Join
+         Hash Cond: (t3.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t3
+               ->  Seq Scan on prt1_p1 t3_1
+               ->  Seq Scan on prt1_p2 t3_2
+               ->  Seq Scan on prt1_p3 t3_3
+               ->  Seq Scan on prt1_p4 t3_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
    ->  Hash
          ->  Result
                One-Time Filter: false
@@ -1271,16 +4058,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 --
 -- tests for hash partitioned tables.
@@ -1356,41 +4149,9 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  273.0000000000000000 | 273.0000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
 (6 rows)
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
- Sort
-   Sort Key: t1.a
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
-               ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
-               ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
-                           Filter: (b = 0)
-(21 rows)
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
@@ -1405,26 +4166,24 @@ SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c
    Sort Key: t1.c
    ->  HashAggregate
          Group Key: t1.c, t2.c
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t2.c = t1.c)
-                     ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p1 t1
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on plt1_p4 t1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_1.c = t1_1.c)
-                     ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_p2 t1_1
+                           ->  Seq Scan on plt1_p1 t1_1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_2.c = t1_2.c)
-                     ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Seq Scan on plt1_p2 t1_2
                                  Filter: ((a % 25) = 0)
-(23 rows)
+                           ->  Seq Scan on plt1_p3 t1_3
+                                 Filter: ((a % 25) = 0)
+(21 rows)
 
 --
 -- multiple levels of partitioning
@@ -1826,64 +4585,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
  Hash Join
    Hash Cond: (t1.a = t2.a)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt4_n_p1 t2
                ->  Seq Scan on prt4_n_p2 t2_1
                ->  Seq Scan on prt4_n_p3 t2_2
-(11 rows)
+(13 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a and t1.a = t3.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                        QUERY PLAN                        
+----------------------------------------------------------
  Hash Join
-   Hash Cond: (t2.a = t1.a)
+   Hash Cond: (t3.b = t1.a)
    ->  Append
-         ->  Seq Scan on prt4_n_p1 t2
-         ->  Seq Scan on prt4_n_p2 t2_1
-         ->  Seq Scan on prt4_n_p3 t2_2
+         ->  Seq Scan on prt2_p0 t3
+         ->  Seq Scan on prt2_p1 t3_1
+         ->  Seq Scan on prt2_p2 t3_2
+         ->  Seq Scan on prt2_p3 t3_3
+         ->  Seq Scan on prt2_p4 t3_4
+         ->  Seq Scan on prt2_p5 t3_5
    ->  Hash
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.a = t3_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_2.b)
-                     ->  Seq Scan on prt1_p3 t1_2
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3 t3_2
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt4_n_p1 t2
+                           ->  Seq Scan on prt4_n_p2 t2_1
+                           ->  Seq Scan on prt4_n_p3 t2_2
 (23 rows)
 
 -- partitionwise join can not be applied if there are no equi-join conditions
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (t1.a < b)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (t1.a < b)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (t1.a < b)
-(12 rows)
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+(16 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partitionwise join
@@ -1969,16 +4734,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
-                     ->  Seq Scan on plt1_p1 t3
-                     ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3
+                     ->  Seq Scan on plt1_p1 t3_1
+                     ->  Seq Scan on plt1_p2 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partitionwise join can not be applied for a join between list and range
 -- partitioned table
@@ -1989,14 +4755,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
  Hash Full Join
    Hash Cond: ((t2.c)::text = (t1.c)::text)
    ->  Append
-         ->  Seq Scan on prt1_p1 t2
-         ->  Seq Scan on prt1_p2 t2_1
-         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p0 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p2 t2_2
+         ->  Seq Scan on prt1_p3 t2_3
+         ->  Seq Scan on prt1_p4 t2_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt1_n_p1 t1
                ->  Seq Scan on prt1_n_p2 t1_1
-(10 rows)
+(12 rows)
 
 -- partitionwise join can not be applied if only one of joining table has
 -- default partition
@@ -2012,16 +4780,23 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
    ->  Hash Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index c1c9859651..0e884835fb 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -10,25 +10,39 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -67,11 +81,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -93,20 +115,30 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 
 EXPLAIN (COSTS OFF)
@@ -169,6 +201,128 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -193,28 +347,79 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
 -- test partition matching with N-way join
@@ -222,6 +427,175 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
@@ -267,27 +641,18 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
 ALTER TABLE plt2 DETACH PARTITION plt2_p3;
 ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
 ANALYZE plt2;
-
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.a % 25 = 0 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
 --
 -- multiple levels of partitioning
 --
-- 
2.16.4

#57Dmitry Dolgov
9erthalion6@gmail.com
In reply to: Dmitry Dolgov (#56)
4 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Mon, Nov 26, 2018 at 1:41 PM Dmitry Dolgov <9erthalion6@gmail.com> wrote:

Thanks, I've messed this up too - rebased a wrong branch, the correct one
doesn't have this code already. Sorry for that, and here is the proper version
of this patch.

And one more version, because I haven't fixed the tests (sorry for the noise).

Attachments:

0002-Targetlist-of-a-child-join-is-produced-by-translating-v15.patchapplication/octet-stream; name=0002-Targetlist-of-a-child-join-is-produced-by-translating-v15.patchDownload
From bd68b5832f5d0812767223a332eac26c1cf0513b Mon Sep 17 00:00:00 2001
From: erthalion <9erthalion6@gmail.com>
Date: Tue, 11 Sep 2018 21:09:44 +0200
Subject: [PATCH 2/5] Targetlist of a child-join is produced by translating
 that of parent join

The next patch adds more general partition matching algorithm for
partition-wise join. With that change, the first pair of joining
relations for the parent joinrel may not produce a partition-wise join
because of the restrictions in partition_bounds_merge(), to be added
in the next patch.  Hence the first pair of joining relations, which
is used to build the child join and presented to
build_child_join_rel(), for the child joinrel does not correspond to
the first pair of joining relations for the parent joinrel. The
targetlist built using different pairs have the targetlist entries
arranged in different order. An appendrel expects that all its
children have their targetlists ordered in the same fashion.  Hence
translate the parent's targetlist so that parent and child joinrels
have their targetlists in sync.

Basic partition-wise join commit
f49842d1ee31b976c681322f76025d7732e860f3 modified
build_joinrel_tlist() to build targetlist for child-join. With the
above changes we don't need it anymore.  Similarly, the same commit
added code to set_append_rel_size() to compute attr_needed for a
child-join relation. That change too is not needed with the above
change. This commit reverts those two changes.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/optimizer/util/relnode.c | 20 ++++++++++++++++++++
 1 file changed, 20 insertions(+)

diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 39f5729b91..2f1137c13d 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -802,6 +802,19 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 
 	appinfos = find_appinfos_by_relids(root, joinrel->relids, &nappinfos);
 
+	/*
+	 * The first pair of joining relations for the parent joinrel may not
+	 * produce a partition-wise join because of the restrictions in
+	 * partition_bounds_merge(). Hence the first pair of joining relations,
+	 * which is used to build the child join and presented here, for the child
+	 * joinrel does not correspond to the first pair of joining relations for
+	 * the parent joinrel. The targetlist built using different pairs have the
+	 * targetlist nodes arranged in different order. An appendrel expects that
+	 * all its children have their targetlists ordered in the same fashion.
+	 * Hence translate the parent's targetlist so that parent and child
+	 * joinrels have their targetlists in sync.
+	 */
+	joinrel->reltarget = copy_pathtarget(parent_joinrel->reltarget);
 	/* Set up reltarget struct */
 	build_child_join_reltarget(root, parent_joinrel, joinrel,
 							   nappinfos, appinfos);
@@ -907,6 +920,12 @@ build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 	Relids		relids = joinrel->relids;
 	ListCell   *vars;
 
+	/*
+	 * We only see parent joins. Targetlist of a child-join is computed by
+	 * translating corresponding parent join's targetlist.
+	 */
+	Assert(joinrel->reloptkind == RELOPT_JOINREL);
+
 	foreach(vars, input_rel->reltarget->exprs)
 	{
 		Var		   *var = (Var *) lfirst(vars);
@@ -934,6 +953,7 @@ build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 
 		/* Is it still needed above this joinrel? */
 		ndx = var->varattno - baserel->min_attr;
+
 		if (bms_nonempty_difference(baserel->attr_needed[ndx], relids))
 		{
 			/* Yup, add it to the output */
-- 
2.16.4

0001-Hash-partition-bound-equality-refactoring-v15.patchapplication/octet-stream; name=0001-Hash-partition-bound-equality-refactoring-v15.patchDownload
From bab6680b6b5f8dff042672fc6e08b9a9cc580353 Mon Sep 17 00:00:00 2001
From: erthalion <9erthalion6@gmail.com>
Date: Tue, 11 Sep 2018 21:08:07 +0200
Subject: [PATCH 1/5] Hash partition bound equality refactoring.

Separate the code to check whether two given hash bounds are equal into a
separate function, so that it can be called from multiple places. Right now
it's only caller is partition_bounds_equal() but later we will use it for
merging partition bounds.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/partitioning/partbounds.c | 95 ++++++++++++++++++++++-------------
 1 file changed, 61 insertions(+), 34 deletions(-)

diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index eeaab2f4c9..0af3372cdf 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -104,6 +104,9 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 						 Expr **keyCol,
 						 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static bool partition_hbounds_equal(PartitionBoundInfo b1,
+									PartitionBoundInfo b2);
+
 
 /*
  * get_qual_from_partbound
@@ -652,6 +655,63 @@ create_range_bounds(PartitionBoundSpec **boundspecs, int nparts,
 	return boundinfo;
 }
 
+/*
+ * Are two hash partition bound collections logically equal?
+ *
+ * Hash partition bounds store modulus and remainder in datums array which are
+ * always integers irrespective of the number of partition keys and their data
+ * types. Hence we can compare the hash bound collection without any partition
+ * key specific information. Separating this logic in a function which does not
+ * require partition key specific information allows it be called from places
+ * where the partition key specific information is not completely available.
+ */
+static bool
+partition_hbounds_equal(PartitionBoundInfo b1, PartitionBoundInfo b2)
+{
+	int			greatest_modulus = get_hash_partition_greatest_modulus(b1);
+	int			i;
+
+	Assert(b1->strategy == PARTITION_STRATEGY_HASH &&
+		   b2->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * If two hash partitioned tables have different greatest moduli,
+	 * their partition schemes don't match.  For hash partitioned table,
+	 * the greatest modulus is given by the last datum and number of
+	 * partitions is given by ndatums.
+	 */
+	if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
+		return false;
+
+	/*
+	 * We arrange the partitions in the ascending order of their modulus and
+	 * remainders.  Also every modulus is factor of next larger modulus.
+	 * Therefore we can safely store index of a given partition in indexes
+	 * array at remainder of that partition.  Also entries at (remainder + N *
+	 * modulus) positions in indexes array are all same for (modulus,
+	 * remainder) specification for any partition.  Thus datums array from both
+	 * the given bounds are same, if and only if their indexes array will be
+	 * same.  So, it suffices to compare indexes array.
+	 */
+	for (i = 0; i < greatest_modulus; i++)
+		if (b1->indexes[i] != b2->indexes[i])
+			return false;
+
+#ifdef USE_ASSERT_CHECKING
+
+	/*
+	 * Nonetheless make sure that the bounds are indeed same when the indexes
+	 * match.  Hash partition bound stores modulus and remainder at
+	 * b1->datums[i][0] and b1->datums[i][1] position respectively.
+	 */
+	for (i = 0; i < b1->ndatums; i++)
+		Assert((b1->datums[i][0] == b2->datums[i][0] &&
+				b1->datums[i][1] == b2->datums[i][1]));
+#endif
+
+	return true;
+}
+
 /*
  * Are two partition bound collections logically equal?
  *
@@ -680,41 +740,8 @@ partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
 
 	if (b1->strategy == PARTITION_STRATEGY_HASH)
 	{
-		int			greatest_modulus = get_hash_partition_greatest_modulus(b1);
-
-		/*
-		 * If two hash partitioned tables have different greatest moduli,
-		 * their partition schemes don't match.
-		 */
-		if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
+		if (!partition_hbounds_equal(b1, b2))
 			return false;
-
-		/*
-		 * We arrange the partitions in the ascending order of their moduli
-		 * and remainders.  Also every modulus is factor of next larger
-		 * modulus.  Therefore we can safely store index of a given partition
-		 * in indexes array at remainder of that partition.  Also entries at
-		 * (remainder + N * modulus) positions in indexes array are all same
-		 * for (modulus, remainder) specification for any partition.  Thus
-		 * datums array from both the given bounds are same, if and only if
-		 * their indexes array will be same.  So, it suffices to compare
-		 * indexes array.
-		 */
-		for (i = 0; i < greatest_modulus; i++)
-			if (b1->indexes[i] != b2->indexes[i])
-				return false;
-
-#ifdef USE_ASSERT_CHECKING
-
-		/*
-		 * Nonetheless make sure that the bounds are indeed same when the
-		 * indexes match.  Hash partition bound stores modulus and remainder
-		 * at b1->datums[i][0] and b1->datums[i][1] position respectively.
-		 */
-		for (i = 0; i < b1->ndatums; i++)
-			Assert((b1->datums[i][0] == b2->datums[i][0] &&
-					b1->datums[i][1] == b2->datums[i][1]));
-#endif
 	}
 	else
 	{
-- 
2.16.4

0003-Partition-wise-join-for-1-1-1-0-0-1-partition-matching-v15.patchapplication/octet-stream; name=0003-Partition-wise-join-for-1-1-1-0-0-1-partition-matching-v15.patchDownload
From 7adc97a2c1849d85cdce9dac2047d27347396924 Mon Sep 17 00:00:00 2001
From: erthalion <9erthalion6@gmail.com>
Date: Tue, 11 Sep 2018 21:10:46 +0200
Subject: [PATCH 3/5] Partition-wise join for 1:1, 1:0, 0:1 partition matching.

Earlier version of partition-wise join implementation allowed
partition-wise join between two relations with exactly same partition
bounds. This commit allows partition-wise join to be applied under
following conditions

1. the partition bounds of joining relations are such that rows from
given partition on one side join can join with rows from maximum one
partition on the other side i.e. bounds of a given partition on one
side match/overlap with those of maximum one partition on the other
side. If the mapping happens to be m:n where m > 1 or n > 1, we have
to gang multiple partition relations together into a single relation.
This means that we have to add simple relations during join
processing, something which is not supported right now.  ALso, in such
a case, different pairs of joining relations can produce different
partition bounds for the same join relation, which again is not
supported right now.

2. For every partition on outer side that can contribute to the result
of an OUTER join, there exists at least one (taken along with item 1,
it means exactly one)  matching partition on the inner side. To
support partition-wise join when the inner matching partition doesn't
exist, we have to add a dummy base relation corresponding to the
non-existent inner partition. We don't have support to add base
relations during join processing.

3. For hash partitioned tables however, we support partition-wise join
only when the bounds exactly match. For hash partitioning it's unusual
to have missing partitions and hence generic partition matching is not
required.

With advanced partition matching, we won't be able to apply
partition-wise join when there are missing matching partitions. So for
a given N-way join, some sub-joins may not be partitioned, while the
overall N-way join is partitioned. We can not try partition-wise join
when one or both of the joining relations are not partitioned in one
of the pairs of joining relation. Skip partition-wise join for that
pair.

An example is A IJ B LJ C where B has an extra partition compared to A
and C. Join B LJ C requires dummy relation to join the extra partition
from B to a missing partition from C, and hence can not use
partition-wise join right now and hence BC is not partitioned.  The
extra partition in B gets eliminated in A IJ B and thus it can use
partition-wise join and is partitioned. Hence (AB)C can use
partition-wise join but not A(BC).

Not every pair of joining relation (including the one presented to
build_joinrel_partition_info()) for the same joinrel can use
partition-wise join or has both the relations partitioned. Hence we
calculate the partition bounds for the join relation in
try_partition_wise_join() instead of doing it here.

The partition bounds produced for the first pair that can use
partition-wise are saved in the RelOptInfo of the joinrel. Partition
bounds produced for subsequent pairs should match that produced by the
first pair.

Support for default partition in list partitioned tables
========================================================

When a list value is present in one of the joining relations and not
the other, and the other relation has default partition, match (join)
the partition containing that list value with the default partition.
If there are multiple matches, we can not proceed with the
partition-wise join similar to the case of matching non-default
partition. If the default partition happens to be on the outer side of
the join, the resulting join partition acts as a default partition as
it will contain all the values from the default partition. If
the partition containing the list value happens to be on the outer
side of the join, the resulting join partition is associated with the
list value, since no other partition key value from the default
partition makes it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join.  Such a
default partition may get joined to a partition from the inner side,
if inner side has a list value that is not present in the outer side.
But if such a join partner does not exist, we won't be able to use
partition-wise join since there is no (dummy) relation on the inner
side to join to.

Support for matching default partition in range partitioned tables
==================================================================

When a range is present in one of the joining relations and not the
other, and the other relation has default partition, match (join) the
partition corresponding range with the default partition. If two
ranges overlap but their ranges don't match exactly, the
non-overlapping portion from one range may join the previous
(non-overlapping portion near the lower bound, if exists), next
(non-overlapping portion near the upper bound, if exists) ranges from
the other relation or default partition from the other relation. This
causes multiple partitions on the other side to be joined with a
single partition on the first side; a case we don't support. Any range
from one side which doesn't overlap with any range on the other side,
may find its join partner in the default partition and the
corresponding range partition will need to be joined with the default
partition.  If the default partition happens to be on the outer side
of the join, the resulting join partition acts as a default partition
as it will contain all the values from the default partition. If the
non-partition corresponding to the range happens to be on the outer
side of the join, the resulting join partition is associated with that
range, since partition key values from the default partition outside
that range won't make it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join.  Such a
default partition may get joined to a non-default partition from the
inner side, if inner side has a range missing in the outer side.  But
if such a join partner does not exist, we won't be able to use
partition-wise join since there is no (dummy) relation on the inner
side to join to.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/optimizer/path/joinrels.c |   95 +-
 src/backend/optimizer/util/relnode.c  |   33 +-
 src/backend/partitioning/partbounds.c | 1676 ++++++++++++++++++++++++++++++++-
 src/include/partitioning/partbounds.h |    5 +
 4 files changed, 1763 insertions(+), 46 deletions(-)

diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index d3d21fed5d..e476079143 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1312,25 +1312,31 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 					   RelOptInfo *joinrel, SpecialJoinInfo *parent_sjinfo,
 					   List *parent_restrictlist)
 {
-	int			nparts;
 	int			cnt_parts;
+	PartitionScheme part_scheme;
+	PartitionBoundInfo join_boundinfo;
+	List	   *parts1;
+	List	   *parts2;
+	ListCell   *lc1;
+	ListCell   *lc2;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1343,32 +1349,63 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	 */
 	Assert(joinrel->part_scheme == rel1->part_scheme &&
 		   joinrel->part_scheme == rel2->part_scheme);
+	part_scheme = joinrel->part_scheme;
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * Get the list of matching partitions to be joined along with the
+	 * partition bounds of the join relation. Because of the restrictions
+	 * imposed by partition matching algorithm, not every pair of joining
+	 * relations for this join will be able to use partition-wise join. But all
+	 * those pairs which can use partition-wise join will produce the same
+	 * partition bounds for the join relation.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	join_boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											part_scheme->partsupfunc,
+											part_scheme->partcollation,
+											rel1, rel2,
+											parent_sjinfo->jointype,
+											&parts1, &parts2);
+
+	if (join_boundinfo == NULL)
+		return;
 
-	nparts = joinrel->nparts;
+	if (joinrel->boundinfo == NULL)
+	{
+		Assert(joinrel->nparts == 0 && joinrel->part_rels == NULL);
+		joinrel->boundinfo = join_boundinfo;
+		joinrel->nparts = list_length(parts1);
+		Assert(joinrel->nparts == list_length(parts2));
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) *
+									joinrel->nparts);
+	}
+	else
+	{
+		Assert(partition_bounds_equal(part_scheme->partnatts,
+									  part_scheme->parttyplen,
+									  part_scheme->parttypbyval,
+									  join_boundinfo, joinrel->boundinfo));
+		/*
+		 * Every pair of joining relations should result in the same number
+		 * of child-joins.
+		 */
+		Assert(joinrel->nparts == list_length(parts1));
+		Assert(joinrel->nparts == list_length(parts2));
+		Assert(joinrel->part_rels);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	cnt_parts = 0;
+	forboth(lc1, parts1, lc2, parts2)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
+		int			part1 = lfirst_int(lc1);
+		int			part2 = lfirst_int(lc2);
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1376,6 +1413,10 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		Assert(part1 >= 0 && part2 >= 0);
+		child_rel1 = rel1->part_rels[part1];
+		child_rel2 = rel2->part_rels[part2];
+
 		/* We should never try to join two overlapping sets of rels. */
 		Assert(!bms_overlap(child_rel1->relids, child_rel2->relids));
 		child_joinrelids = bms_union(child_rel1->relids, child_rel2->relids);
@@ -1409,12 +1450,24 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 			joinrel->part_rels[cnt_parts] = child_joinrel;
 		}
 
+		/*
+		 * For every pair of joining relations, the set of matching partitions
+		 * would change. However, the base relation partitions constituting
+		 * the given child should remain same for all the joining pairs. Since
+		 * the order in which children are stored in the array of child-joins,
+		 * depends upon partition bounds of the join, which are same for all
+		 * the joining pairs, every joining pair yields the child-joins in the
+		 * same order.
+		 */
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
 
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
+		cnt_parts++;
 	}
+
+	Assert(cnt_parts == joinrel->nparts);
 }
 
 /*
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 2f1137c13d..4f15cd8e8b 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1627,7 +1627,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1640,24 +1640,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1669,17 +1651,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 0af3372cdf..771520379d 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -65,6 +65,12 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+typedef struct PartitionMap
+{
+	int from;
+	int to;
+} PartitionMap;
+
 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);
@@ -106,7 +112,58 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 static List *get_range_nulltest(PartitionKey key);
 static bool partition_hbounds_equal(PartitionBoundInfo b1,
 									PartitionBoundInfo b2);
-
+static PartitionBoundInfo partition_range_bounds_merge(
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 List **outer_parts, List **inner_parts,
+							 JoinType jointype, int partnatts,
+							 FmgrInfo *supfuncs, Oid *collations);
+static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype);
+static PartitionBoundInfo partition_hash_bounds_merge(RelOptInfo *outer_rel,
+							RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype);
+static void generate_matching_part_pairs(PartitionMap *outer_maps,
+										 PartitionMap *inner_maps,
+										 int nparts1, int nparts2,
+										 JoinType jointype, int nparts,
+										 List **parts1, List **parts2);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int map_and_merge_partitions(PartitionMap *outer_maps,
+										PartitionMap *inner_maps,
+										int index1, int index2, int *next_index);
+static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *collations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2);
+static bool partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
+						   Oid *collations, PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, int *ub_cmpval,
+						   int *lb_cmpval);
+static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes);
+static bool merge_default_partitions(PartitionBoundInfo outer_bi,
+						 			 PartitionBoundInfo inner_bi,
+						 			 PartitionMap *outer_maps,
+						 			 PartitionMap *inner_maps,
+						 			 JoinType jointype,
+									 int *next_index, int *default_index);
+static bool merge_null_partitions(PartitionBoundInfo outer_bi,
+								  PartitionBoundInfo inner_bi,
+								  PartitionMap *outer_maps,
+								  PartitionMap *inner_maps,
+								  JoinType jointype,
+								  int *next_index, int *null_index,
+								  int *default_index);
 
 /*
  * get_qual_from_partbound
@@ -2941,3 +2998,1620 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * The function produces the partition bounds for a join between two relations
+ * whose partition bounds are given. The function also returns two lists of
+ * partition indexes one for each of the joining relations. Both the lists
+ * contain the same number of elements. The partition indexes at the same
+ * positions in the lists indicate the pair partitions, one from each side, to
+ * be joined and the position itself corresponds to the index of partition
+ * produced by that child-join in the partitioned join.
+ *
+ * The function returns NULL if we can not find the matching pair of
+ * partitions. This happens if 1. multiple partitions on one side match with
+ * one partition on the other side. 2. a given partition on the outer side
+ * doesn't have a matching partition on the inner side. We can not support the
+ * first case since we don't have a way to represent multiple partitions as a
+ * single relation (RelOptInfo) and then perform join using the ganged
+ * relation. We can not support the second case since the missing inner
+ * partition needs to be represented as an empty relation and we don't have a
+ * way to introduce empty relation during join planning after creating paths
+ * for all the base relations.
+ */
+extern PartitionBoundInfo
+partition_bounds_merge(int partnatts, FmgrInfo *partsupfunc,
+					   Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
+{
+	PartitionBoundInfo 	merged_bounds;
+	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
+					   	inner_binfo = inner_rel->boundinfo;
+	char				strategy = outer_binfo->strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partsupfunc,
+														partcollation,
+														outer_rel, inner_rel,
+														outer_parts, inner_parts,
+														jointype);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(outer_rel, inner_rel,
+														 outer_parts, inner_parts,
+														 jointype, partnatts,
+														 partsupfunc,
+														 partcollation);
+			break;
+
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = partition_hash_bounds_merge(outer_rel, inner_rel,
+														outer_parts, inner_parts,
+														jointype);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	Assert((*outer_parts == NIL || *inner_parts != NIL) &&
+		   (*inner_parts == NIL || *outer_parts != NIL));
+
+	return merged_bounds;
+}
+
+/*
+ * partition_get_range_bounds
+ *
+ * Given the index of lower bound in datums array, return lower and upper
+ * bounds and the index of the partition with that lower bound.
+ */
+static int
+partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
+						   PartitionRangeBound *lower,
+						   PartitionRangeBound *upper)
+{
+	int			part_index;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The lower bound should correspond to a valid partition. */
+	part_index = bi->indexes[lb_index + 1];
+	Assert(part_index >= 0);
+
+	lower->kind = bi->kind[lb_index];
+	lower->datums = bi->datums[lb_index];
+	lower->lower = true;
+	upper->kind = bi->kind[lb_index + 1];
+	upper->datums = bi->datums[lb_index + 1];
+	upper->lower = false;
+
+	return part_index;
+}
+
+/*
+ * partition_range_get_next_lb_index
+ *
+ * Given the index of lower bound in datums array return the
+ * index of lower bound of the next partition. When the given index corresponds
+ * to the last partition, return number of datums (ndatums).
+ */
+static int
+partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
+{
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The partition index corresponding to the upper bound should be valid. */
+	Assert(bi->indexes[lb_index + 1] >= 0);
+
+	/*
+	 * If there are no bounds left beyond the upper bound, we have reached the
+	 * last partition.
+	 */
+	if (lb_index + 2 < bi->ndatums)
+	{
+		/*
+		 * If the bound next to the upper bound corresponds to no partition,
+		 * that's the next lower bound of the next partition. Otherwise, the
+		 * current upper bound is the lower bound of the next partition.
+		 */
+		if (bi->indexes[lb_index + 2] < 0)
+			return lb_index + 2;
+		else
+			return lb_index + 1;
+	}
+	else
+		return bi->ndatums;
+}
+
+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *partcollations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2)
+{
+	return partition_rbound_cmp(partnatts, partsupfunc, partcollations,
+								bound1->datums, bound1->kind, bound1->lower,
+								bound2);
+}
+
+/*
+ * partition_range_cmp
+ *
+ * Compare the bounds of two range partitions. Set ub_cmpval <, = or > 0, if the
+ * first partition's upper bound is lower than, equal to or higher than the
+ * second partition's upper bound resp. Similarly set lb_cmpval <, =  or > 0,
+ * if the first partition's lower bound is lower than, equal to or higher than
+ * the second partition's lower bound resp.
+ *
+ * Return true, if the ranges overlap, otherwise return false.
+ */
+static bool
+partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
+					PartitionRangeBound *lower_bound1,
+					PartitionRangeBound *upper_bound1,
+					PartitionRangeBound *lower_bound2,
+					PartitionRangeBound *upper_bound2, int *ub_cmpval,
+					int *lb_cmpval)
+{
+	bool		overlap;
+
+	/*
+	 * Compare upper bound of the first partition with the lower bound of the
+	 * second and vice-versa. If lower bound is higher than the upper bound,
+	 * the partitions are not overlapping. All other cases indicate overlapping
+	 * partitions.
+	 */
+	if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+								  lower_bound1, upper_bound2) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = 1;
+		*lb_cmpval = 1;
+	}
+	else if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+									   lower_bound2, upper_bound1) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = -1;
+		*lb_cmpval = -1;
+	}
+	else
+	{
+		overlap = true;
+		*ub_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, upper_bound1,
+											   upper_bound2);
+		*lb_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, lower_bound1,
+											   lower_bound2);
+	}
+
+	return overlap;
+}
+
+/*
+ * partition_range_merge
+ *
+ * Merge the partition bounds of given two partitions such that the join
+ * between the given two partitions fits merged bounds.
+ *
+ * "merged_upper" will be set to one of the given upper bounds and
+ * "merged_lower" will be set to one of the given lower bounds.
+ */
+static void
+partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
+					  Oid *partcollations, JoinType jointype,
+					  PartitionRangeBound *left_lb,
+					  PartitionRangeBound *left_ub,
+					  PartitionRangeBound *right_lb,
+					  PartitionRangeBound *right_ub,
+					  PartitionRangeBound **merged_lb,
+					  PartitionRangeBound **merged_ub)
+{
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = left_ub;
+			*merged_lb = left_lb;
+			break;
+
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) < 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) > 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		case JOIN_FULL:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) > 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) < 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+
+	return;
+}
+
+/*
+ * Add the lower bound of the next range to the list of bounds, if the lower
+ * bound is higher or equal to the previous upper bound. If successful return
+ * true, otherwise false.
+ */
+static bool
+partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
+							  Oid *partcollations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		prev_ub.datums = llast(*merged_datums);
+		prev_ub.kind = llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  next_lb_datums, next_lb_kind, false,
+									  &prev_ub);
+	}
+
+	/*
+	 * The lower bound is lower than the last upper bound, thus does not fit
+	 * the bounds created so far and hence can not be merged with the existing
+	 * bounds.
+	 */
+	if (cmpval < 0)
+		return false;
+
+	/*
+	 * Add bounds of the new merged partition. If the next lower bound is
+	 * higher than the last upper bound, add new range with index
+	 * corresponding to the lower bound as -1. If the merged lower bound
+	 * is same as the last merged upper bound, the last upper bound will be
+	 * reused as the lower bound of the next range.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, next_lb_datums);
+		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	return true;
+}
+
+/*
+ * handle_missing_partition
+ *
+ * If a range appears in one of the joining relations but not the other, a row
+ * in the corresponding partition will not have any join partner in the other
+ * relation, unless the other relation has a default partition. If a given list
+ * value is present in one joining relation but not the other, the default
+ * partition on the other side may contain that value.
+ *
+ * In both these cases, such an extra partition forms a joining pair with the
+ * default partition, if any,  on the other side.
+ *
+ * If the default partition happens to be on the outer side of the join, the
+ * resultant partition will act as the default partition of the join relation.
+ * Otherwise the resultant partition will be associated with the range.
+ *
+ * When the default partition is not present in the other relation, the rows in
+ * the extra partition will be included in the bounds of the join result, if it
+ * appears on the outer side of the join, since all rows from the outer side
+ * are included in the join result.
+ *
+ * This function handles all these cases.
+ *
+ * maps_with_missing and missing_side_default are the partition maps (See
+ * partition_range/list_bounds_merge() for details) and the index of default
+ * partition respectively corresponding the side with missing partition.
+ *
+ * maps_with_extra and extra_part are the partition maps (See
+ * partition_range/list_bounds_merge() for details) and the index of extra
+ * partition respectively corresponding to the side with the extra partition.
+ *
+ * It returns true if the matching succeeds, otherwise returns false.
+ */
+static bool
+handle_missing_partition(PartitionMap *maps_with_missing,
+						 PartitionMap *maps_with_extra,
+						 int missing_side_default,
+						 int extra_part,
+						 bool missing_side_outer,
+						 bool missing_side_inner,
+						 int *next_index, int *default_index,
+						 int *merged_index)
+{
+	bool missing_has_default = (missing_side_default != -1);
+
+	if (missing_has_default)
+	{
+		*merged_index = map_and_merge_partitions(maps_with_missing,
+												 maps_with_extra,
+												 missing_side_default,
+												 extra_part,
+												 next_index);
+		if (*merged_index < 0)
+			return false;
+
+		if (missing_side_outer)
+		{
+			/*
+			 * Default partition on the outer side forms the default
+			 * partition of the join result.
+			 */
+			if (*default_index < 0)
+				*default_index = *merged_index;
+			else if(*default_index != *merged_index)
+			{
+				/*
+				 * Ended up with default partition on the outer side
+				 * being joined with multiple partitions on the inner
+				 * side. We don't support this case.
+				 */
+				return false;
+			}
+
+			/*
+			 * Since the merged partition acts as a default partition, it
+			 * doesn't need a separate index.
+			 */
+			*merged_index = -1;
+		}
+	}
+	else if (missing_side_inner)
+	{
+		/*
+		 * If this partition has already been mapped (say because we
+		 * found an overlapping range earlier), we know where does it
+		 * fit in the join result. Nothing to do in that case. Else
+		 * create a new merged partition.
+		 */
+		PartitionMap *extra_map = &maps_with_extra[extra_part];
+		if (extra_map->to < 0)
+		{
+			extra_map->to = *next_index;
+			*next_index = *next_index + 1;
+			*merged_index = extra_map->to;
+		}
+	}
+
+	return true;
+}
+
+static PartitionMap*
+init_partition_map(RelOptInfo *rel)
+{
+	int i, nparts = rel->nparts;
+	PartitionMap *map;
+
+	map = (PartitionMap *) palloc(sizeof(PartitionMap) * nparts);
+
+	for (i = 0; i < nparts; i++)
+	{
+		map[i].from = -1;
+		map[i].to = -1;
+	}
+
+	return map;
+}
+
+/*
+ * Allocate and initialize partition maps. We maintain four maps, two maps
+ * for each joining relation. pmap[i] gives the partition from the other
+ * relation which would join with ith partition of the given relation.
+ * Partition i from the given relation will join with partition pmap[i]
+ * from the other relation to produce partition mmap[i] of the join (merged
+ * partition).
+ *
+ * pmap[i] = -1 indicates that ith partition of a given relation does not
+ * have a matching partition from the other relation.
+ *
+ * mmap[i] = -1 indicates that ith partition of a given relation does not
+ * contribute to the join result. That can happen only when the given
+ * relation is the inner relation and it doesn't have a matching partition
+ * from the outer relation, hence pmap[i] should be -1.
+ *
+ * In case of an outer join, every partition of the outer join will appear
+ * in the join result, and thus has mmap[i] set for all i. But it's not
+ * necessary that every partition on the outer side will have a matching
+ * partition on the inner side. In such a case, we end up with pmap[i] = -1
+ * and mmap[i] != -1.
+ */
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 List **outer_parts, List **inner_parts,
+							 JoinType jointype, int partnatts,
+							 FmgrInfo *partsupfuncs, Oid *partcollations)
+
+{
+	PartitionMap *outer_maps = NULL;
+	PartitionMap *inner_maps = NULL;
+	int			outer_part = 0;
+	int			inner_part = 0;
+	PartitionBoundInfo merged_bounds = NULL;
+	int			outer_lb_index;
+	int			inner_lb_index;
+	int			next_index;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	List	   *merged_kinds = NIL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			inner_default = inner_bi->default_index;
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	outer_maps = init_partition_map(outer_rel);
+	inner_maps = init_partition_map(inner_rel);
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = 0;
+	inner_lb_index = 0;
+	next_index = 0;
+	while (outer_lb_index < outer_bi->ndatums ||
+		   inner_lb_index < inner_bi->ndatums)
+	{
+		PartitionRangeBound outer_lb, outer_ub,
+							inner_lb, inner_ub,
+							*merged_lb = NULL,
+							*merged_ub = NULL;
+
+		int			merged_index = -1;
+		bool		overlap;
+		bool		finished_outer = false;
+		bool		finished_inner = false;
+
+		/* Result of bounds comparison per partition_rbound_cmp(). */
+		int			ub_cmpval;	/* Upper bounds comparison result. */
+		int			lb_cmpval;	/* Lower bounds comparison result. */
+
+		/* Get the range bounds of the next pair of partitions. */
+		if (outer_lb_index < outer_bi->ndatums)
+			outer_part = partition_get_range_bounds(outer_bi, outer_lb_index,
+												&outer_lb, &outer_ub);
+		else
+			finished_outer = true;
+
+		if (inner_lb_index < inner_bi->ndatums)
+			inner_part = partition_get_range_bounds(inner_bi, inner_lb_index,
+												&inner_lb, &inner_ub);
+		else
+			finished_inner = true;
+
+		Assert(!finished_outer || !finished_inner);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (finished_outer)
+		{
+			overlap = false;
+			ub_cmpval = 1;
+			lb_cmpval = 1;
+		}
+		else if (finished_inner)
+		{
+			overlap = false;
+			ub_cmpval = -1;
+			lb_cmpval = -1;
+		}
+		else
+			overlap = partition_range_cmp(partnatts, partsupfuncs, partcollations,
+										  &outer_lb, &outer_ub, &inner_lb,
+										  &inner_ub, &ub_cmpval, &lb_cmpval);
+
+		if (overlap)
+		{
+			/*
+			 * The rows from overlapping portion of ranges on both sides may
+			 * join, hence the corresponding pair of partitions form a joining
+			 * pair. Match them and produce the bounds of the joint partition
+			 * and its index by merging the bounds according to the type of
+			 * join.
+			 */
+			partition_range_merge(partnatts, partsupfuncs, partcollations,
+								  jointype, &outer_lb, &outer_ub, &inner_lb,
+								  &inner_ub, &merged_lb, &merged_ub);
+
+			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+													outer_part, inner_part,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				/* Failed to match the partitions. */
+				return NULL;
+			}
+
+			/*
+			 * If the ranges overlap but don't exactly match, a row from
+			 * non-overlapping portion of the range from one side of join may
+			 * find its join partner in the previous or next overlapping
+			 * partition or default partition on the other side , if such a
+			 * partition exists. All those cases, if true, will cause one
+			 * partition from that side to match at least two partitions on the
+			 * other side; a case that we do not support now. Previous
+			 * partition has been delt with in the previous iteration of this
+			 * loop, next partition will be delt in the next iteration. We will
+			 * deal with the default partition here.
+			 */
+			if ((lb_cmpval < 0 && inner_has_default) ||
+				/* Non-overlapping range on the lower side of outer range. */
+				(lb_cmpval > 0 && outer_has_default) ||
+				/* Non-overlapping range on the lower side of inner range. */
+				(ub_cmpval < 0 && outer_has_default) ||
+				/* Non-overlapping range on the upper side of inner range. */
+				(ub_cmpval > 0 && inner_has_default))
+				/* Non-overlapping range on the upper side of outer range. */
+				return NULL;
+		}
+
+		if (ub_cmpval == 0)
+		{
+			/* Upper bounds of both the ranges match. */
+			Assert(overlap);
+
+			/* Move to the next pair of partitions. */
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the inner side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &outer_lb;
+				merged_ub = &outer_ub;
+
+				/*
+				 * For a FULL join, inner relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the inner relation acts as
+				 * INNER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_inner = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_outer = (jointype == JOIN_FULL);
+				if (!handle_missing_partition(inner_maps,
+											  outer_maps,
+											  inner_default,
+											  outer_part,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the outer side. */
+			Assert(!finished_outer);
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+		}
+		else
+		{
+			Assert(ub_cmpval > 0);
+
+			/* Upper bound of outer range higher than that of the inner. */
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the outer side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &inner_lb;
+				merged_ub = &inner_ub;
+
+				/*
+				 * For a FULL join, outer relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the outer relation acts as
+				 * OUTER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_outer = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_inner = (jointype == JOIN_FULL);
+
+				if (!handle_missing_partition(outer_maps,
+											  inner_maps,
+											  outer_default,
+											  inner_part,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the inner side. */
+			Assert (!finished_inner);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+
+		if (merged_index < 0)
+		{
+			/* We didn't find a new merged partition. */
+			continue;
+		}
+
+		/*
+		 * We have a valid partition index for the next partition of join. The
+		 * partition should have valid range.
+		 */
+		Assert(merged_lb && merged_ub);
+
+		/* Try merging new lower bound with the last upper bound. */
+		if (!partition_range_merge_next_lb(partnatts, partsupfuncs,
+										   partcollations,
+										   merged_lb->datums,
+										   merged_lb->kind, &merged_datums,
+										   &merged_kinds, &merged_indexes))
+			return NULL;
+
+		/* Add upper bound with the merged partition index. */
+		merged_datums = lappend(merged_datums, merged_ub->datums);
+		merged_kinds = lappend(merged_kinds, merged_ub->kind);
+		merged_indexes = lappend_int(merged_indexes, merged_index);
+	}
+
+	if (!merge_default_partitions(outer_bi, inner_bi,
+										  outer_maps, inner_maps,
+										  jointype, &next_index,
+										  &default_index))
+		return NULL;
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_maps, inner_maps,
+								 outer_nparts, inner_nparts,
+								 jointype, next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes,
+													  merged_kinds,
+													  -1, default_index);
+	}
+
+	/* Free any memory we used in this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype)
+{
+	PartitionMap *outer_maps = NULL;
+	PartitionMap *inner_maps = NULL;
+	int			cnto;
+	int			cnti;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			next_index = 0;
+	int			null_index;
+	int			default_index = -1;
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			      *outer_indexes = outer_bi->indexes;
+	int			      *inner_indexes = inner_bi->indexes;
+	int				   outer_default = outer_bi->default_index;
+	int				   inner_default = inner_bi->default_index;
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	outer_maps = init_partition_map(outer_rel);
+	inner_maps = init_partition_map(inner_rel);
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			o_index;
+		int			i_index;
+		int			cmpval;
+		int			merged_index = -1;
+		Datum	   *merged_datum;
+		bool		finished_inner;
+		bool		finished_outer;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+		{
+			finished_outer = true;
+			odatums = NULL;
+			o_index = -1;
+		}
+		else
+		{
+			finished_outer = false;
+			odatums = outer_bi->datums[cnto];
+			o_index = outer_indexes[cnto];
+		}
+
+		if (cnti >= inner_bi->ndatums)
+		{
+			finished_inner = true;
+			idatums = NULL;
+			i_index = -1;
+		}
+		else
+		{
+			finished_inner = false;
+			idatums = inner_bi->datums[cnti];
+			i_index = inner_indexes[cnti];
+		}
+
+		/* If we exhausted both the sides, we won't enter the loop. */
+		Assert(!finished_inner || !finished_outer);
+
+		if (finished_outer)
+			cmpval = 1;
+		else if (finished_inner)
+			cmpval = -1;
+		else
+		{
+			/* Every list datum should map to a valid partition index. */
+			Assert(o_index >= 0 && i_index >= 0 &&
+				   odatums != NULL && idatums != NULL);
+
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0], idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+													o_index, i_index,
+													&next_index);
+
+			if (merged_index < 0)
+				return NULL;
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			/* A datum missing from the inner side. */
+			merged_index = -1;
+			merged_datum = odatums;
+
+			/*
+			 * For a FULL join, inner relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the inner relation acts as
+			 * INNER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_inner = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_outer = (jointype == JOIN_FULL);
+
+			if (!handle_missing_partition(inner_maps,
+										  outer_maps,
+										  inner_default,
+										  o_index,
+										  missing_side_outer,
+										  missing_side_inner,
+										  &next_index,
+										  &default_index,
+										  &merged_index))
+				return NULL;
+
+			/* Move to the next datum on the outer side. */
+			Assert(!finished_outer);
+			cnto++;
+		}
+		else
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			Assert(cmpval > 0);
+
+			/* A datum missing from the outer side. */
+			merged_index = -1;
+			merged_datum = idatums;
+
+			/*
+			 * For a FULL join, outer relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the outer relation acts as
+			 * OUTER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_outer = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_inner = (jointype == JOIN_FULL);
+
+			if (!handle_missing_partition(outer_maps,
+										  inner_maps,
+										  outer_default,
+										  i_index,
+										  missing_side_outer,
+										  missing_side_inner,
+										  &next_index,
+										  &default_index,
+										  &merged_index))
+				return NULL;
+
+			/* Move to the next datum on the right side. */
+			Assert(!finished_inner);
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	if (!merge_null_partitions(outer_bi, inner_bi,
+							   outer_maps, inner_maps,
+							   jointype, &next_index, &null_index,
+							   &default_index))
+		return NULL;
+
+	if (!merge_default_partitions(outer_bi, inner_bi,
+								  outer_maps, inner_maps,
+								  jointype, &next_index,
+								  &default_index))
+		return NULL;
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_maps, inner_maps,
+								 outer_nparts, inner_nparts,
+								 jointype, next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes, NIL,
+													  null_index, default_index);
+	}
+
+	/* Free up all extra memory before returning from this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_bounds_merge()'s arm for hash partitioned tables.
+ *
+ * If the given two hash bounds are same, the function returns the first one
+ * without any change, alongwith the lists of matching partitions. Otherwise it
+ * returns NULL.
+ *
+ * We could try merging the bounds when both the bounds have same greatest
+ * modulii. But there seems to be hardly any requirement for the same.
+ */
+static PartitionBoundInfo
+partition_hash_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype)
+{
+	int			nparts;
+	int			cnt;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * Hash partitioned table does not have explicit NULL accepting partition
+	 * and also does not have a default partition.
+	 */
+	Assert(!partition_bound_has_default(outer_bi) &&
+		   !partition_bound_has_default(inner_bi));
+	Assert(!partition_bound_accepts_nulls(outer_bi) &&
+		   !partition_bound_accepts_nulls(inner_bi));
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+
+	if (outer_nparts != inner_nparts)
+		return NULL;
+	nparts = outer_nparts;
+
+	if (outer_bi->ndatums != inner_bi->ndatums ||
+		!partition_hbounds_equal(outer_bi, inner_bi))
+		return NULL;
+
+	 /*
+	  * Cook up list of matching partitions. Since bounds are exactly same the
+	  * partitions at the same position from both the relations match.
+	  */
+	for (cnt = 0; cnt < nparts; cnt++)
+	{
+		*outer_parts = lappend_int(*outer_parts, cnt);
+		*inner_parts = lappend_int(*inner_parts, cnt);
+	}
+
+	return outer_bi;
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * If the two given partitions (given by index1 and index2 resp.) are
+ * already mapped to each other return the index of corresponding partition in
+ * the merged set of partitions.  If they do not have a merged partition
+ * associated with them, assign a new merged partition index.  If the
+ * partitions are already mapped and their mapped partitions are different from
+ * each other, they can not be merged, so return -1.
+ *
+ * partmaps1[i] gives the mapping of partitions for both relations. It
+ * describes which partition of relation 2 matches ith partition of relation 1,
+ * and which partition in the merged set matches ith partition of relation 1
+ * maps to. Similarly for partmap2.
+ *
+ * index1 and index2 are the indexes of matching partition from respective
+ * relations.
+ *
+ * *next_index is used and incremented when the given partitions require a new
+ * merged partition.
+ */
+
+static int
+map_and_merge_partitions(PartitionMap *partmaps1, PartitionMap *partmaps2,
+						 int index1, int index2, int *next_index)
+{
+	PartitionMap 	*partmap1 = &partmaps1[index1];
+	PartitionMap 	*partmap2 = &partmaps2[index2];
+	int				merged_index;
+
+	/*
+	 * If both the partitions are not mapped to each other, update the
+	 * maps.
+	 */
+	if (partmap1->from < 0 && partmap2->from < 0)
+	{
+		partmap1->from = index2;
+		partmap2->from = index1;
+	}
+
+	/*
+	 * If the given to partitions map to each other, find the corresponding
+	 * merged partition index .
+	 */
+	if (partmap1->from == index2 && partmap2->from == index1)
+	{
+		/*
+		 * If both the partitions are mapped to the same merged partition, get
+		 * the index of merged partition.
+		 */
+		if (partmap1->to == partmap2->to)
+		{
+			merged_index = partmap1->to;
+
+			/*
+			 * If the given two partitions do not have a merged partition
+			 * associated with them, allocate a new merged partition.
+			 */
+			if (merged_index < 0)
+			{
+				merged_index = *next_index;
+				*next_index = *next_index + 1;
+				partmap1->to = merged_index;
+				partmap2->to = merged_index;
+			}
+		}
+
+		/*
+		 * If partition from one relation was mapped to a merged partition but
+		 * not the partition from the other relation, map the same merged
+		 * partition to the partition from other relation, since matching
+		 * partitions map to the same merged partition.
+		 */
+		else if (partmap1->to >= 0 && partmap2->to < 0)
+		{
+			partmap2->to = partmap1->to;
+			merged_index = partmap1->to;
+		}
+		else if (partmap1->to < 0 && partmap2->to >= 0)
+		{
+			partmap1->to = partmap2->to;
+			merged_index = partmap2->to;
+		}
+		else
+		{
+			Assert(partmap1->to != partmap2->to &&
+				   partmap1->to >= 0 && partmap2->to >= 0);
+
+			/*
+			 * Both the partitions map to different merged partitions. This
+			 * means that multiple partitions from one relation matches to one
+			 * partition from the other relation. Partition-wise join does not
+			 * handle this case right now, since it requires ganging multiple
+			 * partitions together (into one RelOptInfo).
+			 */
+			merged_index = -1;
+		}
+	}
+	else
+	{
+		/*
+		 * Multiple partitions from one relation map to one partition from the
+		 * other relation. Partition-wise join does not handle this case right
+		 * now, since it requires ganging multiple partitions together (into
+		 * one RelOptInfo).
+		 */
+		merged_index = -1;
+	}
+
+	return merged_index;
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * partmaps1 map each partition from either side of the join to a merged
+ * partition resp. E.g. partmaps1[i].to gives the merged partition to which ith
+ * partition of first relation maps. Similarly for partmap2. If
+ * partmaps1[i].to == partmaps2[j].to, i and j form the matching pair of
+ * partitions.
+ *
+ * Given these maps this function produces the list pairs of partitions which
+ * when joined produce the merged partitions in the order of merged partition
+ * indexes.
+ *
+ * nparts1 and nparts2 are the number of partitions of the joining relations
+ * resp.
+ *
+ * nparts is the number of merged partitions.
+ *
+ * If successful, the pairs of partitions are returned as two separate lists,
+ * parts1 and parts2 resp., one for each side. Otherwise, those lists will be
+ * set to NIL.
+ */
+static void
+generate_matching_part_pairs(PartitionMap *partmaps1, PartitionMap *partmaps2,
+							 int nparts1, int nparts2,
+							 JoinType jointype, int nparts,
+							 List **matched_parts1, List **matched_parts2)
+{
+	bool	merged = true;
+	int		*matching1 = (int *) palloc(sizeof(int) * nparts),
+			*matching2 = (int *) palloc(sizeof(int) * nparts);
+	int 	i;
+
+	*matched_parts1 = NIL;
+	*matched_parts2 = NIL;
+
+	/* Set pairs of matching partitions. */
+	for (i = 0; i < nparts; i++)
+	{
+		if (i >= nparts1)
+			matching1[i] = -1;
+		else
+		{
+			PartitionMap outer_map = partmaps1[i];
+
+			if (outer_map.to >= 0)
+			{
+				Assert(outer_map.to < nparts);
+				matching1[outer_map.to] = i;
+			}
+		}
+
+		if (i >= nparts2)
+			matching2[i] = -1;
+		else
+		{
+			PartitionMap inner_map = partmaps2[i];
+
+			if (inner_map.to >= 0)
+			{
+				Assert(inner_map.to < nparts);
+				matching2[inner_map.to] = i;
+			}
+		}
+	}
+
+	/*
+	 * If we have a partition missing on an inner side, we need to add a dummy
+	 * relation which joins with the outer partition. If the inner relation
+	 * happens to be a base relation, it will require adding a dummy child
+	 * base relation during join processing. Right now, we freeze the base
+	 * relation arrays like PlannerInfo::simple_rte_array after planning for
+	 * base relations. Adding a new (dummy) base relation would require some
+	 * changes to that. So, right now, we do not implement partition-wise join
+	 * in such cases.
+	 */
+	for (i = 0; i < nparts; i++)
+	{
+		int			part1 = matching1[i];
+		int			part2 = matching2[i];
+
+		/* At least one of the partitions should exist. */
+		Assert(part1 >= 0 || part2 >= 0);
+
+		switch (jointype)
+		{
+			case JOIN_INNER:
+			case JOIN_SEMI:
+
+				/*
+				 * An inner or semi join can not return any row when the
+				 * matching partition on either side is missing. We should
+				 * have eliminated all such cases while merging the bounds.
+				 */
+				Assert(part1 >= 0 && part2 >= 0);
+				break;
+
+			case JOIN_LEFT:
+			case JOIN_ANTI:
+				Assert(part1 >= 0);
+				if (part2 < 0)
+					merged = false;
+				break;
+
+			case JOIN_FULL:
+				if (part1 < 0 || part2 < 0)
+					merged = false;
+				break;
+
+			default:
+				elog(ERROR, "unrecognized join type: %d", (int) jointype);
+		}
+
+		if (!merged)
+			break;
+
+		*matched_parts1 = lappend_int(*matched_parts1, part1);
+		*matched_parts2 = lappend_int(*matched_parts2, part2);
+	}
+
+	pfree(matching1);
+	pfree(matching2);
+
+	if (!merged)
+	{
+		list_free(*matched_parts1);
+		list_free(*matched_parts2);
+		*matched_parts1 = NIL;
+		*matched_parts2 = NIL;
+	}
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * Merge default partitions from both sides, if any, and assign the default
+ * partition for the join result, if necessary.
+ *
+ * If both the relations have default partitions, try mapping those to each
+ * other. If the mapping succeeds corresponding merged partition will act as
+ * the default partition of the join result.
+ *
+ * If inner side of the join has default but not the outer side, rows in it
+ * won't appear in the join result. So don't create a default partition. If
+ * outer side of the join has default but not the inner side, rows in it will
+ * appear in the join result, so create a default merged partition.
+ */
+static bool
+merge_default_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+						 PartitionMap *outer_maps, PartitionMap *inner_maps,
+						 JoinType jointype, int *next_index, int *default_index)
+{
+	int				outer_default = outer_bi->default_index;
+	int				inner_default = inner_bi->default_index;
+	bool			outer_has_default = partition_bound_has_default(outer_bi);
+	bool			inner_has_default = partition_bound_has_default(inner_bi);
+	bool			merged = true;
+	PartitionMap 	*outer_default_map = NULL;
+	PartitionMap 	*inner_default_map = NULL;
+
+	if (outer_has_default)
+		outer_default_map = &outer_maps[outer_default];
+
+	if (inner_has_default)
+		inner_default_map = &inner_maps[inner_default];
+
+	if (!outer_has_default && !inner_has_default)
+		Assert(*default_index < 0);
+	else if (outer_default_map != NULL && inner_default_map == NULL)
+	{
+		if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+			jointype == JOIN_ANTI)
+		{
+			if (outer_default_map->to < 0)
+			{
+				outer_default_map->to = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = outer_default_map->to;
+			}
+			else
+				Assert(*default_index == outer_default_map->to);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else if (outer_default_map == NULL && inner_default_map != NULL)
+	{
+		if (jointype == JOIN_FULL)
+		{
+			if (inner_default_map->to < 0)
+			{
+				inner_default_map->to = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = inner_default_map->to;
+			}
+			else
+				Assert(*default_index == inner_default_map->to);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		*default_index = map_and_merge_partitions(outer_maps, inner_maps,
+												  outer_default, inner_default,
+												  next_index);
+
+		if (*default_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
+
+/*
+ * merge_null_partitions
+ *
+ * Merge NULL partitions, i.e. a partition that can hold NULL values for a list
+ * partitioned table, if any. Find the index of merged partition to which the
+ * NULL values would belong in the join result. If one joining relation has a
+ * NULL partition but not the other, try matching it with the default partition
+ * from the other relation since the default partition may have rows with NULL
+ * partition key. We can eliminate a NULL partition when it appears only on the
+ * inner side of the join and the outer side doesn't have a default partition.
+ *
+ * When the equality operator used for join is strict, two NULL values will not
+ * be considered as equal, and thus a NULL partition can be eliminated for an
+ * inner join. But we don't check the strictness operator here.
+ */
+static bool
+merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_maps, PartitionMap *inner_maps,
+					  JoinType jointype, int *next_index,
+					  int *null_index, int *default_index)
+{
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	int			outer_ni = outer_bi->null_index;
+	int			inner_ni = inner_bi->null_index;
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	bool		merged = true;
+
+	if (!outer_has_null && !inner_has_null)
+		*null_index = -1;
+	else if (outer_has_null && !inner_has_null)
+	{
+		int			merged_index;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, inner relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the inner relation acts as
+		 * INNER relation. For INNER and SEMI join OUTER and INNER
+		 * differentiation is immaterial.
+		 */
+		missing_side_inner = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_outer = (jointype == JOIN_FULL);
+
+		merged = handle_missing_partition(inner_maps,
+										  outer_maps,
+										  inner_default,
+										  outer_ni,
+										  missing_side_outer,
+										  missing_side_inner, next_index,
+										  default_index, &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join to which the outer null partition maps
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = outer_maps[outer_ni].to;
+	}
+	else if (!outer_has_null && inner_has_null)
+	{
+		int			merged_index;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, outer relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the outer relation acts as OUTER
+		 * relation. For INNER and SEMI join OUTER and INNER differentiation is
+		 * immaterial.
+		 */
+		missing_side_outer = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_inner = (jointype == JOIN_FULL);
+		merged = handle_missing_partition(outer_maps,
+										  inner_maps,
+										  outer_default,
+										  inner_ni,
+										  missing_side_outer,
+										  missing_side_inner,
+										  next_index, default_index,
+										  &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join, to which the outer side null partition maps,
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = inner_maps[inner_ni].to;
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		*null_index = map_and_merge_partitions(outer_maps,
+											   inner_maps,
+											   outer_ni,
+											   inner_ni,
+											   next_index);
+		if (*null_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index 36fb584e23..789d089ddc 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -107,5 +107,10 @@ extern int partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 							  int nvalues, Datum *values, bool *is_equal);
 extern int partition_hash_bsearch(PartitionBoundInfo boundinfo,
 					   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
-- 
2.16.4

0004-Tests-for-0-1-1-1-and-1-0-partition-matching-v15.patchapplication/octet-stream; name=0004-Tests-for-0-1-1-1-and-1-0-partition-matching-v15.patchDownload
From d9fad87d22916380db8b14cc05cdd20314ba17ac Mon Sep 17 00:00:00 2001
From: erthalion <9erthalion6@gmail.com>
Date: Tue, 11 Sep 2018 21:11:55 +0200
Subject: [PATCH 4/5] Tests for 0:1, 1:1 and 1:0 partition matching

Rajkumar Raghuvanshi and Ashutosh Bapat.
---
 src/test/regress/expected/partition_join.out | 4131 +++++++++++++++++++++-----
 src/test/regress/sql/partition_join.sql      |  427 ++-
 2 files changed, 3855 insertions(+), 703 deletions(-)

diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index c55de5d476..f19611c853 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -8,59 +8,86 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-(4 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+(7 rows)
 
 -- left outer join, with whole-row reference; partitionwise join does not apply
 EXPLAIN (COSTS OFF)
@@ -72,35 +99,50 @@ SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER
    ->  Hash Right Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(22 rows)
 
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-      t1      |      t2      
---------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
- (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
- (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
- (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
- (550,0,0550) | 
-(12 rows)
+       t1       |       t2       
+----------------+----------------
+ (-250,0,-0250) | 
+ (-200,0,-0200) | 
+ (-150,0,-0150) | (0,-150,-0150)
+ (-100,0,-0100) | 
+ (-50,0,-0050)  | 
+ (0,0,0000)     | (0,0,0000)
+ (50,0,0050)    | 
+ (100,0,0100)   | 
+ (150,0,0150)   | (0,150,0150)
+ (200,0,0200)   | 
+ (250,0,0250)   | 
+ (300,0,0300)   | (0,300,0300)
+ (350,0,0350)   | 
+ (400,0,0400)   | 
+ (450,0,0450)   | (0,450,0450)
+ (500,0,0500)   | 
+ (550,0,0550)   | 
+ (600,0,0600)   | (0,600,0600)
+ (650,0,0650)   | 
+ (700,0,0700)   | 
+ (750,0,0750)   | (0,750,0750)
+(21 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
@@ -112,35 +154,53 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
    ->  Append
          ->  Hash Right Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Right Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t2_2.b)
-(20 rows)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+      |       | -225 | -0225
+      |       |  -75 | -0075
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(14 rows)
 
 -- full outer join, with placeholder vars
 EXPLAIN (COSTS OFF)
@@ -148,8 +208,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                             QUERY PLAN                            
 ------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               Filter: (((50) = prt1_p0.a) OR ((75) = prt2_p0.b))
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
@@ -174,7 +242,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               Filter: (((50) = prt1_p4.a) OR ((75) = prt2_p4.b))
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
@@ -211,8 +287,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Hash Left Join
+               Hash Cond: (prt1_p0.a = b)
+               ->  Seq Scan on prt1_p0
+                     Filter: ((a < 450) AND (b = 0))
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
          ->  Hash Left Join
                Hash Cond: (prt1_p1.a = b)
                ->  Seq Scan on prt1_p1
@@ -227,29 +310,42 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                ->  Hash
                      ->  Seq Scan on prt1_p2
                            Filter: ((a < 450) AND (b = 0))
-(17 rows)
+(24 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-(9 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+(14 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
                          QUERY PLAN                         
 ------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = b)
+               Filter: ((prt1_p0.b = 0) OR (a = 0))
+               ->  Seq Scan on prt1_p0
+                     Filter: (a < 450)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = b)
                Filter: ((prt1_p1.b = 0) OR (a = 0))
@@ -274,64 +370,153 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                ->  Hash
                      ->  Result
                            One-Time Filter: false
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt2_p4.b = a)
+               Filter: ((b = 0) OR (prt2_p4.a = 0))
+               ->  Seq Scan on prt2_p4
+                     Filter: (b > 250)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
-(12 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+      |       | 375 | 0375
+      |       | 450 | 0450
+      |       | 525 | 0525
+      |       | 600 | 0600
+      |       | 675 | 0675
+      |       | 750 | 0750
+(20 rows)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Semi Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Semi Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                            Filter: (a = 0)
-         ->  Nested Loop Semi Join
-               Join Filter: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
-               ->  Materialize
-                     ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
-(24 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt1_p3 t2_3
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+                           Filter: (b = 0)
+(37 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
 
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
@@ -342,27 +527,82 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
    ->  Append
          ->  Hash Anti Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Hash Anti Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Hash Anti Join
                Hash Cond: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
-(17 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(27 rows)
 
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
-  sum  |         avg          | sum  |         avg         
--------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+  sum  |         avg          | sum  |        avg         
+-------+----------------------+------+--------------------
+ 95550 | 273.0000000000000000 | 2200 | 6.2857142857142857
 (1 row)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Append
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p0 t1
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+               Index Cond: (a = t1.b)
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.b = t2_1.a)
+         ->  Seq Scan on prt2_p1 t1_1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p1 t2_1
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p2 t1_2
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+               Index Cond: (a = t1_2.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p3 t1_3
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+               Index Cond: (a = t1_3.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p4 t1_4
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+               Index Cond: (a = t1_4.b)
+(27 rows)
+
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+  b   |   c   
+------+-------
+ -225 | -0225
+  -75 | -0075
+   75 | 0075
+  225 | 0225
+  375 | 0375
+  525 | 0525
+  675 | 0675
+(7 rows)
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -374,49 +614,74 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
+                     ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
                            Index Cond: (a = t1.a)
-                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t3
                            Index Cond: (b = t2.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_1
+                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
                            Index Cond: (a = t1_1.a)
-                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3_1
                            Index Cond: (b = t2_1.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_2
+                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
                            Index Cond: (a = t1_2.a)
-                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_2
                            Index Cond: (b = t2_2.a)
-(27 rows)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                           Index Cond: (a = t1_3.a)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_3
+                           Index Cond: (b = t2_3.a)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                           Index Cond: (a = t1_4.a)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t3_4
+                           Index Cond: (b = t2_4.a)
+(43 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   | t2a | t3a | least 
------+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
- 100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
- 250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
- 400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
- 550 | 0 | 0550 |     |     |      
-(12 rows)
+  a   | b |   c   | t2a  | t3a | least 
+------+---+-------+------+-----+-------
+ -250 | 0 | -0250 |      |     |      
+ -200 | 0 | -0200 |      |     |      
+ -150 | 0 | -0150 | -150 |   0 |  -150
+ -100 | 0 | -0100 |      |     |      
+  -50 | 0 | -0050 |      |     |      
+    0 | 0 | 0000  |    0 |   0 |     0
+   50 | 0 | 0050  |      |     |      
+  100 | 0 | 0100  |      |     |      
+  150 | 0 | 0150  |  150 |   0 |   150
+  200 | 0 | 0200  |      |     |      
+  250 | 0 | 0250  |      |     |      
+  300 | 0 | 0300  |  300 |   0 |   300
+  350 | 0 | 0350  |      |     |      
+  400 | 0 | 0400  |      |     |      
+  450 | 0 | 0450  |  450 |   0 |   450
+  500 | 0 | 0500  |      |     |      
+  550 | 0 | 0550  |      |     |      
+  600 | 0 | 0600  |  600 |   0 |   600
+  650 | 0 | 0650  |      |     |      
+  700 | 0 | 0700  |      |     |      
+  750 | 0 | 0750  |  750 |   0 |   750
+(21 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
@@ -430,64 +695,95 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
          Hash Cond: ((t1.c)::text = (t2.c)::text)
          Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Seq Scan on prt1_p1 t1
-               ->  Seq Scan on prt1_p2 t1_1
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
          ->  Hash
                ->  Append
                      ->  Hash Join
                            Hash Cond: (t2.a = t3.b)
-                           ->  Seq Scan on prt1_p1 t2
+                           ->  Seq Scan on prt1_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t3
+                                 ->  Seq Scan on prt2_p0 t3
                      ->  Hash Join
                            Hash Cond: (t2_1.a = t3_1.b)
-                           ->  Seq Scan on prt1_p2 t2_1
+                           ->  Seq Scan on prt1_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t3_1
+                                 ->  Seq Scan on prt2_p1 t3_1
                      ->  Hash Join
                            Hash Cond: (t2_2.a = t3_2.b)
-                           ->  Seq Scan on prt1_p3 t2_2
+                           ->  Seq Scan on prt1_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p3 t3_2
-(26 rows)
+                                 ->  Seq Scan on prt2_p2 t3_2
+                     ->  Hash Join
+                           Hash Cond: (t2_3.a = t3_3.b)
+                           ->  Seq Scan on prt1_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t3_3
+                     ->  Hash Join
+                           Hash Cond: (t2_4.a = t3_4.b)
+                           ->  Seq Scan on prt1_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t3_4
+(38 rows)
 
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, t2.b t2b, t2.c t2c, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
-  a  | t2a | t2c  
------+-----+------
-   0 |   0 | 0000
-  50 |     | 
- 100 |     | 
- 150 | 150 | 0150
- 200 |     | 
- 250 |     | 
- 300 | 300 | 0300
- 350 |     | 
- 400 |     | 
- 450 | 450 | 0450
- 500 |     | 
- 550 |     | 
-(12 rows)
+  a   | t2a  |  t2c  
+------+------+-------
+ -250 |      | 
+ -200 |      | 
+ -150 | -150 | -0150
+ -100 |      | 
+  -50 |      | 
+    0 |    0 | 0000
+   50 |      | 
+  100 |      | 
+  150 |  150 | 0150
+  200 |      | 
+  250 |      | 
+  300 |  300 | 0300
+  350 |      | 
+  400 |      | 
+  450 |  450 | 0450
+  500 |      | 
+  550 |      | 
+  600 |  600 | 0600
+  650 |      | 
+  700 |      | 
+  750 |  750 | 0750
+(21 rows)
 
 --
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
@@ -498,32 +794,49 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
    ->  Append
          ->  Hash Join
                Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
-               ->  Seq Scan on prt2_e_p1 t2
+               ->  Seq Scan on prt2_e_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 t1
+                     ->  Seq Scan on prt1_e_p0 t1
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
-               ->  Seq Scan on prt2_e_p2 t2_1
+               Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+               ->  Seq Scan on prt1_e_p1 t1_1
+                     Filter: (c = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 t1_1
-                           Filter: (c = 0)
+                     ->  Seq Scan on prt2_e_p1 t2_1
          ->  Hash Join
                Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
-               ->  Seq Scan on prt2_e_p3 t2_2
+               ->  Seq Scan on prt2_e_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 t1_2
+                     ->  Seq Scan on prt1_e_p2 t1_2
                            Filter: (c = 0)
-(21 rows)
+         ->  Hash Join
+               Hash Cond: (((t2_3.b + t2_3.a) / 2) = ((t1_3.a + t1_3.b) / 2))
+               ->  Seq Scan on prt2_e_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p3 t1_3
+                           Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (((t2_4.b + t2_4.a) / 2) = ((t1_4.a + t1_4.b) / 2))
+               ->  Seq Scan on prt2_e_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4 t1_4
+                           Filter: (c = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
- 150 | 0 | 150 | 0
- 300 | 0 | 300 | 0
- 450 | 0 | 450 | 0
-(4 rows)
+  a   | c |  b   | c 
+------+---+------+---
+ -250 | 0 | -250 | 0
+ -100 | 0 | -100 | 0
+    0 | 0 |    0 | 0
+    0 | 0 |    0 | 0
+  150 | 0 |  150 | 0
+  300 | 0 |  300 | 0
+  450 | 0 |  450 | 0
+  600 | 0 |  600 | 0
+  750 | 0 |  750 | 0
+(9 rows)
 
 --
 -- N-way join
@@ -536,154 +849,232 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = ((t3.a + t3.b) / 2))
+               Join Filter: (t1.a = t2.b)
                ->  Hash Join
-                     Hash Cond: (t2.b = t1.a)
-                     ->  Seq Scan on prt2_p1 t2
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on prt1_e_p0 t3
                      ->  Hash
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                     Index Cond: (((a + b) / 2) = t2.b)
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
+                     Index Cond: (b = ((t3.a + t3.b) / 2))
          ->  Nested Loop
                Join Filter: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_1.b = t1_1.a)
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
+               ->  Index Scan using iprt1_e_p4_ab2 on prt1_e_p1 t3_1
                      Index Cond: (((a + b) / 2) = t2_1.b)
          ->  Nested Loop
                Join Filter: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_2.b = t1_2.a)
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
+               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_2
                      Index Cond: (((a + b) / 2) = t2_2.b)
-(33 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = ((t3_3.a + t3_3.b) / 2))
+               ->  Nested Loop
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t2_3.b)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t2_4.b)
+               ->  Hash Join
+                     Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                     ->  Seq Scan on prt1_e_p4 t3_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_p4 t1_4
+                                 Filter: (b = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = ((t3_4.a + t3_4.b) / 2))
+(52 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
-(4 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(8 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                          QUERY PLAN                          
---------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Hash Right Join
                Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
-               ->  Seq Scan on prt1_e_p1 t3
+               ->  Seq Scan on prt1_e_p0 t3
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2.b = t1.a)
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_1.a)
-               ->  Seq Scan on prt1_e_p2 t3_1
+               ->  Seq Scan on prt1_e_p1 t3_1
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_1.b = t1_1.a)
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_2.a)
-               ->  Seq Scan on prt1_e_p3 t3_2
+               ->  Seq Scan on prt1_e_p2 t3_2
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_2.b = t1_2.a)
-                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                                        Filter: (b = 0)
-(33 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (t1_3.a = b)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (t1_3.a = ((a + b) / 2))
+         ->  Hash Right Join
+               Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+               ->  Seq Scan on prt1_e_p4 t3_4
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_4.b = t1_4.a)
+                           ->  Seq Scan on prt2_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_p4 t1_4
+                                       Filter: (b = 0)
+(51 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                            QUERY PLAN                             
--------------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p1 t3
+                           ->  Seq Scan on prt1_e_p0 t3
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p1_b on prt2_p1 t2
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
                      Index Cond: (t1.a = b)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p2 t3_1
+                           ->  Seq Scan on prt1_e_p1 t3_1
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
+               ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
                      Index Cond: (t1_1.a = b)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p3 t3_2
+                           ->  Seq Scan on prt1_e_p2 t3_2
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
                      Index Cond: (t1_2.a = b)
-(30 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_e_p3 t3_3
+                           Filter: (c = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                           Index Cond: (a = ((t3_3.a + t3_3.b) / 2))
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (t1_3.a = b)
+         ->  Nested Loop Left Join
+               ->  Hash Right Join
+                     Hash Cond: (t1_4.a = ((t3_4.a + t3_4.b) / 2))
+                     ->  Seq Scan on prt1_p4 t1_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p4 t3_4
+                                 Filter: (c = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (t1_4.a = b)
+(47 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -692,21 +1083,34 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                                                    QUERY PLAN                                                   
 ----------------------------------------------------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   Sort Key: prt1_p0.a, prt2_p0.b, ((prt1_e_p0.a + prt1_e_p0.b))
    ->  Append
          ->  Hash Full Join
-               Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
-               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               Hash Cond: (prt1_p0.a = ((prt1_e_p0.a + prt1_e_p0.b) / 2))
+               Filter: ((prt1_p0.a = (50)) OR (prt2_p0.b = (75)) OR (((prt1_e_p0.a + prt1_e_p0.b) / 2) = (50)))
                ->  Hash Full Join
-                     Hash Cond: (prt1_p1.a = prt2_p1.b)
-                     ->  Seq Scan on prt1_p1
+                     Hash Cond: (prt1_p0.a = prt2_p0.b)
+                     ->  Seq Scan on prt1_p0
                            Filter: (b = 0)
                      ->  Hash
-                           ->  Seq Scan on prt2_p1
+                           ->  Seq Scan on prt2_p0
                                  Filter: (a = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1
+                     ->  Seq Scan on prt1_e_p0
                            Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = prt1_p1.a)
+               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               ->  Seq Scan on prt1_e_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on prt1_p1
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1
+                                       Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
                Filter: ((prt1_p2.a = (50)) OR (prt2_p2.b = (75)) OR (((prt1_e_p2.a + prt1_e_p2.b) / 2) = (50)))
@@ -733,7 +1137,20 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                ->  Hash
                      ->  Seq Scan on prt1_e_p3
                            Filter: (c = 0)
-(42 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = ((prt1_e_p4.a + prt1_e_p4.b) / 2))
+               Filter: ((prt1_p4.a = (50)) OR (prt2_p4.b = (75)) OR (((prt1_e_p4.a + prt1_e_p4.b) / 2) = (50)))
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = prt2_p4.b)
+                     ->  Seq Scan on prt1_p4
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4
+                                 Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4
+                           Filter: (c = 0)
+(68 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
@@ -751,172 +1168,260 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = t1_3.b)
+               Join Filter: (t1.a = t1_5.b)
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Join
-                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
-                           ->  Seq Scan on prt1_e_p1 t2
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t1_3
+                                 ->  Seq Scan on prt2_p0 t1_5
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
                      Index Cond: (a = ((t2.a + t2.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_1.a = t1_4.b)
+               Join Filter: (t1_1.a = t1_6.b)
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Join
-                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
-                           ->  Seq Scan on prt1_e_p2 t2_1
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_6.b)
+                           ->  Seq Scan on prt1_e_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t1_4
+                                 ->  Seq Scan on prt2_p1 t1_6
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
                      Index Cond: (a = ((t2_1.a + t2_1.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_2.a = t1_5.b)
+               Join Filter: (t1_2.a = t1_7.b)
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
+                           ->  Seq Scan on prt2_p2 t1_7
                                  Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
+                           ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t2_2
+                                 Index Cond: (((a + b) / 2) = t1_7.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t1_8.b)
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Nested Loop
+                           ->  Seq Scan on prt2_p3 t1_8
+                                 Filter: (a = 0)
+                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_3
+                                 Index Cond: (((a + b) / 2) = t1_8.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = ((t2_3.a + t2_3.b) / 2))
+                     Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t1_9.b)
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Join
+                           Hash Cond: (((t2_4.a + t2_4.b) / 2) = t1_9.b)
+                           ->  Seq Scan on prt1_e_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t1_9
+                                       Filter: (a = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = ((t2_4.a + t2_4.b) / 2))
+                     Filter: (b = 0)
+(66 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                               QUERY PLAN                                
--------------------------------------------------------------------------
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
-                           ->  Seq Scan on prt2_p1 t1_3
+                           Hash Cond: (t1_5.b = ((t1_10.a + t1_10.b) / 2))
+                           ->  Seq Scan on prt2_p0 t1_5
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t1_6
+                                 ->  Seq Scan on prt1_e_p0 t1_10
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
-                     Index Cond: (a = t1_3.b)
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t1_5.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
-                           ->  Seq Scan on prt2_p2 t1_4
+                           Hash Cond: (t1_6.b = ((t1_11.a + t1_11.b) / 2))
+                           ->  Seq Scan on prt2_p1 t1_6
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t1_7
+                                 ->  Seq Scan on prt1_e_p1 t1_11
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
-                     Index Cond: (a = t1_4.b)
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
+                     Index Cond: (a = t1_6.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
-                           ->  Seq Scan on prt2_p3 t1_5
+                           Hash Cond: (t1_7.b = ((t1_12.a + t1_12.b) / 2))
+                           ->  Seq Scan on prt2_p2 t1_7
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p3 t1_8
+                                 ->  Seq Scan on prt1_e_p2 t1_12
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t1_5.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t1_7.b)
                      Filter: (b = 0)
-(39 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_8.b = ((t1_13.a + t1_13.b) / 2))
+                           ->  Seq Scan on prt2_p3 t1_8
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p3 t1_13
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t1_8.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_9.b = ((t1_14.a + t1_14.b) / 2))
+                           ->  Seq Scan on prt2_p4 t1_9
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t1_14
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = t1_9.b)
+                     Filter: (b = 0)
+(63 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 -- test merge joins
 SET enable_hashjoin TO off;
 SET enable_nestloop TO off;
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                           QUERY PLAN                           
-----------------------------------------------------------------
+                            QUERY PLAN                            
+------------------------------------------------------------------
  Merge Append
    Sort Key: t1.a
    ->  Merge Semi Join
-         Merge Cond: (t1.a = t1_3.b)
+         Merge Cond: (t1.a = t1_5.b)
          ->  Sort
                Sort Key: t1.a
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+               Merge Cond: (t1_5.b = (((t1_10.a + t1_10.b) / 2)))
                ->  Sort
-                     Sort Key: t1_3.b
-                     ->  Seq Scan on prt2_p1 t1_3
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on prt2_p0 t1_5
                ->  Sort
-                     Sort Key: (((t1_6.a + t1_6.b) / 2))
-                     ->  Seq Scan on prt1_e_p1 t1_6
+                     Sort Key: (((t1_10.a + t1_10.b) / 2))
+                     ->  Seq Scan on prt1_e_p0 t1_10
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_1.a = t1_4.b)
+         Merge Cond: (t1_1.a = t1_6.b)
          ->  Sort
                Sort Key: t1_1.a
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+               Merge Cond: (t1_6.b = (((t1_11.a + t1_11.b) / 2)))
                ->  Sort
-                     Sort Key: t1_4.b
-                     ->  Seq Scan on prt2_p2 t1_4
+                     Sort Key: t1_6.b
+                     ->  Seq Scan on prt2_p1 t1_6
                ->  Sort
-                     Sort Key: (((t1_7.a + t1_7.b) / 2))
-                     ->  Seq Scan on prt1_e_p2 t1_7
+                     Sort Key: (((t1_11.a + t1_11.b) / 2))
+                     ->  Seq Scan on prt1_e_p1 t1_11
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_2.a = t1_5.b)
+         Merge Cond: (t1_2.a = t1_7.b)
          ->  Sort
                Sort Key: t1_2.a
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+               Merge Cond: (t1_7.b = (((t1_12.a + t1_12.b) / 2)))
                ->  Sort
-                     Sort Key: t1_5.b
-                     ->  Seq Scan on prt2_p3 t1_5
+                     Sort Key: t1_7.b
+                     ->  Seq Scan on prt2_p2 t1_7
                ->  Sort
-                     Sort Key: (((t1_8.a + t1_8.b) / 2))
-                     ->  Seq Scan on prt1_e_p3 t1_8
+                     Sort Key: (((t1_12.a + t1_12.b) / 2))
+                     ->  Seq Scan on prt1_e_p2 t1_12
                            Filter: (c = 0)
-(47 rows)
+   ->  Merge Semi Join
+         Merge Cond: (t1_3.a = t1_8.b)
+         ->  Sort
+               Sort Key: t1_3.a
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_8.b = (((t1_13.a + t1_13.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_8.b
+                     ->  Seq Scan on prt2_p3 t1_8
+               ->  Sort
+                     Sort Key: (((t1_13.a + t1_13.b) / 2))
+                     ->  Seq Scan on prt1_e_p3 t1_13
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_4.a = t1_9.b)
+         ->  Sort
+               Sort Key: t1_4.a
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_9.b = (((t1_14.a + t1_14.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_9.b
+                     ->  Seq Scan on prt2_p4 t1_9
+               ->  Sort
+                     Sort Key: (((t1_14.a + t1_14.b) / 2))
+                     ->  Seq Scan on prt1_e_p4 t1_14
+                           Filter: (c = 0)
+(77 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
@@ -933,14 +1438,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
                            ->  Sort
                                  Sort Key: (((t3.a + t3.b) / 2))
-                                 ->  Seq Scan on prt1_e_p1 t3
+                                 ->  Seq Scan on prt1_e_p0 t3
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1.a
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                ->  Sort
                      Sort Key: t2.b
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Merge Left Join
                Merge Cond: (t1_1.a = t2_1.b)
                ->  Sort
@@ -949,14 +1454,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
                            ->  Sort
                                  Sort Key: (((t3_1.a + t3_1.b) / 2))
-                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                 ->  Seq Scan on prt1_e_p1 t3_1
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_1.a
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                ->  Sort
                      Sort Key: t2_1.b
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Merge Left Join
                Merge Cond: (t1_2.a = t2_2.b)
                ->  Sort
@@ -965,32 +1470,74 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
                            ->  Sort
                                  Sort Key: (((t3_2.a + t3_2.b) / 2))
-                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                 ->  Seq Scan on prt1_e_p2 t3_2
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_2.a
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                ->  Sort
                      Sort Key: t2_2.b
-                     ->  Seq Scan on prt2_p3 t2_2
-(51 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Merge Left Join
+               Merge Cond: (t1_3.a = t2_3.b)
+               ->  Sort
+                     Sort Key: t1_3.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_3.a + t3_3.b) / 2)) = t1_3.a)
+                           ->  Sort
+                                 Sort Key: (((t3_3.a + t3_3.b) / 2))
+                                 ->  Seq Scan on prt1_e_p3 t3_3
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_3.a
+                                 ->  Seq Scan on prt1_p3 t1_3
+               ->  Sort
+                     Sort Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Merge Left Join
+               Merge Cond: (t1_4.a = t2_4.b)
+               ->  Sort
+                     Sort Key: t1_4.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_4.a + t3_4.b) / 2)) = t1_4.a)
+                           ->  Sort
+                                 Sort Key: (((t3_4.a + t3_4.b) / 2))
+                                 ->  Seq Scan on prt1_e_p4 t3_4
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_4.a
+                                 ->  Seq Scan on prt1_p4 t1_4
+               ->  Sort
+                     Sort Key: t2_4.b
+                     ->  Seq Scan on prt2_p4 t2_4
+(83 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- MergeAppend on nullable column
 EXPLAIN (COSTS OFF)
@@ -998,8 +1545,18 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Merge Left Join
+               Merge Cond: (prt1_p0.a = b)
+               ->  Sort
+                     Sort Key: prt1_p0.a
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
+               ->  Sort
+                     Sort Key: b
+                     ->  Result
+                           One-Time Filter: false
          ->  Merge Left Join
                Merge Cond: (prt1_p1.a = b)
                ->  Sort
@@ -1020,21 +1577,26 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                      Sort Key: prt2_p2.b
                      ->  Seq Scan on prt2_p2
                            Filter: (b > 250)
-(23 rows)
+(33 rows)
 
 SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  b  
------+-----
-   0 |    
-  50 |    
- 100 |    
- 150 |    
- 200 |    
- 250 |    
- 300 | 300
- 350 |    
- 400 |    
-(9 rows)
+  a   |  b  
+------+-----
+ -250 |    
+ -200 |    
+ -150 |    
+ -100 |    
+  -50 |    
+    0 |    
+   50 |    
+  100 |    
+  150 |    
+  200 |    
+  250 |    
+  300 | 300
+  350 |    
+  400 |    
+(14 rows)
 
 -- merge join when expression with whole-row reference needs to be sorted;
 -- partitionwise join does not apply
@@ -1048,175 +1610,2412 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
          Sort Key: t1.a, ((((t1.*)::prt1))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
    ->  Sort
          Sort Key: t2.b, ((((t2.*)::prt2))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt2_p1 t2
-                     ->  Seq Scan on prt2_p2 t2_1
-                     ->  Seq Scan on prt2_p3 t2_2
-(16 rows)
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+(20 rows)
 
 SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.b ORDER BY t1.a;
- a  | b  
-----+----
-  0 |  0
-  6 |  6
- 12 | 12
- 18 | 18
- 24 | 24
-(5 rows)
+  a  |  b  
+-----+-----
+ -24 | -24
+ -18 | -18
+ -12 | -12
+  -6 |  -6
+   0 |   0
+   6 |   6
+  12 |  12
+  18 |  18
+  24 |  24
+(9 rows)
 
 RESET enable_hashjoin;
 RESET enable_nestloop;
---
--- partitioned by multiple columns
---
-CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
-CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
-ANALYZE prt1_m;
-CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
-CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
-ANALYZE prt2_m;
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-                                                             QUERY PLAN                                                             
-------------------------------------------------------------------------------------------------------------------------------------
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p4 t2_3
+               ->  Seq Scan on prt2_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_4
+                           Filter: (b = 0)
+(22 rows)
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p3 t1_2
+                           Filter: (b = 0)
+(21 rows)
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -150 | -0150 | 0 | -0150
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+(7 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (t1_3.a = b)
+         ->  Hash Right Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 |   | 
+ -200 | -0200 |   | 
+ -150 | -0150 | 0 | -0150
+ -100 | -0100 |   | 
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+(21 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -200 | 0 | -0200
+ -100 | 0 | -0100
+  -50 | 0 | -0050
+   50 | 0 | 0050
+  100 | 0 | 0100
+  200 | 0 | 0200
+  250 | 0 | 0250
+  350 | 0 | 0350
+  400 | 0 | 0400
+  500 | 0 | 0500
+  550 | 0 | 0550
+  650 | 0 | 0650
+  700 | 0 | 0700
+(14 rows)
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+                             QUERY PLAN                              
+---------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t3.c
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p0_a on prt1_p0 t3
+                           Index Cond: (a = t2.b)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t2.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_1.a = t2_1.b)
+                           ->  Seq Scan on prt1_p1 t3_1
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1 t2_1
+                                       Filter: (a = 0)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p2_a on prt1_p2 t3_2
+                           Index Cond: (a = t2_2.b)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t3_3
+                           Index Cond: (a = t2_3.b)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_4.a = t2_4.b)
+                           ->  Seq Scan on prt1_p4 t3_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t2_4
+                                       Filter: (a = 0)
+(47 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+  a   | a |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.b
+   ->  Hash Right Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p5 t2_5
+                           Filter: (a = 0)
+(24 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: (t1.a = t2.b)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.a, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
+                     Filter: (a = 0)
+(22 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
+               ->  Seq Scan on prt1_m_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p1
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
+               ->  Seq Scan on prt1_m_p2
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p2
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
+               ->  Seq Scan on prt1_m_p3
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p3
+                           Filter: (c = 0)
+(24 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+  a  | c |  b  | c 
+-----+---+-----+---
+   0 | 0 |   0 | 0
+  50 | 0 |     |  
+ 100 | 0 |     |  
+ 150 | 0 | 150 | 0
+ 200 | 0 |     |  
+ 250 | 0 |     |  
+ 300 | 0 | 300 | 0
+ 350 | 0 |     |  
+ 400 | 0 |     |  
+ 450 | 0 | 450 | 0
+ 500 | 0 |     |  
+ 550 | 0 |     |  
+     |   |  75 | 0
+     |   | 225 | 0
+     |   | 375 | 0
+     |   | 525 | 0
+(16 rows)
+
+--
+-- tests for list partitioned tables.
+--
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
+--
+-- list partitioned by expression
+--
+CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
+ANALYZE plt1_e;
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+                                           QUERY PLAN                                           
+------------------------------------------------------------------------------------------------
+ GroupAggregate
+   Group Key: t1.c, t2.c, t3.c
+   ->  Sort
+         Sort Key: t1.c, t3.c
+         ->  Append
+               ->  Hash Join
+                     Hash Cond: ((t1.c)::text = ltrim(t3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2.b = t1.b) AND ((t2.c)::text = (t1.c)::text))
+                           ->  Seq Scan on plt2_p4 t2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p4 t1
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p4 t3
+               ->  Hash Join
+                     Hash Cond: ((t1_1.c)::text = ltrim(t3_1.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t1_1.b = t2_1.b) AND ((t1_1.c)::text = (t2_1.c)::text))
+                           ->  Seq Scan on plt1_p1 t1_1
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p1 t2_1
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p1 t3_1
+               ->  Hash Join
+                     Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t1_2.b = t2_2.b) AND ((t1_2.c)::text = (t2_2.c)::text))
+                           ->  Seq Scan on plt1_p2 t1_2
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p2 t2_2
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p2 t3_2
+               ->  Hash Join
+                     Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2_3.b = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text))
+                           ->  Seq Scan on plt2_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p3 t3_3
+(41 rows)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p5 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_3
+                                 ->  Seq Scan on plt2_p3 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
  Sort
-   Sort Key: prt1_m_p1.a, prt2_m_p1.b
-   ->  Append
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
-               ->  Seq Scan on prt1_m_p1
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p1
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
-               ->  Seq Scan on prt1_m_p2
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p2
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
-               ->  Seq Scan on prt1_m_p3
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p3
-                           Filter: (c = 0)
-(24 rows)
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
 
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
-  50 | 0 |     |  
- 100 | 0 |     |  
- 150 | 0 | 150 | 0
- 200 | 0 |     |  
- 250 | 0 |     |  
- 300 | 0 | 300 | 0
- 350 | 0 |     |  
- 400 | 0 |     |  
- 450 | 0 | 450 | 0
- 500 | 0 |     |  
- 550 | 0 |     |  
-     |   |  75 | 0
-     |   | 225 | 0
-     |   | 375 | 0
-     |   | 525 | 0
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
 (16 rows)
 
---
--- tests for list partitioned tables.
---
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
---
--- list partitioned by expression
---
-CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1_e;
--- test partition matching with N-way join
+-- semi join
 EXPLAIN (COSTS OFF)
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                   QUERY PLAN                                   
---------------------------------------------------------------------------------
- GroupAggregate
-   Group Key: t1.c, t2.c, t3.c
-   ->  Sort
-         Sort Key: t1.c, t3.c
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
          ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.c = ltrim(t3.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1.b = t2.b) AND (t1.c = t2.c))
-                           ->  Seq Scan on plt1_p1 t1
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.c = ltrim(t3_1.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
-                           ->  Seq Scan on plt1_p2 t1_1
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.c = ltrim(t3_2.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
-                           ->  Seq Scan on plt1_p3 t1_2
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p3 t3_2
-(32 rows)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Seq Scan on plt2_p2 t2_2
+                                 ->  Seq Scan on plt2_p3 t2_3
+(22 rows)
 
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  75.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 123.0000000000000000 | 123.0000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 225.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 273.0000000000000000 | 273.0000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 375.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 423.0000000000000000 | 423.0000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 525.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 573.0000000000000000 | 573.0000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1241,22 +4040,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
 --------------------------------------------------
  Hash Left Join
    Hash Cond: (t2.b = a)
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t3.a = t2.b)
-               ->  Seq Scan on prt1_p1 t3
-               ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
-         ->  Hash Join
-               Hash Cond: (t3_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t3_1
-               ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
-         ->  Hash Join
-               Hash Cond: (t3_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t3_2
-               ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
+   ->  Hash Join
+         Hash Cond: (t3.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t3
+               ->  Seq Scan on prt1_p1 t3_1
+               ->  Seq Scan on prt1_p2 t3_2
+               ->  Seq Scan on prt1_p3 t3_3
+               ->  Seq Scan on prt1_p4 t3_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
    ->  Hash
          ->  Result
                One-Time Filter: false
@@ -1271,16 +4070,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 --
 -- tests for hash partitioned tables.
@@ -1356,41 +4161,9 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  273.0000000000000000 | 273.0000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
 (6 rows)
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
- Sort
-   Sort Key: t1.a
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
-               ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
-               ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
-                           Filter: (b = 0)
-(21 rows)
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
@@ -1405,26 +4178,24 @@ SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c
    Sort Key: t1.c
    ->  HashAggregate
          Group Key: t1.c, t2.c
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t2.c = t1.c)
-                     ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p1 t1
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on plt1_p4 t1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_1.c = t1_1.c)
-                     ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_p2 t1_1
+                           ->  Seq Scan on plt1_p1 t1_1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_2.c = t1_2.c)
-                     ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Seq Scan on plt1_p2 t1_2
                                  Filter: ((a % 25) = 0)
-(23 rows)
+                           ->  Seq Scan on plt1_p3 t1_3
+                                 Filter: ((a % 25) = 0)
+(21 rows)
 
 --
 -- multiple levels of partitioning
@@ -1826,64 +4597,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
  Hash Join
    Hash Cond: (t1.a = t2.a)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt4_n_p1 t2
                ->  Seq Scan on prt4_n_p2 t2_1
                ->  Seq Scan on prt4_n_p3 t2_2
-(11 rows)
+(13 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a and t1.a = t3.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                        QUERY PLAN                        
+----------------------------------------------------------
  Hash Join
-   Hash Cond: (t2.a = t1.a)
+   Hash Cond: (t3.b = t1.a)
    ->  Append
-         ->  Seq Scan on prt4_n_p1 t2
-         ->  Seq Scan on prt4_n_p2 t2_1
-         ->  Seq Scan on prt4_n_p3 t2_2
+         ->  Seq Scan on prt2_p0 t3
+         ->  Seq Scan on prt2_p1 t3_1
+         ->  Seq Scan on prt2_p2 t3_2
+         ->  Seq Scan on prt2_p3 t3_3
+         ->  Seq Scan on prt2_p4 t3_4
+         ->  Seq Scan on prt2_p5 t3_5
    ->  Hash
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.a = t3_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_2.b)
-                     ->  Seq Scan on prt1_p3 t1_2
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3 t3_2
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt4_n_p1 t2
+                           ->  Seq Scan on prt4_n_p2 t2_1
+                           ->  Seq Scan on prt4_n_p3 t2_2
 (23 rows)
 
 -- partitionwise join can not be applied if there are no equi-join conditions
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (t1.a < b)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (t1.a < b)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (t1.a < b)
-(12 rows)
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+(16 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partitionwise join
@@ -1969,16 +4746,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
-                     ->  Seq Scan on plt1_p1 t3
-                     ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3
+                     ->  Seq Scan on plt1_p1 t3_1
+                     ->  Seq Scan on plt1_p2 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partitionwise join can not be applied for a join between list and range
 -- partitioned table
@@ -1989,14 +4767,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
  Hash Full Join
    Hash Cond: ((t2.c)::text = (t1.c)::text)
    ->  Append
-         ->  Seq Scan on prt1_p1 t2
-         ->  Seq Scan on prt1_p2 t2_1
-         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p0 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p2 t2_2
+         ->  Seq Scan on prt1_p3 t2_3
+         ->  Seq Scan on prt1_p4 t2_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt1_n_p1 t1
                ->  Seq Scan on prt1_n_p2 t1_1
-(10 rows)
+(12 rows)
 
 -- partitionwise join can not be applied if only one of joining table has
 -- default partition
@@ -2012,16 +4792,23 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
    ->  Hash Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index c1c9859651..0e884835fb 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -10,25 +10,39 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -67,11 +81,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -93,20 +115,30 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 
 EXPLAIN (COSTS OFF)
@@ -169,6 +201,128 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -193,28 +347,79 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
 -- test partition matching with N-way join
@@ -222,6 +427,175 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
@@ -267,27 +641,18 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
 ALTER TABLE plt2 DETACH PARTITION plt2_p3;
 ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
 ANALYZE plt2;
-
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.a % 25 = 0 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
 --
 -- multiple levels of partitioning
 --
-- 
2.16.4

#58amul sul
sulamul@gmail.com
In reply to: Dmitry Dolgov (#57)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Mon, Nov 26, 2018 at 9:33 PM Dmitry Dolgov <9erthalion6@gmail.com> wrote:

On Mon, Nov 26, 2018 at 1:41 PM Dmitry Dolgov <9erthalion6@gmail.com>

wrote:

Thanks, I've messed this up too - rebased a wrong branch, the correct one
doesn't have this code already. Sorry for that, and here is the proper

version

of this patch.

And one more version, because I haven't fixed the tests (sorry for the
noise).

0003 patch need a rebase.

Regards,
Amul

#59Etsuro Fujita
fujita.etsuro@lab.ntt.co.jp
In reply to: amul sul (#58)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

(2019/01/21 20:56), amul sul wrote:

On Mon, Nov 26, 2018 at 9:33 PM Dmitry Dolgov <9erthalion6@gmail.com
<mailto:9erthalion6@gmail.com>> wrote:

On Mon, Nov 26, 2018 at 1:41 PM Dmitry Dolgov

<9erthalion6@gmail.com <mailto:9erthalion6@gmail.com>> wrote:

Thanks, I've messed this up too - rebased a wrong branch, the

correct one

doesn't have this code already. Sorry for that, and here is the

proper version

of this patch.

And one more version, because I haven't fixed the tests (sorry for
the noise).

0003 patch need a rebase.

Will do.

Thank you for letting us know!

Best regards,
Etsuro Fujita

#60Etsuro Fujita
fujita.etsuro@lab.ntt.co.jp
In reply to: Etsuro Fujita (#59)
3 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

(2019/01/21 21:09), Etsuro Fujita wrote:

(2019/01/21 20:56), amul sul wrote:

0003 patch need a rebase.

Will do.

While doing the rebase, I noticed that 0002 would not be necessary
anymore; what was originally proposed by it on how to generate the
tlists for child-joins in [1]/messages/by-id/CAFjFpResoxfp1rnV4Op9JOnG19VNEnjvjRN5DVd8QRHD+agTDw@mail.gmail.com is entirely included in commit
7cfdc77023ad50731723e85c215a4127436ed09c, which I pushed. I didn't pay
attention to this thread, but that was pretty much the same as what I
did in that commit (except the handling of PHVs to avoid extra work in
that commit), so I think I should have mentioned about that in the
commit message. Sorry for that, Ashutosh.

Here is an updated version of the patchset. As above, I don't think
0002 is necessary; it added copy_pathtarget to build_child_join_rel, but
I think it would be redundant because we do create_empty_pathtarget at
the top of that function and then build_child_join_reltarget. Also, it
added an assertion to build_joinrel_tlist, but I don't think it's really
necessary IMO. So I removed it entirely from the patchset. No changes
other that.

Will continue to review.

Best regards,
Etsuro Fujita

[1]: /messages/by-id/CAFjFpResoxfp1rnV4Op9JOnG19VNEnjvjRN5DVd8QRHD+agTDw@mail.gmail.com
/messages/by-id/CAFjFpResoxfp1rnV4Op9JOnG19VNEnjvjRN5DVd8QRHD+agTDw@mail.gmail.com

Attachments:

0001-Hash-partition-bound-equality-refactoring-v16.patchtext/x-patch; name=0001-Hash-partition-bound-equality-refactoring-v16.patchDownload
From 70074d78ebcb734fdf1dfd48c6df9105ec6de30b Mon Sep 17 00:00:00 2001
From: Etsuro Fujita <efujita@postgresql.org>
Date: Tue, 22 Jan 2019 21:10:54 +0900
Subject: [PATCH 1/3] Hash partition bound equality refactoring.

Separate the code to check whether two given hash bounds are equal into a
separate function, so that it can be called from multiple places. Right now
it's only caller is partition_bounds_equal() but later we will use it for
merging partition bounds.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/partitioning/partbounds.c | 95 +++++++++++++++++----------
 1 file changed, 61 insertions(+), 34 deletions(-)

diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index f21c9b32a6..cd49b5d01f 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -105,6 +105,9 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 						 Expr **keyCol,
 						 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static bool partition_hbounds_equal(PartitionBoundInfo b1,
+									PartitionBoundInfo b2);
+
 
 /*
  * get_qual_from_partbound
@@ -653,6 +656,63 @@ create_range_bounds(PartitionBoundSpec **boundspecs, int nparts,
 	return boundinfo;
 }
 
+/*
+ * Are two hash partition bound collections logically equal?
+ *
+ * Hash partition bounds store modulus and remainder in datums array which are
+ * always integers irrespective of the number of partition keys and their data
+ * types. Hence we can compare the hash bound collection without any partition
+ * key specific information. Separating this logic in a function which does not
+ * require partition key specific information allows it be called from places
+ * where the partition key specific information is not completely available.
+ */
+static bool
+partition_hbounds_equal(PartitionBoundInfo b1, PartitionBoundInfo b2)
+{
+	int			greatest_modulus = get_hash_partition_greatest_modulus(b1);
+	int			i;
+
+	Assert(b1->strategy == PARTITION_STRATEGY_HASH &&
+		   b2->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * If two hash partitioned tables have different greatest moduli,
+	 * their partition schemes don't match.  For hash partitioned table,
+	 * the greatest modulus is given by the last datum and number of
+	 * partitions is given by ndatums.
+	 */
+	if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
+		return false;
+
+	/*
+	 * We arrange the partitions in the ascending order of their modulus and
+	 * remainders.  Also every modulus is factor of next larger modulus.
+	 * Therefore we can safely store index of a given partition in indexes
+	 * array at remainder of that partition.  Also entries at (remainder + N *
+	 * modulus) positions in indexes array are all same for (modulus,
+	 * remainder) specification for any partition.  Thus datums array from both
+	 * the given bounds are same, if and only if their indexes array will be
+	 * same.  So, it suffices to compare indexes array.
+	 */
+	for (i = 0; i < greatest_modulus; i++)
+		if (b1->indexes[i] != b2->indexes[i])
+			return false;
+
+#ifdef USE_ASSERT_CHECKING
+
+	/*
+	 * Nonetheless make sure that the bounds are indeed same when the indexes
+	 * match.  Hash partition bound stores modulus and remainder at
+	 * b1->datums[i][0] and b1->datums[i][1] position respectively.
+	 */
+	for (i = 0; i < b1->ndatums; i++)
+		Assert((b1->datums[i][0] == b2->datums[i][0] &&
+				b1->datums[i][1] == b2->datums[i][1]));
+#endif
+
+	return true;
+}
+
 /*
  * Are two partition bound collections logically equal?
  *
@@ -681,41 +741,8 @@ partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
 
 	if (b1->strategy == PARTITION_STRATEGY_HASH)
 	{
-		int			greatest_modulus = get_hash_partition_greatest_modulus(b1);
-
-		/*
-		 * If two hash partitioned tables have different greatest moduli,
-		 * their partition schemes don't match.
-		 */
-		if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
+		if (!partition_hbounds_equal(b1, b2))
 			return false;
-
-		/*
-		 * We arrange the partitions in the ascending order of their moduli
-		 * and remainders.  Also every modulus is factor of next larger
-		 * modulus.  Therefore we can safely store index of a given partition
-		 * in indexes array at remainder of that partition.  Also entries at
-		 * (remainder + N * modulus) positions in indexes array are all same
-		 * for (modulus, remainder) specification for any partition.  Thus
-		 * datums array from both the given bounds are same, if and only if
-		 * their indexes array will be same.  So, it suffices to compare
-		 * indexes array.
-		 */
-		for (i = 0; i < greatest_modulus; i++)
-			if (b1->indexes[i] != b2->indexes[i])
-				return false;
-
-#ifdef USE_ASSERT_CHECKING
-
-		/*
-		 * Nonetheless make sure that the bounds are indeed same when the
-		 * indexes match.  Hash partition bound stores modulus and remainder
-		 * at b1->datums[i][0] and b1->datums[i][1] position respectively.
-		 */
-		for (i = 0; i < b1->ndatums; i++)
-			Assert((b1->datums[i][0] == b2->datums[i][0] &&
-					b1->datums[i][1] == b2->datums[i][1]));
-#endif
 	}
 	else
 	{
-- 
2.19.2

0002-Partition-wise-join-for-1-1-1-0-0-1-partition-matchi-v16.patchtext/x-patch; name=0002-Partition-wise-join-for-1-1-1-0-0-1-partition-matchi-v16.patchDownload
From 7c46da16346c5d7e54c905df60d39bef7b88cc0b Mon Sep 17 00:00:00 2001
From: Etsuro Fujita <efujita@postgresql.org>
Date: Tue, 22 Jan 2019 21:25:58 +0900
Subject: [PATCH 2/3] Partition-wise join for 1:1, 1:0, 0:1 partition matching.

Earlier version of partition-wise join implementation allowed
partition-wise join between two relations with exactly same partition
bounds. This commit allows partition-wise join to be applied under
following conditions

1. the partition bounds of joining relations are such that rows from
given partition on one side join can join with rows from maximum one
partition on the other side i.e. bounds of a given partition on one
side match/overlap with those of maximum one partition on the other
side. If the mapping happens to be m:n where m > 1 or n > 1, we have
to gang multiple partition relations together into a single relation.
This means that we have to add simple relations during join
processing, something which is not supported right now.  ALso, in such
a case, different pairs of joining relations can produce different
partition bounds for the same join relation, which again is not
supported right now.

2. For every partition on outer side that can contribute to the result
of an OUTER join, there exists at least one (taken along with item 1,
it means exactly one)  matching partition on the inner side. To
support partition-wise join when the inner matching partition doesn't
exist, we have to add a dummy base relation corresponding to the
non-existent inner partition. We don't have support to add base
relations during join processing.

3. For hash partitioned tables however, we support partition-wise join
only when the bounds exactly match. For hash partitioning it's unusual
to have missing partitions and hence generic partition matching is not
required.

With advanced partition matching, we won't be able to apply
partition-wise join when there are missing matching partitions. So for
a given N-way join, some sub-joins may not be partitioned, while the
overall N-way join is partitioned. We can not try partition-wise join
when one or both of the joining relations are not partitioned in one
of the pairs of joining relation. Skip partition-wise join for that
pair.

An example is A IJ B LJ C where B has an extra partition compared to A
and C. Join B LJ C requires dummy relation to join the extra partition
from B to a missing partition from C, and hence can not use
partition-wise join right now and hence BC is not partitioned.  The
extra partition in B gets eliminated in A IJ B and thus it can use
partition-wise join and is partitioned. Hence (AB)C can use
partition-wise join but not A(BC).

Not every pair of joining relation (including the one presented to
build_joinrel_partition_info()) for the same joinrel can use
partition-wise join or has both the relations partitioned. Hence we
calculate the partition bounds for the join relation in
try_partition_wise_join() instead of doing it here.

The partition bounds produced for the first pair that can use
partition-wise are saved in the RelOptInfo of the joinrel. Partition
bounds produced for subsequent pairs should match that produced by the
first pair.

Support for default partition in list partitioned tables
========================================================

When a list value is present in one of the joining relations and not
the other, and the other relation has default partition, match (join)
the partition containing that list value with the default partition.
If there are multiple matches, we can not proceed with the
partition-wise join similar to the case of matching non-default
partition. If the default partition happens to be on the outer side of
the join, the resulting join partition acts as a default partition as
it will contain all the values from the default partition. If
the partition containing the list value happens to be on the outer
side of the join, the resulting join partition is associated with the
list value, since no other partition key value from the default
partition makes it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join.  Such a
default partition may get joined to a partition from the inner side,
if inner side has a list value that is not present in the outer side.
But if such a join partner does not exist, we won't be able to use
partition-wise join since there is no (dummy) relation on the inner
side to join to.

Support for matching default partition in range partitioned tables
==================================================================

When a range is present in one of the joining relations and not the
other, and the other relation has default partition, match (join) the
partition corresponding range with the default partition. If two
ranges overlap but their ranges don't match exactly, the
non-overlapping portion from one range may join the previous
(non-overlapping portion near the lower bound, if exists), next
(non-overlapping portion near the upper bound, if exists) ranges from
the other relation or default partition from the other relation. This
causes multiple partitions on the other side to be joined with a
single partition on the first side; a case we don't support. Any range
from one side which doesn't overlap with any range on the other side,
may find its join partner in the default partition and the
corresponding range partition will need to be joined with the default
partition.  If the default partition happens to be on the outer side
of the join, the resulting join partition acts as a default partition
as it will contain all the values from the default partition. If the
non-partition corresponding to the range happens to be on the outer
side of the join, the resulting join partition is associated with that
range, since partition key values from the default partition outside
that range won't make it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join.  Such a
default partition may get joined to a non-default partition from the
inner side, if inner side has a range missing in the outer side.  But
if such a join partner does not exist, we won't be able to use
partition-wise join since there is no (dummy) relation on the inner
side to join to.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/optimizer/path/joinrels.c |   95 +-
 src/backend/optimizer/util/relnode.c  |   33 +-
 src/backend/partitioning/partbounds.c | 1676 ++++++++++++++++++++++++-
 src/include/partitioning/partbounds.h |    5 +
 4 files changed, 1763 insertions(+), 46 deletions(-)

diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 8bfe9c3ff7..578016c65d 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1319,25 +1319,31 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	PartitionScheme part_scheme;
+	PartitionBoundInfo join_boundinfo;
+	List	   *parts1;
+	List	   *parts2;
+	ListCell   *lc1;
+	ListCell   *lc2;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1350,32 +1356,63 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	 */
 	Assert(joinrel->part_scheme == rel1->part_scheme &&
 		   joinrel->part_scheme == rel2->part_scheme);
+	part_scheme = joinrel->part_scheme;
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * Get the list of matching partitions to be joined along with the
+	 * partition bounds of the join relation. Because of the restrictions
+	 * imposed by partition matching algorithm, not every pair of joining
+	 * relations for this join will be able to use partition-wise join. But all
+	 * those pairs which can use partition-wise join will produce the same
+	 * partition bounds for the join relation.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	join_boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											part_scheme->partsupfunc,
+											part_scheme->partcollation,
+											rel1, rel2,
+											parent_sjinfo->jointype,
+											&parts1, &parts2);
+
+	if (join_boundinfo == NULL)
+		return;
 
-	nparts = joinrel->nparts;
+	if (joinrel->boundinfo == NULL)
+	{
+		Assert(joinrel->nparts == 0 && joinrel->part_rels == NULL);
+		joinrel->boundinfo = join_boundinfo;
+		joinrel->nparts = list_length(parts1);
+		Assert(joinrel->nparts == list_length(parts2));
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) *
+									joinrel->nparts);
+	}
+	else
+	{
+		Assert(partition_bounds_equal(part_scheme->partnatts,
+									  part_scheme->parttyplen,
+									  part_scheme->parttypbyval,
+									  join_boundinfo, joinrel->boundinfo));
+		/*
+		 * Every pair of joining relations should result in the same number
+		 * of child-joins.
+		 */
+		Assert(joinrel->nparts == list_length(parts1));
+		Assert(joinrel->nparts == list_length(parts2));
+		Assert(joinrel->part_rels);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	cnt_parts = 0;
+	forboth(lc1, parts1, lc2, parts2)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
+		int			part1 = lfirst_int(lc1);
+		int			part2 = lfirst_int(lc2);
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1383,6 +1420,10 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		Assert(part1 >= 0 && part2 >= 0);
+		child_rel1 = rel1->part_rels[part1];
+		child_rel2 = rel2->part_rels[part2];
+
 		/*
 		 * If a child table has consider_partitionwise_join=false, it means
 		 * that it's a dummy relation for which we skipped setting up tlist
@@ -1437,12 +1478,24 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 			joinrel->part_rels[cnt_parts] = child_joinrel;
 		}
 
+		/*
+		 * For every pair of joining relations, the set of matching partitions
+		 * would change. However, the base relation partitions constituting
+		 * the given child should remain same for all the joining pairs. Since
+		 * the order in which children are stored in the array of child-joins,
+		 * depends upon partition bounds of the join, which are same for all
+		 * the joining pairs, every joining pair yields the child-joins in the
+		 * same order.
+		 */
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
 
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
+		cnt_parts++;
 	}
+
+	Assert(cnt_parts == joinrel->nparts);
 }
 
 /*
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index fe83ec4519..ad3552d761 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1608,7 +1608,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1621,24 +1621,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1650,17 +1632,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index cd49b5d01f..89c448040d 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -66,6 +66,12 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+typedef struct PartitionMap
+{
+	int from;
+	int to;
+} PartitionMap;
+
 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);
@@ -107,7 +113,58 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 static List *get_range_nulltest(PartitionKey key);
 static bool partition_hbounds_equal(PartitionBoundInfo b1,
 									PartitionBoundInfo b2);
-
+static PartitionBoundInfo partition_range_bounds_merge(
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 List **outer_parts, List **inner_parts,
+							 JoinType jointype, int partnatts,
+							 FmgrInfo *supfuncs, Oid *collations);
+static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype);
+static PartitionBoundInfo partition_hash_bounds_merge(RelOptInfo *outer_rel,
+							RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype);
+static void generate_matching_part_pairs(PartitionMap *outer_maps,
+										 PartitionMap *inner_maps,
+										 int nparts1, int nparts2,
+										 JoinType jointype, int nparts,
+										 List **parts1, List **parts2);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int map_and_merge_partitions(PartitionMap *outer_maps,
+										PartitionMap *inner_maps,
+										int index1, int index2, int *next_index);
+static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *collations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2);
+static bool partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
+						   Oid *collations, PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, int *ub_cmpval,
+						   int *lb_cmpval);
+static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes);
+static bool merge_default_partitions(PartitionBoundInfo outer_bi,
+						 			 PartitionBoundInfo inner_bi,
+						 			 PartitionMap *outer_maps,
+						 			 PartitionMap *inner_maps,
+						 			 JoinType jointype,
+									 int *next_index, int *default_index);
+static bool merge_null_partitions(PartitionBoundInfo outer_bi,
+								  PartitionBoundInfo inner_bi,
+								  PartitionMap *outer_maps,
+								  PartitionMap *inner_maps,
+								  JoinType jointype,
+								  int *next_index, int *null_index,
+								  int *default_index);
 
 /*
  * get_qual_from_partbound
@@ -2942,3 +2999,1620 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * The function produces the partition bounds for a join between two relations
+ * whose partition bounds are given. The function also returns two lists of
+ * partition indexes one for each of the joining relations. Both the lists
+ * contain the same number of elements. The partition indexes at the same
+ * positions in the lists indicate the pair partitions, one from each side, to
+ * be joined and the position itself corresponds to the index of partition
+ * produced by that child-join in the partitioned join.
+ *
+ * The function returns NULL if we can not find the matching pair of
+ * partitions. This happens if 1. multiple partitions on one side match with
+ * one partition on the other side. 2. a given partition on the outer side
+ * doesn't have a matching partition on the inner side. We can not support the
+ * first case since we don't have a way to represent multiple partitions as a
+ * single relation (RelOptInfo) and then perform join using the ganged
+ * relation. We can not support the second case since the missing inner
+ * partition needs to be represented as an empty relation and we don't have a
+ * way to introduce empty relation during join planning after creating paths
+ * for all the base relations.
+ */
+extern PartitionBoundInfo
+partition_bounds_merge(int partnatts, FmgrInfo *partsupfunc,
+					   Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
+{
+	PartitionBoundInfo 	merged_bounds;
+	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
+					   	inner_binfo = inner_rel->boundinfo;
+	char				strategy = outer_binfo->strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partsupfunc,
+														partcollation,
+														outer_rel, inner_rel,
+														outer_parts, inner_parts,
+														jointype);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(outer_rel, inner_rel,
+														 outer_parts, inner_parts,
+														 jointype, partnatts,
+														 partsupfunc,
+														 partcollation);
+			break;
+
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = partition_hash_bounds_merge(outer_rel, inner_rel,
+														outer_parts, inner_parts,
+														jointype);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	Assert((*outer_parts == NIL || *inner_parts != NIL) &&
+		   (*inner_parts == NIL || *outer_parts != NIL));
+
+	return merged_bounds;
+}
+
+/*
+ * partition_get_range_bounds
+ *
+ * Given the index of lower bound in datums array, return lower and upper
+ * bounds and the index of the partition with that lower bound.
+ */
+static int
+partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
+						   PartitionRangeBound *lower,
+						   PartitionRangeBound *upper)
+{
+	int			part_index;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The lower bound should correspond to a valid partition. */
+	part_index = bi->indexes[lb_index + 1];
+	Assert(part_index >= 0);
+
+	lower->kind = bi->kind[lb_index];
+	lower->datums = bi->datums[lb_index];
+	lower->lower = true;
+	upper->kind = bi->kind[lb_index + 1];
+	upper->datums = bi->datums[lb_index + 1];
+	upper->lower = false;
+
+	return part_index;
+}
+
+/*
+ * partition_range_get_next_lb_index
+ *
+ * Given the index of lower bound in datums array return the
+ * index of lower bound of the next partition. When the given index corresponds
+ * to the last partition, return number of datums (ndatums).
+ */
+static int
+partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
+{
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The partition index corresponding to the upper bound should be valid. */
+	Assert(bi->indexes[lb_index + 1] >= 0);
+
+	/*
+	 * If there are no bounds left beyond the upper bound, we have reached the
+	 * last partition.
+	 */
+	if (lb_index + 2 < bi->ndatums)
+	{
+		/*
+		 * If the bound next to the upper bound corresponds to no partition,
+		 * that's the next lower bound of the next partition. Otherwise, the
+		 * current upper bound is the lower bound of the next partition.
+		 */
+		if (bi->indexes[lb_index + 2] < 0)
+			return lb_index + 2;
+		else
+			return lb_index + 1;
+	}
+	else
+		return bi->ndatums;
+}
+
+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *partcollations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2)
+{
+	return partition_rbound_cmp(partnatts, partsupfunc, partcollations,
+								bound1->datums, bound1->kind, bound1->lower,
+								bound2);
+}
+
+/*
+ * partition_range_cmp
+ *
+ * Compare the bounds of two range partitions. Set ub_cmpval <, = or > 0, if the
+ * first partition's upper bound is lower than, equal to or higher than the
+ * second partition's upper bound resp. Similarly set lb_cmpval <, =  or > 0,
+ * if the first partition's lower bound is lower than, equal to or higher than
+ * the second partition's lower bound resp.
+ *
+ * Return true, if the ranges overlap, otherwise return false.
+ */
+static bool
+partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
+					PartitionRangeBound *lower_bound1,
+					PartitionRangeBound *upper_bound1,
+					PartitionRangeBound *lower_bound2,
+					PartitionRangeBound *upper_bound2, int *ub_cmpval,
+					int *lb_cmpval)
+{
+	bool		overlap;
+
+	/*
+	 * Compare upper bound of the first partition with the lower bound of the
+	 * second and vice-versa. If lower bound is higher than the upper bound,
+	 * the partitions are not overlapping. All other cases indicate overlapping
+	 * partitions.
+	 */
+	if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+								  lower_bound1, upper_bound2) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = 1;
+		*lb_cmpval = 1;
+	}
+	else if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+									   lower_bound2, upper_bound1) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = -1;
+		*lb_cmpval = -1;
+	}
+	else
+	{
+		overlap = true;
+		*ub_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, upper_bound1,
+											   upper_bound2);
+		*lb_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, lower_bound1,
+											   lower_bound2);
+	}
+
+	return overlap;
+}
+
+/*
+ * partition_range_merge
+ *
+ * Merge the partition bounds of given two partitions such that the join
+ * between the given two partitions fits merged bounds.
+ *
+ * "merged_upper" will be set to one of the given upper bounds and
+ * "merged_lower" will be set to one of the given lower bounds.
+ */
+static void
+partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
+					  Oid *partcollations, JoinType jointype,
+					  PartitionRangeBound *left_lb,
+					  PartitionRangeBound *left_ub,
+					  PartitionRangeBound *right_lb,
+					  PartitionRangeBound *right_ub,
+					  PartitionRangeBound **merged_lb,
+					  PartitionRangeBound **merged_ub)
+{
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = left_ub;
+			*merged_lb = left_lb;
+			break;
+
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) < 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) > 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		case JOIN_FULL:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) > 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) < 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+
+	return;
+}
+
+/*
+ * Add the lower bound of the next range to the list of bounds, if the lower
+ * bound is higher or equal to the previous upper bound. If successful return
+ * true, otherwise false.
+ */
+static bool
+partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
+							  Oid *partcollations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		prev_ub.datums = llast(*merged_datums);
+		prev_ub.kind = llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  next_lb_datums, next_lb_kind, false,
+									  &prev_ub);
+	}
+
+	/*
+	 * The lower bound is lower than the last upper bound, thus does not fit
+	 * the bounds created so far and hence can not be merged with the existing
+	 * bounds.
+	 */
+	if (cmpval < 0)
+		return false;
+
+	/*
+	 * Add bounds of the new merged partition. If the next lower bound is
+	 * higher than the last upper bound, add new range with index
+	 * corresponding to the lower bound as -1. If the merged lower bound
+	 * is same as the last merged upper bound, the last upper bound will be
+	 * reused as the lower bound of the next range.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, next_lb_datums);
+		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	return true;
+}
+
+/*
+ * handle_missing_partition
+ *
+ * If a range appears in one of the joining relations but not the other, a row
+ * in the corresponding partition will not have any join partner in the other
+ * relation, unless the other relation has a default partition. If a given list
+ * value is present in one joining relation but not the other, the default
+ * partition on the other side may contain that value.
+ *
+ * In both these cases, such an extra partition forms a joining pair with the
+ * default partition, if any,  on the other side.
+ *
+ * If the default partition happens to be on the outer side of the join, the
+ * resultant partition will act as the default partition of the join relation.
+ * Otherwise the resultant partition will be associated with the range.
+ *
+ * When the default partition is not present in the other relation, the rows in
+ * the extra partition will be included in the bounds of the join result, if it
+ * appears on the outer side of the join, since all rows from the outer side
+ * are included in the join result.
+ *
+ * This function handles all these cases.
+ *
+ * maps_with_missing and missing_side_default are the partition maps (See
+ * partition_range/list_bounds_merge() for details) and the index of default
+ * partition respectively corresponding the side with missing partition.
+ *
+ * maps_with_extra and extra_part are the partition maps (See
+ * partition_range/list_bounds_merge() for details) and the index of extra
+ * partition respectively corresponding to the side with the extra partition.
+ *
+ * It returns true if the matching succeeds, otherwise returns false.
+ */
+static bool
+handle_missing_partition(PartitionMap *maps_with_missing,
+						 PartitionMap *maps_with_extra,
+						 int missing_side_default,
+						 int extra_part,
+						 bool missing_side_outer,
+						 bool missing_side_inner,
+						 int *next_index, int *default_index,
+						 int *merged_index)
+{
+	bool missing_has_default = (missing_side_default != -1);
+
+	if (missing_has_default)
+	{
+		*merged_index = map_and_merge_partitions(maps_with_missing,
+												 maps_with_extra,
+												 missing_side_default,
+												 extra_part,
+												 next_index);
+		if (*merged_index < 0)
+			return false;
+
+		if (missing_side_outer)
+		{
+			/*
+			 * Default partition on the outer side forms the default
+			 * partition of the join result.
+			 */
+			if (*default_index < 0)
+				*default_index = *merged_index;
+			else if(*default_index != *merged_index)
+			{
+				/*
+				 * Ended up with default partition on the outer side
+				 * being joined with multiple partitions on the inner
+				 * side. We don't support this case.
+				 */
+				return false;
+			}
+
+			/*
+			 * Since the merged partition acts as a default partition, it
+			 * doesn't need a separate index.
+			 */
+			*merged_index = -1;
+		}
+	}
+	else if (missing_side_inner)
+	{
+		/*
+		 * If this partition has already been mapped (say because we
+		 * found an overlapping range earlier), we know where does it
+		 * fit in the join result. Nothing to do in that case. Else
+		 * create a new merged partition.
+		 */
+		PartitionMap *extra_map = &maps_with_extra[extra_part];
+		if (extra_map->to < 0)
+		{
+			extra_map->to = *next_index;
+			*next_index = *next_index + 1;
+			*merged_index = extra_map->to;
+		}
+	}
+
+	return true;
+}
+
+static PartitionMap*
+init_partition_map(RelOptInfo *rel)
+{
+	int i, nparts = rel->nparts;
+	PartitionMap *map;
+
+	map = (PartitionMap *) palloc(sizeof(PartitionMap) * nparts);
+
+	for (i = 0; i < nparts; i++)
+	{
+		map[i].from = -1;
+		map[i].to = -1;
+	}
+
+	return map;
+}
+
+/*
+ * Allocate and initialize partition maps. We maintain four maps, two maps
+ * for each joining relation. pmap[i] gives the partition from the other
+ * relation which would join with ith partition of the given relation.
+ * Partition i from the given relation will join with partition pmap[i]
+ * from the other relation to produce partition mmap[i] of the join (merged
+ * partition).
+ *
+ * pmap[i] = -1 indicates that ith partition of a given relation does not
+ * have a matching partition from the other relation.
+ *
+ * mmap[i] = -1 indicates that ith partition of a given relation does not
+ * contribute to the join result. That can happen only when the given
+ * relation is the inner relation and it doesn't have a matching partition
+ * from the outer relation, hence pmap[i] should be -1.
+ *
+ * In case of an outer join, every partition of the outer join will appear
+ * in the join result, and thus has mmap[i] set for all i. But it's not
+ * necessary that every partition on the outer side will have a matching
+ * partition on the inner side. In such a case, we end up with pmap[i] = -1
+ * and mmap[i] != -1.
+ */
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 List **outer_parts, List **inner_parts,
+							 JoinType jointype, int partnatts,
+							 FmgrInfo *partsupfuncs, Oid *partcollations)
+
+{
+	PartitionMap *outer_maps = NULL;
+	PartitionMap *inner_maps = NULL;
+	int			outer_part = 0;
+	int			inner_part = 0;
+	PartitionBoundInfo merged_bounds = NULL;
+	int			outer_lb_index;
+	int			inner_lb_index;
+	int			next_index;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	List	   *merged_kinds = NIL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			inner_default = inner_bi->default_index;
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	outer_maps = init_partition_map(outer_rel);
+	inner_maps = init_partition_map(inner_rel);
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = 0;
+	inner_lb_index = 0;
+	next_index = 0;
+	while (outer_lb_index < outer_bi->ndatums ||
+		   inner_lb_index < inner_bi->ndatums)
+	{
+		PartitionRangeBound outer_lb, outer_ub,
+							inner_lb, inner_ub,
+							*merged_lb = NULL,
+							*merged_ub = NULL;
+
+		int			merged_index = -1;
+		bool		overlap;
+		bool		finished_outer = false;
+		bool		finished_inner = false;
+
+		/* Result of bounds comparison per partition_rbound_cmp(). */
+		int			ub_cmpval;	/* Upper bounds comparison result. */
+		int			lb_cmpval;	/* Lower bounds comparison result. */
+
+		/* Get the range bounds of the next pair of partitions. */
+		if (outer_lb_index < outer_bi->ndatums)
+			outer_part = partition_get_range_bounds(outer_bi, outer_lb_index,
+												&outer_lb, &outer_ub);
+		else
+			finished_outer = true;
+
+		if (inner_lb_index < inner_bi->ndatums)
+			inner_part = partition_get_range_bounds(inner_bi, inner_lb_index,
+												&inner_lb, &inner_ub);
+		else
+			finished_inner = true;
+
+		Assert(!finished_outer || !finished_inner);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (finished_outer)
+		{
+			overlap = false;
+			ub_cmpval = 1;
+			lb_cmpval = 1;
+		}
+		else if (finished_inner)
+		{
+			overlap = false;
+			ub_cmpval = -1;
+			lb_cmpval = -1;
+		}
+		else
+			overlap = partition_range_cmp(partnatts, partsupfuncs, partcollations,
+										  &outer_lb, &outer_ub, &inner_lb,
+										  &inner_ub, &ub_cmpval, &lb_cmpval);
+
+		if (overlap)
+		{
+			/*
+			 * The rows from overlapping portion of ranges on both sides may
+			 * join, hence the corresponding pair of partitions form a joining
+			 * pair. Match them and produce the bounds of the joint partition
+			 * and its index by merging the bounds according to the type of
+			 * join.
+			 */
+			partition_range_merge(partnatts, partsupfuncs, partcollations,
+								  jointype, &outer_lb, &outer_ub, &inner_lb,
+								  &inner_ub, &merged_lb, &merged_ub);
+
+			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+													outer_part, inner_part,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				/* Failed to match the partitions. */
+				return NULL;
+			}
+
+			/*
+			 * If the ranges overlap but don't exactly match, a row from
+			 * non-overlapping portion of the range from one side of join may
+			 * find its join partner in the previous or next overlapping
+			 * partition or default partition on the other side , if such a
+			 * partition exists. All those cases, if true, will cause one
+			 * partition from that side to match at least two partitions on the
+			 * other side; a case that we do not support now. Previous
+			 * partition has been delt with in the previous iteration of this
+			 * loop, next partition will be delt in the next iteration. We will
+			 * deal with the default partition here.
+			 */
+			if ((lb_cmpval < 0 && inner_has_default) ||
+				/* Non-overlapping range on the lower side of outer range. */
+				(lb_cmpval > 0 && outer_has_default) ||
+				/* Non-overlapping range on the lower side of inner range. */
+				(ub_cmpval < 0 && outer_has_default) ||
+				/* Non-overlapping range on the upper side of inner range. */
+				(ub_cmpval > 0 && inner_has_default))
+				/* Non-overlapping range on the upper side of outer range. */
+				return NULL;
+		}
+
+		if (ub_cmpval == 0)
+		{
+			/* Upper bounds of both the ranges match. */
+			Assert(overlap);
+
+			/* Move to the next pair of partitions. */
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the inner side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &outer_lb;
+				merged_ub = &outer_ub;
+
+				/*
+				 * For a FULL join, inner relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the inner relation acts as
+				 * INNER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_inner = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_outer = (jointype == JOIN_FULL);
+				if (!handle_missing_partition(inner_maps,
+											  outer_maps,
+											  inner_default,
+											  outer_part,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the outer side. */
+			Assert(!finished_outer);
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+		}
+		else
+		{
+			Assert(ub_cmpval > 0);
+
+			/* Upper bound of outer range higher than that of the inner. */
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the outer side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &inner_lb;
+				merged_ub = &inner_ub;
+
+				/*
+				 * For a FULL join, outer relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the outer relation acts as
+				 * OUTER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_outer = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_inner = (jointype == JOIN_FULL);
+
+				if (!handle_missing_partition(outer_maps,
+											  inner_maps,
+											  outer_default,
+											  inner_part,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the inner side. */
+			Assert (!finished_inner);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+
+		if (merged_index < 0)
+		{
+			/* We didn't find a new merged partition. */
+			continue;
+		}
+
+		/*
+		 * We have a valid partition index for the next partition of join. The
+		 * partition should have valid range.
+		 */
+		Assert(merged_lb && merged_ub);
+
+		/* Try merging new lower bound with the last upper bound. */
+		if (!partition_range_merge_next_lb(partnatts, partsupfuncs,
+										   partcollations,
+										   merged_lb->datums,
+										   merged_lb->kind, &merged_datums,
+										   &merged_kinds, &merged_indexes))
+			return NULL;
+
+		/* Add upper bound with the merged partition index. */
+		merged_datums = lappend(merged_datums, merged_ub->datums);
+		merged_kinds = lappend(merged_kinds, merged_ub->kind);
+		merged_indexes = lappend_int(merged_indexes, merged_index);
+	}
+
+	if (!merge_default_partitions(outer_bi, inner_bi,
+										  outer_maps, inner_maps,
+										  jointype, &next_index,
+										  &default_index))
+		return NULL;
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_maps, inner_maps,
+								 outer_nparts, inner_nparts,
+								 jointype, next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes,
+													  merged_kinds,
+													  -1, default_index);
+	}
+
+	/* Free any memory we used in this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype)
+{
+	PartitionMap *outer_maps = NULL;
+	PartitionMap *inner_maps = NULL;
+	int			cnto;
+	int			cnti;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			next_index = 0;
+	int			null_index;
+	int			default_index = -1;
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			      *outer_indexes = outer_bi->indexes;
+	int			      *inner_indexes = inner_bi->indexes;
+	int				   outer_default = outer_bi->default_index;
+	int				   inner_default = inner_bi->default_index;
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	outer_maps = init_partition_map(outer_rel);
+	inner_maps = init_partition_map(inner_rel);
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			o_index;
+		int			i_index;
+		int			cmpval;
+		int			merged_index = -1;
+		Datum	   *merged_datum;
+		bool		finished_inner;
+		bool		finished_outer;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+		{
+			finished_outer = true;
+			odatums = NULL;
+			o_index = -1;
+		}
+		else
+		{
+			finished_outer = false;
+			odatums = outer_bi->datums[cnto];
+			o_index = outer_indexes[cnto];
+		}
+
+		if (cnti >= inner_bi->ndatums)
+		{
+			finished_inner = true;
+			idatums = NULL;
+			i_index = -1;
+		}
+		else
+		{
+			finished_inner = false;
+			idatums = inner_bi->datums[cnti];
+			i_index = inner_indexes[cnti];
+		}
+
+		/* If we exhausted both the sides, we won't enter the loop. */
+		Assert(!finished_inner || !finished_outer);
+
+		if (finished_outer)
+			cmpval = 1;
+		else if (finished_inner)
+			cmpval = -1;
+		else
+		{
+			/* Every list datum should map to a valid partition index. */
+			Assert(o_index >= 0 && i_index >= 0 &&
+				   odatums != NULL && idatums != NULL);
+
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0], idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+													o_index, i_index,
+													&next_index);
+
+			if (merged_index < 0)
+				return NULL;
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			/* A datum missing from the inner side. */
+			merged_index = -1;
+			merged_datum = odatums;
+
+			/*
+			 * For a FULL join, inner relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the inner relation acts as
+			 * INNER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_inner = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_outer = (jointype == JOIN_FULL);
+
+			if (!handle_missing_partition(inner_maps,
+										  outer_maps,
+										  inner_default,
+										  o_index,
+										  missing_side_outer,
+										  missing_side_inner,
+										  &next_index,
+										  &default_index,
+										  &merged_index))
+				return NULL;
+
+			/* Move to the next datum on the outer side. */
+			Assert(!finished_outer);
+			cnto++;
+		}
+		else
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			Assert(cmpval > 0);
+
+			/* A datum missing from the outer side. */
+			merged_index = -1;
+			merged_datum = idatums;
+
+			/*
+			 * For a FULL join, outer relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the outer relation acts as
+			 * OUTER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_outer = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_inner = (jointype == JOIN_FULL);
+
+			if (!handle_missing_partition(outer_maps,
+										  inner_maps,
+										  outer_default,
+										  i_index,
+										  missing_side_outer,
+										  missing_side_inner,
+										  &next_index,
+										  &default_index,
+										  &merged_index))
+				return NULL;
+
+			/* Move to the next datum on the right side. */
+			Assert(!finished_inner);
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	if (!merge_null_partitions(outer_bi, inner_bi,
+							   outer_maps, inner_maps,
+							   jointype, &next_index, &null_index,
+							   &default_index))
+		return NULL;
+
+	if (!merge_default_partitions(outer_bi, inner_bi,
+								  outer_maps, inner_maps,
+								  jointype, &next_index,
+								  &default_index))
+		return NULL;
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_maps, inner_maps,
+								 outer_nparts, inner_nparts,
+								 jointype, next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes, NIL,
+													  null_index, default_index);
+	}
+
+	/* Free up all extra memory before returning from this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_bounds_merge()'s arm for hash partitioned tables.
+ *
+ * If the given two hash bounds are same, the function returns the first one
+ * without any change, alongwith the lists of matching partitions. Otherwise it
+ * returns NULL.
+ *
+ * We could try merging the bounds when both the bounds have same greatest
+ * modulii. But there seems to be hardly any requirement for the same.
+ */
+static PartitionBoundInfo
+partition_hash_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype)
+{
+	int			nparts;
+	int			cnt;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * Hash partitioned table does not have explicit NULL accepting partition
+	 * and also does not have a default partition.
+	 */
+	Assert(!partition_bound_has_default(outer_bi) &&
+		   !partition_bound_has_default(inner_bi));
+	Assert(!partition_bound_accepts_nulls(outer_bi) &&
+		   !partition_bound_accepts_nulls(inner_bi));
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+
+	if (outer_nparts != inner_nparts)
+		return NULL;
+	nparts = outer_nparts;
+
+	if (outer_bi->ndatums != inner_bi->ndatums ||
+		!partition_hbounds_equal(outer_bi, inner_bi))
+		return NULL;
+
+	 /*
+	  * Cook up list of matching partitions. Since bounds are exactly same the
+	  * partitions at the same position from both the relations match.
+	  */
+	for (cnt = 0; cnt < nparts; cnt++)
+	{
+		*outer_parts = lappend_int(*outer_parts, cnt);
+		*inner_parts = lappend_int(*inner_parts, cnt);
+	}
+
+	return outer_bi;
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * If the two given partitions (given by index1 and index2 resp.) are
+ * already mapped to each other return the index of corresponding partition in
+ * the merged set of partitions.  If they do not have a merged partition
+ * associated with them, assign a new merged partition index.  If the
+ * partitions are already mapped and their mapped partitions are different from
+ * each other, they can not be merged, so return -1.
+ *
+ * partmaps1[i] gives the mapping of partitions for both relations. It
+ * describes which partition of relation 2 matches ith partition of relation 1,
+ * and which partition in the merged set matches ith partition of relation 1
+ * maps to. Similarly for partmap2.
+ *
+ * index1 and index2 are the indexes of matching partition from respective
+ * relations.
+ *
+ * *next_index is used and incremented when the given partitions require a new
+ * merged partition.
+ */
+
+static int
+map_and_merge_partitions(PartitionMap *partmaps1, PartitionMap *partmaps2,
+						 int index1, int index2, int *next_index)
+{
+	PartitionMap 	*partmap1 = &partmaps1[index1];
+	PartitionMap 	*partmap2 = &partmaps2[index2];
+	int				merged_index;
+
+	/*
+	 * If both the partitions are not mapped to each other, update the
+	 * maps.
+	 */
+	if (partmap1->from < 0 && partmap2->from < 0)
+	{
+		partmap1->from = index2;
+		partmap2->from = index1;
+	}
+
+	/*
+	 * If the given to partitions map to each other, find the corresponding
+	 * merged partition index .
+	 */
+	if (partmap1->from == index2 && partmap2->from == index1)
+	{
+		/*
+		 * If both the partitions are mapped to the same merged partition, get
+		 * the index of merged partition.
+		 */
+		if (partmap1->to == partmap2->to)
+		{
+			merged_index = partmap1->to;
+
+			/*
+			 * If the given two partitions do not have a merged partition
+			 * associated with them, allocate a new merged partition.
+			 */
+			if (merged_index < 0)
+			{
+				merged_index = *next_index;
+				*next_index = *next_index + 1;
+				partmap1->to = merged_index;
+				partmap2->to = merged_index;
+			}
+		}
+
+		/*
+		 * If partition from one relation was mapped to a merged partition but
+		 * not the partition from the other relation, map the same merged
+		 * partition to the partition from other relation, since matching
+		 * partitions map to the same merged partition.
+		 */
+		else if (partmap1->to >= 0 && partmap2->to < 0)
+		{
+			partmap2->to = partmap1->to;
+			merged_index = partmap1->to;
+		}
+		else if (partmap1->to < 0 && partmap2->to >= 0)
+		{
+			partmap1->to = partmap2->to;
+			merged_index = partmap2->to;
+		}
+		else
+		{
+			Assert(partmap1->to != partmap2->to &&
+				   partmap1->to >= 0 && partmap2->to >= 0);
+
+			/*
+			 * Both the partitions map to different merged partitions. This
+			 * means that multiple partitions from one relation matches to one
+			 * partition from the other relation. Partition-wise join does not
+			 * handle this case right now, since it requires ganging multiple
+			 * partitions together (into one RelOptInfo).
+			 */
+			merged_index = -1;
+		}
+	}
+	else
+	{
+		/*
+		 * Multiple partitions from one relation map to one partition from the
+		 * other relation. Partition-wise join does not handle this case right
+		 * now, since it requires ganging multiple partitions together (into
+		 * one RelOptInfo).
+		 */
+		merged_index = -1;
+	}
+
+	return merged_index;
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * partmaps1 map each partition from either side of the join to a merged
+ * partition resp. E.g. partmaps1[i].to gives the merged partition to which ith
+ * partition of first relation maps. Similarly for partmap2. If
+ * partmaps1[i].to == partmaps2[j].to, i and j form the matching pair of
+ * partitions.
+ *
+ * Given these maps this function produces the list pairs of partitions which
+ * when joined produce the merged partitions in the order of merged partition
+ * indexes.
+ *
+ * nparts1 and nparts2 are the number of partitions of the joining relations
+ * resp.
+ *
+ * nparts is the number of merged partitions.
+ *
+ * If successful, the pairs of partitions are returned as two separate lists,
+ * parts1 and parts2 resp., one for each side. Otherwise, those lists will be
+ * set to NIL.
+ */
+static void
+generate_matching_part_pairs(PartitionMap *partmaps1, PartitionMap *partmaps2,
+							 int nparts1, int nparts2,
+							 JoinType jointype, int nparts,
+							 List **matched_parts1, List **matched_parts2)
+{
+	bool	merged = true;
+	int		*matching1 = (int *) palloc(sizeof(int) * nparts),
+			*matching2 = (int *) palloc(sizeof(int) * nparts);
+	int 	i;
+
+	*matched_parts1 = NIL;
+	*matched_parts2 = NIL;
+
+	/* Set pairs of matching partitions. */
+	for (i = 0; i < nparts; i++)
+	{
+		if (i >= nparts1)
+			matching1[i] = -1;
+		else
+		{
+			PartitionMap outer_map = partmaps1[i];
+
+			if (outer_map.to >= 0)
+			{
+				Assert(outer_map.to < nparts);
+				matching1[outer_map.to] = i;
+			}
+		}
+
+		if (i >= nparts2)
+			matching2[i] = -1;
+		else
+		{
+			PartitionMap inner_map = partmaps2[i];
+
+			if (inner_map.to >= 0)
+			{
+				Assert(inner_map.to < nparts);
+				matching2[inner_map.to] = i;
+			}
+		}
+	}
+
+	/*
+	 * If we have a partition missing on an inner side, we need to add a dummy
+	 * relation which joins with the outer partition. If the inner relation
+	 * happens to be a base relation, it will require adding a dummy child
+	 * base relation during join processing. Right now, we freeze the base
+	 * relation arrays like PlannerInfo::simple_rte_array after planning for
+	 * base relations. Adding a new (dummy) base relation would require some
+	 * changes to that. So, right now, we do not implement partition-wise join
+	 * in such cases.
+	 */
+	for (i = 0; i < nparts; i++)
+	{
+		int			part1 = matching1[i];
+		int			part2 = matching2[i];
+
+		/* At least one of the partitions should exist. */
+		Assert(part1 >= 0 || part2 >= 0);
+
+		switch (jointype)
+		{
+			case JOIN_INNER:
+			case JOIN_SEMI:
+
+				/*
+				 * An inner or semi join can not return any row when the
+				 * matching partition on either side is missing. We should
+				 * have eliminated all such cases while merging the bounds.
+				 */
+				Assert(part1 >= 0 && part2 >= 0);
+				break;
+
+			case JOIN_LEFT:
+			case JOIN_ANTI:
+				Assert(part1 >= 0);
+				if (part2 < 0)
+					merged = false;
+				break;
+
+			case JOIN_FULL:
+				if (part1 < 0 || part2 < 0)
+					merged = false;
+				break;
+
+			default:
+				elog(ERROR, "unrecognized join type: %d", (int) jointype);
+		}
+
+		if (!merged)
+			break;
+
+		*matched_parts1 = lappend_int(*matched_parts1, part1);
+		*matched_parts2 = lappend_int(*matched_parts2, part2);
+	}
+
+	pfree(matching1);
+	pfree(matching2);
+
+	if (!merged)
+	{
+		list_free(*matched_parts1);
+		list_free(*matched_parts2);
+		*matched_parts1 = NIL;
+		*matched_parts2 = NIL;
+	}
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * Merge default partitions from both sides, if any, and assign the default
+ * partition for the join result, if necessary.
+ *
+ * If both the relations have default partitions, try mapping those to each
+ * other. If the mapping succeeds corresponding merged partition will act as
+ * the default partition of the join result.
+ *
+ * If inner side of the join has default but not the outer side, rows in it
+ * won't appear in the join result. So don't create a default partition. If
+ * outer side of the join has default but not the inner side, rows in it will
+ * appear in the join result, so create a default merged partition.
+ */
+static bool
+merge_default_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+						 PartitionMap *outer_maps, PartitionMap *inner_maps,
+						 JoinType jointype, int *next_index, int *default_index)
+{
+	int				outer_default = outer_bi->default_index;
+	int				inner_default = inner_bi->default_index;
+	bool			outer_has_default = partition_bound_has_default(outer_bi);
+	bool			inner_has_default = partition_bound_has_default(inner_bi);
+	bool			merged = true;
+	PartitionMap 	*outer_default_map = NULL;
+	PartitionMap 	*inner_default_map = NULL;
+
+	if (outer_has_default)
+		outer_default_map = &outer_maps[outer_default];
+
+	if (inner_has_default)
+		inner_default_map = &inner_maps[inner_default];
+
+	if (!outer_has_default && !inner_has_default)
+		Assert(*default_index < 0);
+	else if (outer_default_map != NULL && inner_default_map == NULL)
+	{
+		if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+			jointype == JOIN_ANTI)
+		{
+			if (outer_default_map->to < 0)
+			{
+				outer_default_map->to = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = outer_default_map->to;
+			}
+			else
+				Assert(*default_index == outer_default_map->to);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else if (outer_default_map == NULL && inner_default_map != NULL)
+	{
+		if (jointype == JOIN_FULL)
+		{
+			if (inner_default_map->to < 0)
+			{
+				inner_default_map->to = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = inner_default_map->to;
+			}
+			else
+				Assert(*default_index == inner_default_map->to);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		*default_index = map_and_merge_partitions(outer_maps, inner_maps,
+												  outer_default, inner_default,
+												  next_index);
+
+		if (*default_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
+
+/*
+ * merge_null_partitions
+ *
+ * Merge NULL partitions, i.e. a partition that can hold NULL values for a list
+ * partitioned table, if any. Find the index of merged partition to which the
+ * NULL values would belong in the join result. If one joining relation has a
+ * NULL partition but not the other, try matching it with the default partition
+ * from the other relation since the default partition may have rows with NULL
+ * partition key. We can eliminate a NULL partition when it appears only on the
+ * inner side of the join and the outer side doesn't have a default partition.
+ *
+ * When the equality operator used for join is strict, two NULL values will not
+ * be considered as equal, and thus a NULL partition can be eliminated for an
+ * inner join. But we don't check the strictness operator here.
+ */
+static bool
+merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_maps, PartitionMap *inner_maps,
+					  JoinType jointype, int *next_index,
+					  int *null_index, int *default_index)
+{
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	int			outer_ni = outer_bi->null_index;
+	int			inner_ni = inner_bi->null_index;
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	bool		merged = true;
+
+	if (!outer_has_null && !inner_has_null)
+		*null_index = -1;
+	else if (outer_has_null && !inner_has_null)
+	{
+		int			merged_index;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, inner relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the inner relation acts as
+		 * INNER relation. For INNER and SEMI join OUTER and INNER
+		 * differentiation is immaterial.
+		 */
+		missing_side_inner = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_outer = (jointype == JOIN_FULL);
+
+		merged = handle_missing_partition(inner_maps,
+										  outer_maps,
+										  inner_default,
+										  outer_ni,
+										  missing_side_outer,
+										  missing_side_inner, next_index,
+										  default_index, &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join to which the outer null partition maps
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = outer_maps[outer_ni].to;
+	}
+	else if (!outer_has_null && inner_has_null)
+	{
+		int			merged_index;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, outer relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the outer relation acts as OUTER
+		 * relation. For INNER and SEMI join OUTER and INNER differentiation is
+		 * immaterial.
+		 */
+		missing_side_outer = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_inner = (jointype == JOIN_FULL);
+		merged = handle_missing_partition(outer_maps,
+										  inner_maps,
+										  outer_default,
+										  inner_ni,
+										  missing_side_outer,
+										  missing_side_inner,
+										  next_index, default_index,
+										  &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join, to which the outer side null partition maps,
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = inner_maps[inner_ni].to;
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		*null_index = map_and_merge_partitions(outer_maps,
+											   inner_maps,
+											   outer_ni,
+											   inner_ni,
+											   next_index);
+		if (*null_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index b1ae39ad63..a71f6d99e6 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -107,5 +107,10 @@ extern int partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 							  int nvalues, Datum *values, bool *is_equal);
 extern int partition_hash_bsearch(PartitionBoundInfo boundinfo,
 					   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
-- 
2.19.2

0003-Tests-for-0-1-1-1-and-1-0-partition-matching-v16.patchtext/x-patch; name=0003-Tests-for-0-1-1-1-and-1-0-partition-matching-v16.patchDownload
From b648d8490b7a9af970530eff3cfdf44e38baf2e9 Mon Sep 17 00:00:00 2001
From: Etsuro Fujita <efujita@postgresql.org>
Date: Tue, 22 Jan 2019 21:28:58 +0900
Subject: [PATCH 3/3] Tests for 0:1, 1:1 and 1:0 partition matching

Rajkumar Raghuvanshi and Ashutosh Bapat.
---
 src/test/regress/expected/partition_join.out | 4131 +++++++++++++++---
 src/test/regress/sql/partition_join.sql      |  427 +-
 2 files changed, 3855 insertions(+), 703 deletions(-)

diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index c55de5d476..f19611c853 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -8,59 +8,86 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-(4 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+(7 rows)
 
 -- left outer join, with whole-row reference; partitionwise join does not apply
 EXPLAIN (COSTS OFF)
@@ -72,35 +99,50 @@ SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER
    ->  Hash Right Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(22 rows)
 
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-      t1      |      t2      
---------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
- (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
- (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
- (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
- (550,0,0550) | 
-(12 rows)
+       t1       |       t2       
+----------------+----------------
+ (-250,0,-0250) | 
+ (-200,0,-0200) | 
+ (-150,0,-0150) | (0,-150,-0150)
+ (-100,0,-0100) | 
+ (-50,0,-0050)  | 
+ (0,0,0000)     | (0,0,0000)
+ (50,0,0050)    | 
+ (100,0,0100)   | 
+ (150,0,0150)   | (0,150,0150)
+ (200,0,0200)   | 
+ (250,0,0250)   | 
+ (300,0,0300)   | (0,300,0300)
+ (350,0,0350)   | 
+ (400,0,0400)   | 
+ (450,0,0450)   | (0,450,0450)
+ (500,0,0500)   | 
+ (550,0,0550)   | 
+ (600,0,0600)   | (0,600,0600)
+ (650,0,0650)   | 
+ (700,0,0700)   | 
+ (750,0,0750)   | (0,750,0750)
+(21 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
@@ -112,35 +154,53 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
    ->  Append
          ->  Hash Right Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Right Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t2_2.b)
-(20 rows)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+      |       | -225 | -0225
+      |       |  -75 | -0075
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(14 rows)
 
 -- full outer join, with placeholder vars
 EXPLAIN (COSTS OFF)
@@ -148,8 +208,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                             QUERY PLAN                            
 ------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               Filter: (((50) = prt1_p0.a) OR ((75) = prt2_p0.b))
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
@@ -174,7 +242,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               Filter: (((50) = prt1_p4.a) OR ((75) = prt2_p4.b))
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
@@ -211,8 +287,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Hash Left Join
+               Hash Cond: (prt1_p0.a = b)
+               ->  Seq Scan on prt1_p0
+                     Filter: ((a < 450) AND (b = 0))
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
          ->  Hash Left Join
                Hash Cond: (prt1_p1.a = b)
                ->  Seq Scan on prt1_p1
@@ -227,29 +310,42 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                ->  Hash
                      ->  Seq Scan on prt1_p2
                            Filter: ((a < 450) AND (b = 0))
-(17 rows)
+(24 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-(9 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+(14 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
                          QUERY PLAN                         
 ------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = b)
+               Filter: ((prt1_p0.b = 0) OR (a = 0))
+               ->  Seq Scan on prt1_p0
+                     Filter: (a < 450)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = b)
                Filter: ((prt1_p1.b = 0) OR (a = 0))
@@ -274,64 +370,153 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                ->  Hash
                      ->  Result
                            One-Time Filter: false
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt2_p4.b = a)
+               Filter: ((b = 0) OR (prt2_p4.a = 0))
+               ->  Seq Scan on prt2_p4
+                     Filter: (b > 250)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
-(12 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+      |       | 375 | 0375
+      |       | 450 | 0450
+      |       | 525 | 0525
+      |       | 600 | 0600
+      |       | 675 | 0675
+      |       | 750 | 0750
+(20 rows)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Semi Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Semi Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                            Filter: (a = 0)
-         ->  Nested Loop Semi Join
-               Join Filter: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
-               ->  Materialize
-                     ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
-(24 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt1_p3 t2_3
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+                           Filter: (b = 0)
+(37 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
 
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
@@ -342,27 +527,82 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
    ->  Append
          ->  Hash Anti Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Hash Anti Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Hash Anti Join
                Hash Cond: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
-(17 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(27 rows)
 
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
-  sum  |         avg          | sum  |         avg         
--------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+  sum  |         avg          | sum  |        avg         
+-------+----------------------+------+--------------------
+ 95550 | 273.0000000000000000 | 2200 | 6.2857142857142857
 (1 row)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Append
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p0 t1
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+               Index Cond: (a = t1.b)
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.b = t2_1.a)
+         ->  Seq Scan on prt2_p1 t1_1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p1 t2_1
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p2 t1_2
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+               Index Cond: (a = t1_2.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p3 t1_3
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+               Index Cond: (a = t1_3.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p4 t1_4
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+               Index Cond: (a = t1_4.b)
+(27 rows)
+
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+  b   |   c   
+------+-------
+ -225 | -0225
+  -75 | -0075
+   75 | 0075
+  225 | 0225
+  375 | 0375
+  525 | 0525
+  675 | 0675
+(7 rows)
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -374,49 +614,74 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
+                     ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
                            Index Cond: (a = t1.a)
-                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t3
                            Index Cond: (b = t2.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_1
+                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
                            Index Cond: (a = t1_1.a)
-                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3_1
                            Index Cond: (b = t2_1.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_2
+                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
                            Index Cond: (a = t1_2.a)
-                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_2
                            Index Cond: (b = t2_2.a)
-(27 rows)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                           Index Cond: (a = t1_3.a)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_3
+                           Index Cond: (b = t2_3.a)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                           Index Cond: (a = t1_4.a)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t3_4
+                           Index Cond: (b = t2_4.a)
+(43 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   | t2a | t3a | least 
------+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
- 100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
- 250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
- 400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
- 550 | 0 | 0550 |     |     |      
-(12 rows)
+  a   | b |   c   | t2a  | t3a | least 
+------+---+-------+------+-----+-------
+ -250 | 0 | -0250 |      |     |      
+ -200 | 0 | -0200 |      |     |      
+ -150 | 0 | -0150 | -150 |   0 |  -150
+ -100 | 0 | -0100 |      |     |      
+  -50 | 0 | -0050 |      |     |      
+    0 | 0 | 0000  |    0 |   0 |     0
+   50 | 0 | 0050  |      |     |      
+  100 | 0 | 0100  |      |     |      
+  150 | 0 | 0150  |  150 |   0 |   150
+  200 | 0 | 0200  |      |     |      
+  250 | 0 | 0250  |      |     |      
+  300 | 0 | 0300  |  300 |   0 |   300
+  350 | 0 | 0350  |      |     |      
+  400 | 0 | 0400  |      |     |      
+  450 | 0 | 0450  |  450 |   0 |   450
+  500 | 0 | 0500  |      |     |      
+  550 | 0 | 0550  |      |     |      
+  600 | 0 | 0600  |  600 |   0 |   600
+  650 | 0 | 0650  |      |     |      
+  700 | 0 | 0700  |      |     |      
+  750 | 0 | 0750  |  750 |   0 |   750
+(21 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
@@ -430,64 +695,95 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
          Hash Cond: ((t1.c)::text = (t2.c)::text)
          Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Seq Scan on prt1_p1 t1
-               ->  Seq Scan on prt1_p2 t1_1
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
          ->  Hash
                ->  Append
                      ->  Hash Join
                            Hash Cond: (t2.a = t3.b)
-                           ->  Seq Scan on prt1_p1 t2
+                           ->  Seq Scan on prt1_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t3
+                                 ->  Seq Scan on prt2_p0 t3
                      ->  Hash Join
                            Hash Cond: (t2_1.a = t3_1.b)
-                           ->  Seq Scan on prt1_p2 t2_1
+                           ->  Seq Scan on prt1_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t3_1
+                                 ->  Seq Scan on prt2_p1 t3_1
                      ->  Hash Join
                            Hash Cond: (t2_2.a = t3_2.b)
-                           ->  Seq Scan on prt1_p3 t2_2
+                           ->  Seq Scan on prt1_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p3 t3_2
-(26 rows)
+                                 ->  Seq Scan on prt2_p2 t3_2
+                     ->  Hash Join
+                           Hash Cond: (t2_3.a = t3_3.b)
+                           ->  Seq Scan on prt1_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t3_3
+                     ->  Hash Join
+                           Hash Cond: (t2_4.a = t3_4.b)
+                           ->  Seq Scan on prt1_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t3_4
+(38 rows)
 
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, t2.b t2b, t2.c t2c, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
-  a  | t2a | t2c  
------+-----+------
-   0 |   0 | 0000
-  50 |     | 
- 100 |     | 
- 150 | 150 | 0150
- 200 |     | 
- 250 |     | 
- 300 | 300 | 0300
- 350 |     | 
- 400 |     | 
- 450 | 450 | 0450
- 500 |     | 
- 550 |     | 
-(12 rows)
+  a   | t2a  |  t2c  
+------+------+-------
+ -250 |      | 
+ -200 |      | 
+ -150 | -150 | -0150
+ -100 |      | 
+  -50 |      | 
+    0 |    0 | 0000
+   50 |      | 
+  100 |      | 
+  150 |  150 | 0150
+  200 |      | 
+  250 |      | 
+  300 |  300 | 0300
+  350 |      | 
+  400 |      | 
+  450 |  450 | 0450
+  500 |      | 
+  550 |      | 
+  600 |  600 | 0600
+  650 |      | 
+  700 |      | 
+  750 |  750 | 0750
+(21 rows)
 
 --
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
@@ -498,32 +794,49 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
    ->  Append
          ->  Hash Join
                Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
-               ->  Seq Scan on prt2_e_p1 t2
+               ->  Seq Scan on prt2_e_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 t1
+                     ->  Seq Scan on prt1_e_p0 t1
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
-               ->  Seq Scan on prt2_e_p2 t2_1
+               Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+               ->  Seq Scan on prt1_e_p1 t1_1
+                     Filter: (c = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 t1_1
-                           Filter: (c = 0)
+                     ->  Seq Scan on prt2_e_p1 t2_1
          ->  Hash Join
                Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
-               ->  Seq Scan on prt2_e_p3 t2_2
+               ->  Seq Scan on prt2_e_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 t1_2
+                     ->  Seq Scan on prt1_e_p2 t1_2
                            Filter: (c = 0)
-(21 rows)
+         ->  Hash Join
+               Hash Cond: (((t2_3.b + t2_3.a) / 2) = ((t1_3.a + t1_3.b) / 2))
+               ->  Seq Scan on prt2_e_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p3 t1_3
+                           Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (((t2_4.b + t2_4.a) / 2) = ((t1_4.a + t1_4.b) / 2))
+               ->  Seq Scan on prt2_e_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4 t1_4
+                           Filter: (c = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
- 150 | 0 | 150 | 0
- 300 | 0 | 300 | 0
- 450 | 0 | 450 | 0
-(4 rows)
+  a   | c |  b   | c 
+------+---+------+---
+ -250 | 0 | -250 | 0
+ -100 | 0 | -100 | 0
+    0 | 0 |    0 | 0
+    0 | 0 |    0 | 0
+  150 | 0 |  150 | 0
+  300 | 0 |  300 | 0
+  450 | 0 |  450 | 0
+  600 | 0 |  600 | 0
+  750 | 0 |  750 | 0
+(9 rows)
 
 --
 -- N-way join
@@ -536,154 +849,232 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = ((t3.a + t3.b) / 2))
+               Join Filter: (t1.a = t2.b)
                ->  Hash Join
-                     Hash Cond: (t2.b = t1.a)
-                     ->  Seq Scan on prt2_p1 t2
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on prt1_e_p0 t3
                      ->  Hash
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                     Index Cond: (((a + b) / 2) = t2.b)
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
+                     Index Cond: (b = ((t3.a + t3.b) / 2))
          ->  Nested Loop
                Join Filter: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_1.b = t1_1.a)
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
+               ->  Index Scan using iprt1_e_p4_ab2 on prt1_e_p1 t3_1
                      Index Cond: (((a + b) / 2) = t2_1.b)
          ->  Nested Loop
                Join Filter: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_2.b = t1_2.a)
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
+               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_2
                      Index Cond: (((a + b) / 2) = t2_2.b)
-(33 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = ((t3_3.a + t3_3.b) / 2))
+               ->  Nested Loop
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t2_3.b)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t2_4.b)
+               ->  Hash Join
+                     Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                     ->  Seq Scan on prt1_e_p4 t3_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_p4 t1_4
+                                 Filter: (b = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = ((t3_4.a + t3_4.b) / 2))
+(52 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
-(4 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(8 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                          QUERY PLAN                          
---------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Hash Right Join
                Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
-               ->  Seq Scan on prt1_e_p1 t3
+               ->  Seq Scan on prt1_e_p0 t3
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2.b = t1.a)
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_1.a)
-               ->  Seq Scan on prt1_e_p2 t3_1
+               ->  Seq Scan on prt1_e_p1 t3_1
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_1.b = t1_1.a)
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_2.a)
-               ->  Seq Scan on prt1_e_p3 t3_2
+               ->  Seq Scan on prt1_e_p2 t3_2
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_2.b = t1_2.a)
-                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                                        Filter: (b = 0)
-(33 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (t1_3.a = b)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (t1_3.a = ((a + b) / 2))
+         ->  Hash Right Join
+               Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+               ->  Seq Scan on prt1_e_p4 t3_4
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_4.b = t1_4.a)
+                           ->  Seq Scan on prt2_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_p4 t1_4
+                                       Filter: (b = 0)
+(51 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                            QUERY PLAN                             
--------------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p1 t3
+                           ->  Seq Scan on prt1_e_p0 t3
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p1_b on prt2_p1 t2
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
                      Index Cond: (t1.a = b)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p2 t3_1
+                           ->  Seq Scan on prt1_e_p1 t3_1
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
+               ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
                      Index Cond: (t1_1.a = b)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p3 t3_2
+                           ->  Seq Scan on prt1_e_p2 t3_2
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
                      Index Cond: (t1_2.a = b)
-(30 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_e_p3 t3_3
+                           Filter: (c = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                           Index Cond: (a = ((t3_3.a + t3_3.b) / 2))
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (t1_3.a = b)
+         ->  Nested Loop Left Join
+               ->  Hash Right Join
+                     Hash Cond: (t1_4.a = ((t3_4.a + t3_4.b) / 2))
+                     ->  Seq Scan on prt1_p4 t1_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p4 t3_4
+                                 Filter: (c = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (t1_4.a = b)
+(47 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -692,21 +1083,34 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                                                    QUERY PLAN                                                   
 ----------------------------------------------------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   Sort Key: prt1_p0.a, prt2_p0.b, ((prt1_e_p0.a + prt1_e_p0.b))
    ->  Append
          ->  Hash Full Join
-               Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
-               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               Hash Cond: (prt1_p0.a = ((prt1_e_p0.a + prt1_e_p0.b) / 2))
+               Filter: ((prt1_p0.a = (50)) OR (prt2_p0.b = (75)) OR (((prt1_e_p0.a + prt1_e_p0.b) / 2) = (50)))
                ->  Hash Full Join
-                     Hash Cond: (prt1_p1.a = prt2_p1.b)
-                     ->  Seq Scan on prt1_p1
+                     Hash Cond: (prt1_p0.a = prt2_p0.b)
+                     ->  Seq Scan on prt1_p0
                            Filter: (b = 0)
                      ->  Hash
-                           ->  Seq Scan on prt2_p1
+                           ->  Seq Scan on prt2_p0
                                  Filter: (a = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1
+                     ->  Seq Scan on prt1_e_p0
                            Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = prt1_p1.a)
+               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               ->  Seq Scan on prt1_e_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on prt1_p1
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1
+                                       Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
                Filter: ((prt1_p2.a = (50)) OR (prt2_p2.b = (75)) OR (((prt1_e_p2.a + prt1_e_p2.b) / 2) = (50)))
@@ -733,7 +1137,20 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                ->  Hash
                      ->  Seq Scan on prt1_e_p3
                            Filter: (c = 0)
-(42 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = ((prt1_e_p4.a + prt1_e_p4.b) / 2))
+               Filter: ((prt1_p4.a = (50)) OR (prt2_p4.b = (75)) OR (((prt1_e_p4.a + prt1_e_p4.b) / 2) = (50)))
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = prt2_p4.b)
+                     ->  Seq Scan on prt1_p4
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4
+                                 Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4
+                           Filter: (c = 0)
+(68 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
@@ -751,172 +1168,260 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = t1_3.b)
+               Join Filter: (t1.a = t1_5.b)
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Join
-                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
-                           ->  Seq Scan on prt1_e_p1 t2
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t1_3
+                                 ->  Seq Scan on prt2_p0 t1_5
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
                      Index Cond: (a = ((t2.a + t2.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_1.a = t1_4.b)
+               Join Filter: (t1_1.a = t1_6.b)
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Join
-                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
-                           ->  Seq Scan on prt1_e_p2 t2_1
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_6.b)
+                           ->  Seq Scan on prt1_e_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t1_4
+                                 ->  Seq Scan on prt2_p1 t1_6
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
                      Index Cond: (a = ((t2_1.a + t2_1.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_2.a = t1_5.b)
+               Join Filter: (t1_2.a = t1_7.b)
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
+                           ->  Seq Scan on prt2_p2 t1_7
                                  Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
+                           ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t2_2
+                                 Index Cond: (((a + b) / 2) = t1_7.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t1_8.b)
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Nested Loop
+                           ->  Seq Scan on prt2_p3 t1_8
+                                 Filter: (a = 0)
+                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_3
+                                 Index Cond: (((a + b) / 2) = t1_8.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = ((t2_3.a + t2_3.b) / 2))
+                     Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t1_9.b)
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Join
+                           Hash Cond: (((t2_4.a + t2_4.b) / 2) = t1_9.b)
+                           ->  Seq Scan on prt1_e_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t1_9
+                                       Filter: (a = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = ((t2_4.a + t2_4.b) / 2))
+                     Filter: (b = 0)
+(66 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                               QUERY PLAN                                
--------------------------------------------------------------------------
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
-                           ->  Seq Scan on prt2_p1 t1_3
+                           Hash Cond: (t1_5.b = ((t1_10.a + t1_10.b) / 2))
+                           ->  Seq Scan on prt2_p0 t1_5
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t1_6
+                                 ->  Seq Scan on prt1_e_p0 t1_10
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
-                     Index Cond: (a = t1_3.b)
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t1_5.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
-                           ->  Seq Scan on prt2_p2 t1_4
+                           Hash Cond: (t1_6.b = ((t1_11.a + t1_11.b) / 2))
+                           ->  Seq Scan on prt2_p1 t1_6
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t1_7
+                                 ->  Seq Scan on prt1_e_p1 t1_11
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
-                     Index Cond: (a = t1_4.b)
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
+                     Index Cond: (a = t1_6.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
-                           ->  Seq Scan on prt2_p3 t1_5
+                           Hash Cond: (t1_7.b = ((t1_12.a + t1_12.b) / 2))
+                           ->  Seq Scan on prt2_p2 t1_7
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p3 t1_8
+                                 ->  Seq Scan on prt1_e_p2 t1_12
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t1_5.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t1_7.b)
                      Filter: (b = 0)
-(39 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_8.b = ((t1_13.a + t1_13.b) / 2))
+                           ->  Seq Scan on prt2_p3 t1_8
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p3 t1_13
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t1_8.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_9.b = ((t1_14.a + t1_14.b) / 2))
+                           ->  Seq Scan on prt2_p4 t1_9
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t1_14
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = t1_9.b)
+                     Filter: (b = 0)
+(63 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 -- test merge joins
 SET enable_hashjoin TO off;
 SET enable_nestloop TO off;
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                           QUERY PLAN                           
-----------------------------------------------------------------
+                            QUERY PLAN                            
+------------------------------------------------------------------
  Merge Append
    Sort Key: t1.a
    ->  Merge Semi Join
-         Merge Cond: (t1.a = t1_3.b)
+         Merge Cond: (t1.a = t1_5.b)
          ->  Sort
                Sort Key: t1.a
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+               Merge Cond: (t1_5.b = (((t1_10.a + t1_10.b) / 2)))
                ->  Sort
-                     Sort Key: t1_3.b
-                     ->  Seq Scan on prt2_p1 t1_3
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on prt2_p0 t1_5
                ->  Sort
-                     Sort Key: (((t1_6.a + t1_6.b) / 2))
-                     ->  Seq Scan on prt1_e_p1 t1_6
+                     Sort Key: (((t1_10.a + t1_10.b) / 2))
+                     ->  Seq Scan on prt1_e_p0 t1_10
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_1.a = t1_4.b)
+         Merge Cond: (t1_1.a = t1_6.b)
          ->  Sort
                Sort Key: t1_1.a
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+               Merge Cond: (t1_6.b = (((t1_11.a + t1_11.b) / 2)))
                ->  Sort
-                     Sort Key: t1_4.b
-                     ->  Seq Scan on prt2_p2 t1_4
+                     Sort Key: t1_6.b
+                     ->  Seq Scan on prt2_p1 t1_6
                ->  Sort
-                     Sort Key: (((t1_7.a + t1_7.b) / 2))
-                     ->  Seq Scan on prt1_e_p2 t1_7
+                     Sort Key: (((t1_11.a + t1_11.b) / 2))
+                     ->  Seq Scan on prt1_e_p1 t1_11
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_2.a = t1_5.b)
+         Merge Cond: (t1_2.a = t1_7.b)
          ->  Sort
                Sort Key: t1_2.a
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+               Merge Cond: (t1_7.b = (((t1_12.a + t1_12.b) / 2)))
                ->  Sort
-                     Sort Key: t1_5.b
-                     ->  Seq Scan on prt2_p3 t1_5
+                     Sort Key: t1_7.b
+                     ->  Seq Scan on prt2_p2 t1_7
                ->  Sort
-                     Sort Key: (((t1_8.a + t1_8.b) / 2))
-                     ->  Seq Scan on prt1_e_p3 t1_8
+                     Sort Key: (((t1_12.a + t1_12.b) / 2))
+                     ->  Seq Scan on prt1_e_p2 t1_12
                            Filter: (c = 0)
-(47 rows)
+   ->  Merge Semi Join
+         Merge Cond: (t1_3.a = t1_8.b)
+         ->  Sort
+               Sort Key: t1_3.a
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_8.b = (((t1_13.a + t1_13.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_8.b
+                     ->  Seq Scan on prt2_p3 t1_8
+               ->  Sort
+                     Sort Key: (((t1_13.a + t1_13.b) / 2))
+                     ->  Seq Scan on prt1_e_p3 t1_13
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_4.a = t1_9.b)
+         ->  Sort
+               Sort Key: t1_4.a
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_9.b = (((t1_14.a + t1_14.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_9.b
+                     ->  Seq Scan on prt2_p4 t1_9
+               ->  Sort
+                     Sort Key: (((t1_14.a + t1_14.b) / 2))
+                     ->  Seq Scan on prt1_e_p4 t1_14
+                           Filter: (c = 0)
+(77 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
@@ -933,14 +1438,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
                            ->  Sort
                                  Sort Key: (((t3.a + t3.b) / 2))
-                                 ->  Seq Scan on prt1_e_p1 t3
+                                 ->  Seq Scan on prt1_e_p0 t3
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1.a
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                ->  Sort
                      Sort Key: t2.b
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Merge Left Join
                Merge Cond: (t1_1.a = t2_1.b)
                ->  Sort
@@ -949,14 +1454,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
                            ->  Sort
                                  Sort Key: (((t3_1.a + t3_1.b) / 2))
-                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                 ->  Seq Scan on prt1_e_p1 t3_1
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_1.a
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                ->  Sort
                      Sort Key: t2_1.b
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Merge Left Join
                Merge Cond: (t1_2.a = t2_2.b)
                ->  Sort
@@ -965,32 +1470,74 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
                            ->  Sort
                                  Sort Key: (((t3_2.a + t3_2.b) / 2))
-                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                 ->  Seq Scan on prt1_e_p2 t3_2
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_2.a
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                ->  Sort
                      Sort Key: t2_2.b
-                     ->  Seq Scan on prt2_p3 t2_2
-(51 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Merge Left Join
+               Merge Cond: (t1_3.a = t2_3.b)
+               ->  Sort
+                     Sort Key: t1_3.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_3.a + t3_3.b) / 2)) = t1_3.a)
+                           ->  Sort
+                                 Sort Key: (((t3_3.a + t3_3.b) / 2))
+                                 ->  Seq Scan on prt1_e_p3 t3_3
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_3.a
+                                 ->  Seq Scan on prt1_p3 t1_3
+               ->  Sort
+                     Sort Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Merge Left Join
+               Merge Cond: (t1_4.a = t2_4.b)
+               ->  Sort
+                     Sort Key: t1_4.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_4.a + t3_4.b) / 2)) = t1_4.a)
+                           ->  Sort
+                                 Sort Key: (((t3_4.a + t3_4.b) / 2))
+                                 ->  Seq Scan on prt1_e_p4 t3_4
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_4.a
+                                 ->  Seq Scan on prt1_p4 t1_4
+               ->  Sort
+                     Sort Key: t2_4.b
+                     ->  Seq Scan on prt2_p4 t2_4
+(83 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- MergeAppend on nullable column
 EXPLAIN (COSTS OFF)
@@ -998,8 +1545,18 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Merge Left Join
+               Merge Cond: (prt1_p0.a = b)
+               ->  Sort
+                     Sort Key: prt1_p0.a
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
+               ->  Sort
+                     Sort Key: b
+                     ->  Result
+                           One-Time Filter: false
          ->  Merge Left Join
                Merge Cond: (prt1_p1.a = b)
                ->  Sort
@@ -1020,21 +1577,26 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                      Sort Key: prt2_p2.b
                      ->  Seq Scan on prt2_p2
                            Filter: (b > 250)
-(23 rows)
+(33 rows)
 
 SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  b  
------+-----
-   0 |    
-  50 |    
- 100 |    
- 150 |    
- 200 |    
- 250 |    
- 300 | 300
- 350 |    
- 400 |    
-(9 rows)
+  a   |  b  
+------+-----
+ -250 |    
+ -200 |    
+ -150 |    
+ -100 |    
+  -50 |    
+    0 |    
+   50 |    
+  100 |    
+  150 |    
+  200 |    
+  250 |    
+  300 | 300
+  350 |    
+  400 |    
+(14 rows)
 
 -- merge join when expression with whole-row reference needs to be sorted;
 -- partitionwise join does not apply
@@ -1048,175 +1610,2412 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
          Sort Key: t1.a, ((((t1.*)::prt1))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
    ->  Sort
          Sort Key: t2.b, ((((t2.*)::prt2))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt2_p1 t2
-                     ->  Seq Scan on prt2_p2 t2_1
-                     ->  Seq Scan on prt2_p3 t2_2
-(16 rows)
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+(20 rows)
 
 SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.b ORDER BY t1.a;
- a  | b  
-----+----
-  0 |  0
-  6 |  6
- 12 | 12
- 18 | 18
- 24 | 24
-(5 rows)
+  a  |  b  
+-----+-----
+ -24 | -24
+ -18 | -18
+ -12 | -12
+  -6 |  -6
+   0 |   0
+   6 |   6
+  12 |  12
+  18 |  18
+  24 |  24
+(9 rows)
 
 RESET enable_hashjoin;
 RESET enable_nestloop;
---
--- partitioned by multiple columns
---
-CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
-CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
-ANALYZE prt1_m;
-CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
-CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
-ANALYZE prt2_m;
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-                                                             QUERY PLAN                                                             
-------------------------------------------------------------------------------------------------------------------------------------
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p4 t2_3
+               ->  Seq Scan on prt2_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_4
+                           Filter: (b = 0)
+(22 rows)
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p3 t1_2
+                           Filter: (b = 0)
+(21 rows)
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -150 | -0150 | 0 | -0150
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+(7 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (t1_3.a = b)
+         ->  Hash Right Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 |   | 
+ -200 | -0200 |   | 
+ -150 | -0150 | 0 | -0150
+ -100 | -0100 |   | 
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+(21 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -200 | 0 | -0200
+ -100 | 0 | -0100
+  -50 | 0 | -0050
+   50 | 0 | 0050
+  100 | 0 | 0100
+  200 | 0 | 0200
+  250 | 0 | 0250
+  350 | 0 | 0350
+  400 | 0 | 0400
+  500 | 0 | 0500
+  550 | 0 | 0550
+  650 | 0 | 0650
+  700 | 0 | 0700
+(14 rows)
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+                             QUERY PLAN                              
+---------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t3.c
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p0_a on prt1_p0 t3
+                           Index Cond: (a = t2.b)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t2.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_1.a = t2_1.b)
+                           ->  Seq Scan on prt1_p1 t3_1
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1 t2_1
+                                       Filter: (a = 0)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p2_a on prt1_p2 t3_2
+                           Index Cond: (a = t2_2.b)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t3_3
+                           Index Cond: (a = t2_3.b)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_4.a = t2_4.b)
+                           ->  Seq Scan on prt1_p4 t3_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t2_4
+                                       Filter: (a = 0)
+(47 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+  a   | a |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.b
+   ->  Hash Right Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p5 t2_5
+                           Filter: (a = 0)
+(24 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: (t1.a = t2.b)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.a, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
+                     Filter: (a = 0)
+(22 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
+               ->  Seq Scan on prt1_m_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p1
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
+               ->  Seq Scan on prt1_m_p2
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p2
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
+               ->  Seq Scan on prt1_m_p3
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p3
+                           Filter: (c = 0)
+(24 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+  a  | c |  b  | c 
+-----+---+-----+---
+   0 | 0 |   0 | 0
+  50 | 0 |     |  
+ 100 | 0 |     |  
+ 150 | 0 | 150 | 0
+ 200 | 0 |     |  
+ 250 | 0 |     |  
+ 300 | 0 | 300 | 0
+ 350 | 0 |     |  
+ 400 | 0 |     |  
+ 450 | 0 | 450 | 0
+ 500 | 0 |     |  
+ 550 | 0 |     |  
+     |   |  75 | 0
+     |   | 225 | 0
+     |   | 375 | 0
+     |   | 525 | 0
+(16 rows)
+
+--
+-- tests for list partitioned tables.
+--
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
+--
+-- list partitioned by expression
+--
+CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
+ANALYZE plt1_e;
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+                                           QUERY PLAN                                           
+------------------------------------------------------------------------------------------------
+ GroupAggregate
+   Group Key: t1.c, t2.c, t3.c
+   ->  Sort
+         Sort Key: t1.c, t3.c
+         ->  Append
+               ->  Hash Join
+                     Hash Cond: ((t1.c)::text = ltrim(t3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2.b = t1.b) AND ((t2.c)::text = (t1.c)::text))
+                           ->  Seq Scan on plt2_p4 t2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p4 t1
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p4 t3
+               ->  Hash Join
+                     Hash Cond: ((t1_1.c)::text = ltrim(t3_1.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t1_1.b = t2_1.b) AND ((t1_1.c)::text = (t2_1.c)::text))
+                           ->  Seq Scan on plt1_p1 t1_1
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p1 t2_1
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p1 t3_1
+               ->  Hash Join
+                     Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t1_2.b = t2_2.b) AND ((t1_2.c)::text = (t2_2.c)::text))
+                           ->  Seq Scan on plt1_p2 t1_2
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p2 t2_2
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p2 t3_2
+               ->  Hash Join
+                     Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2_3.b = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text))
+                           ->  Seq Scan on plt2_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p3 t3_3
+(41 rows)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p5 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_3
+                                 ->  Seq Scan on plt2_p3 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
  Sort
-   Sort Key: prt1_m_p1.a, prt2_m_p1.b
-   ->  Append
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
-               ->  Seq Scan on prt1_m_p1
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p1
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
-               ->  Seq Scan on prt1_m_p2
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p2
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
-               ->  Seq Scan on prt1_m_p3
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p3
-                           Filter: (c = 0)
-(24 rows)
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
 
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
-  50 | 0 |     |  
- 100 | 0 |     |  
- 150 | 0 | 150 | 0
- 200 | 0 |     |  
- 250 | 0 |     |  
- 300 | 0 | 300 | 0
- 350 | 0 |     |  
- 400 | 0 |     |  
- 450 | 0 | 450 | 0
- 500 | 0 |     |  
- 550 | 0 |     |  
-     |   |  75 | 0
-     |   | 225 | 0
-     |   | 375 | 0
-     |   | 525 | 0
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
 (16 rows)
 
---
--- tests for list partitioned tables.
---
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
---
--- list partitioned by expression
---
-CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1_e;
--- test partition matching with N-way join
+-- semi join
 EXPLAIN (COSTS OFF)
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                   QUERY PLAN                                   
---------------------------------------------------------------------------------
- GroupAggregate
-   Group Key: t1.c, t2.c, t3.c
-   ->  Sort
-         Sort Key: t1.c, t3.c
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
          ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.c = ltrim(t3.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1.b = t2.b) AND (t1.c = t2.c))
-                           ->  Seq Scan on plt1_p1 t1
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.c = ltrim(t3_1.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
-                           ->  Seq Scan on plt1_p2 t1_1
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.c = ltrim(t3_2.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
-                           ->  Seq Scan on plt1_p3 t1_2
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p3 t3_2
-(32 rows)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Seq Scan on plt2_p2 t2_2
+                                 ->  Seq Scan on plt2_p3 t2_3
+(22 rows)
 
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  75.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 123.0000000000000000 | 123.0000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 225.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 273.0000000000000000 | 273.0000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 375.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 423.0000000000000000 | 423.0000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 525.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 573.0000000000000000 | 573.0000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1241,22 +4040,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
 --------------------------------------------------
  Hash Left Join
    Hash Cond: (t2.b = a)
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t3.a = t2.b)
-               ->  Seq Scan on prt1_p1 t3
-               ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
-         ->  Hash Join
-               Hash Cond: (t3_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t3_1
-               ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
-         ->  Hash Join
-               Hash Cond: (t3_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t3_2
-               ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
+   ->  Hash Join
+         Hash Cond: (t3.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t3
+               ->  Seq Scan on prt1_p1 t3_1
+               ->  Seq Scan on prt1_p2 t3_2
+               ->  Seq Scan on prt1_p3 t3_3
+               ->  Seq Scan on prt1_p4 t3_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
    ->  Hash
          ->  Result
                One-Time Filter: false
@@ -1271,16 +4070,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 --
 -- tests for hash partitioned tables.
@@ -1356,41 +4161,9 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  273.0000000000000000 | 273.0000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
 (6 rows)
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
- Sort
-   Sort Key: t1.a
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
-               ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
-               ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
-                           Filter: (b = 0)
-(21 rows)
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
@@ -1405,26 +4178,24 @@ SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c
    Sort Key: t1.c
    ->  HashAggregate
          Group Key: t1.c, t2.c
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t2.c = t1.c)
-                     ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p1 t1
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on plt1_p4 t1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_1.c = t1_1.c)
-                     ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_p2 t1_1
+                           ->  Seq Scan on plt1_p1 t1_1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_2.c = t1_2.c)
-                     ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Seq Scan on plt1_p2 t1_2
                                  Filter: ((a % 25) = 0)
-(23 rows)
+                           ->  Seq Scan on plt1_p3 t1_3
+                                 Filter: ((a % 25) = 0)
+(21 rows)
 
 --
 -- multiple levels of partitioning
@@ -1826,64 +4597,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
  Hash Join
    Hash Cond: (t1.a = t2.a)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt4_n_p1 t2
                ->  Seq Scan on prt4_n_p2 t2_1
                ->  Seq Scan on prt4_n_p3 t2_2
-(11 rows)
+(13 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a and t1.a = t3.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                        QUERY PLAN                        
+----------------------------------------------------------
  Hash Join
-   Hash Cond: (t2.a = t1.a)
+   Hash Cond: (t3.b = t1.a)
    ->  Append
-         ->  Seq Scan on prt4_n_p1 t2
-         ->  Seq Scan on prt4_n_p2 t2_1
-         ->  Seq Scan on prt4_n_p3 t2_2
+         ->  Seq Scan on prt2_p0 t3
+         ->  Seq Scan on prt2_p1 t3_1
+         ->  Seq Scan on prt2_p2 t3_2
+         ->  Seq Scan on prt2_p3 t3_3
+         ->  Seq Scan on prt2_p4 t3_4
+         ->  Seq Scan on prt2_p5 t3_5
    ->  Hash
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.a = t3_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_2.b)
-                     ->  Seq Scan on prt1_p3 t1_2
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3 t3_2
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt4_n_p1 t2
+                           ->  Seq Scan on prt4_n_p2 t2_1
+                           ->  Seq Scan on prt4_n_p3 t2_2
 (23 rows)
 
 -- partitionwise join can not be applied if there are no equi-join conditions
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (t1.a < b)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (t1.a < b)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (t1.a < b)
-(12 rows)
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+(16 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partitionwise join
@@ -1969,16 +4746,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
-                     ->  Seq Scan on plt1_p1 t3
-                     ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3
+                     ->  Seq Scan on plt1_p1 t3_1
+                     ->  Seq Scan on plt1_p2 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partitionwise join can not be applied for a join between list and range
 -- partitioned table
@@ -1989,14 +4767,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
  Hash Full Join
    Hash Cond: ((t2.c)::text = (t1.c)::text)
    ->  Append
-         ->  Seq Scan on prt1_p1 t2
-         ->  Seq Scan on prt1_p2 t2_1
-         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p0 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p2 t2_2
+         ->  Seq Scan on prt1_p3 t2_3
+         ->  Seq Scan on prt1_p4 t2_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt1_n_p1 t1
                ->  Seq Scan on prt1_n_p2 t1_1
-(10 rows)
+(12 rows)
 
 -- partitionwise join can not be applied if only one of joining table has
 -- default partition
@@ -2012,16 +4792,23 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
    ->  Hash Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index c1c9859651..0e884835fb 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -10,25 +10,39 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -67,11 +81,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -93,20 +115,30 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 
 EXPLAIN (COSTS OFF)
@@ -169,6 +201,128 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -193,28 +347,79 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
 -- test partition matching with N-way join
@@ -222,6 +427,175 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
@@ -267,27 +641,18 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
 ALTER TABLE plt2 DETACH PARTITION plt2_p3;
 ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
 ANALYZE plt2;
-
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.a % 25 = 0 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
 --
 -- multiple levels of partitioning
 --
-- 
2.19.2

#61Etsuro Fujita
fujita.etsuro@lab.ntt.co.jp
In reply to: Etsuro Fujita (#60)
3 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

(2019/01/22 21:38), Etsuro Fujita wrote:

Will continue to review.

I rebased the patch set against the latest HEAD. Attached is a new
version. I'll move this to the next CF, and continue to review it.

Best regards,
Etsuro Fujita

Attachments:

0001-Hash-partition-bound-equality-refactoring-v17.patchtext/x-patch; name=0001-Hash-partition-bound-equality-refactoring-v17.patchDownload
From d4333dec0a3ce91dbf007adb3d811fbb02b1ef2a Mon Sep 17 00:00:00 2001
From: Etsuro Fujita <efujita@postgresql.org>
Date: Thu, 31 Jan 2019 19:18:18 +0900
Subject: [PATCH 1/3] Hash partition bound equality refactoring.

Separate the code to check whether two given hash bounds are equal into a
separate function, so that it can be called from multiple places. Right now
it's only caller is partition_bounds_equal() but later we will use it for
merging partition bounds.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/partitioning/partbounds.c | 95 +++++++++++++++++----------
 1 file changed, 61 insertions(+), 34 deletions(-)

diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index d478ae7e19..c492685d11 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -104,6 +104,9 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 						 Expr **keyCol,
 						 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static bool partition_hbounds_equal(PartitionBoundInfo b1,
+									PartitionBoundInfo b2);
+
 
 /*
  * get_qual_from_partbound
@@ -652,6 +655,63 @@ create_range_bounds(PartitionBoundSpec **boundspecs, int nparts,
 	return boundinfo;
 }
 
+/*
+ * Are two hash partition bound collections logically equal?
+ *
+ * Hash partition bounds store modulus and remainder in datums array which are
+ * always integers irrespective of the number of partition keys and their data
+ * types. Hence we can compare the hash bound collection without any partition
+ * key specific information. Separating this logic in a function which does not
+ * require partition key specific information allows it be called from places
+ * where the partition key specific information is not completely available.
+ */
+static bool
+partition_hbounds_equal(PartitionBoundInfo b1, PartitionBoundInfo b2)
+{
+	int			greatest_modulus = get_hash_partition_greatest_modulus(b1);
+	int			i;
+
+	Assert(b1->strategy == PARTITION_STRATEGY_HASH &&
+		   b2->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * If two hash partitioned tables have different greatest moduli,
+	 * their partition schemes don't match.  For hash partitioned table,
+	 * the greatest modulus is given by the last datum and number of
+	 * partitions is given by ndatums.
+	 */
+	if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
+		return false;
+
+	/*
+	 * We arrange the partitions in the ascending order of their modulus and
+	 * remainders.  Also every modulus is factor of next larger modulus.
+	 * Therefore we can safely store index of a given partition in indexes
+	 * array at remainder of that partition.  Also entries at (remainder + N *
+	 * modulus) positions in indexes array are all same for (modulus,
+	 * remainder) specification for any partition.  Thus datums array from both
+	 * the given bounds are same, if and only if their indexes array will be
+	 * same.  So, it suffices to compare indexes array.
+	 */
+	for (i = 0; i < greatest_modulus; i++)
+		if (b1->indexes[i] != b2->indexes[i])
+			return false;
+
+#ifdef USE_ASSERT_CHECKING
+
+	/*
+	 * Nonetheless make sure that the bounds are indeed same when the indexes
+	 * match.  Hash partition bound stores modulus and remainder at
+	 * b1->datums[i][0] and b1->datums[i][1] position respectively.
+	 */
+	for (i = 0; i < b1->ndatums; i++)
+		Assert((b1->datums[i][0] == b2->datums[i][0] &&
+				b1->datums[i][1] == b2->datums[i][1]));
+#endif
+
+	return true;
+}
+
 /*
  * Are two partition bound collections logically equal?
  *
@@ -680,41 +740,8 @@ partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
 
 	if (b1->strategy == PARTITION_STRATEGY_HASH)
 	{
-		int			greatest_modulus = get_hash_partition_greatest_modulus(b1);
-
-		/*
-		 * If two hash partitioned tables have different greatest moduli,
-		 * their partition schemes don't match.
-		 */
-		if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
+		if (!partition_hbounds_equal(b1, b2))
 			return false;
-
-		/*
-		 * We arrange the partitions in the ascending order of their moduli
-		 * and remainders.  Also every modulus is factor of next larger
-		 * modulus.  Therefore we can safely store index of a given partition
-		 * in indexes array at remainder of that partition.  Also entries at
-		 * (remainder + N * modulus) positions in indexes array are all same
-		 * for (modulus, remainder) specification for any partition.  Thus
-		 * datums array from both the given bounds are same, if and only if
-		 * their indexes array will be same.  So, it suffices to compare
-		 * indexes array.
-		 */
-		for (i = 0; i < greatest_modulus; i++)
-			if (b1->indexes[i] != b2->indexes[i])
-				return false;
-
-#ifdef USE_ASSERT_CHECKING
-
-		/*
-		 * Nonetheless make sure that the bounds are indeed same when the
-		 * indexes match.  Hash partition bound stores modulus and remainder
-		 * at b1->datums[i][0] and b1->datums[i][1] position respectively.
-		 */
-		for (i = 0; i < b1->ndatums; i++)
-			Assert((b1->datums[i][0] == b2->datums[i][0] &&
-					b1->datums[i][1] == b2->datums[i][1]));
-#endif
 	}
 	else
 	{
-- 
2.19.2

0002-Partition-wise-join-for-1-1-1-0-0-1-partition-match-v17.patchtext/x-patch; name=0002-Partition-wise-join-for-1-1-1-0-0-1-partition-match-v17.patchDownload
From a5290bd315d8009ef46d22e170c0ccd5b9dbf968 Mon Sep 17 00:00:00 2001
From: Etsuro Fujita <efujita@postgresql.org>
Date: Thu, 31 Jan 2019 20:26:21 +0900
Subject: [PATCH 2/3] Partition-wise join for 1:1, 1:0, 0:1 partition
 matching.

Earlier version of partition-wise join implementation allowed
partition-wise join between two relations with exactly same partition
bounds. This commit allows partition-wise join to be applied under
following conditions

1. the partition bounds of joining relations are such that rows from
given partition on one side join can join with rows from maximum one
partition on the other side i.e. bounds of a given partition on one
side match/overlap with those of maximum one partition on the other
side. If the mapping happens to be m:n where m > 1 or n > 1, we have
to gang multiple partition relations together into a single relation.
This means that we have to add simple relations during join
processing, something which is not supported right now.  ALso, in such
a case, different pairs of joining relations can produce different
partition bounds for the same join relation, which again is not
supported right now.

2. For every partition on outer side that can contribute to the result
of an OUTER join, there exists at least one (taken along with item 1,
it means exactly one)  matching partition on the inner side. To
support partition-wise join when the inner matching partition doesn't
exist, we have to add a dummy base relation corresponding to the
non-existent inner partition. We don't have support to add base
relations during join processing.

3. For hash partitioned tables however, we support partition-wise join
only when the bounds exactly match. For hash partitioning it's unusual
to have missing partitions and hence generic partition matching is not
required.

With advanced partition matching, we won't be able to apply
partition-wise join when there are missing matching partitions. So for
a given N-way join, some sub-joins may not be partitioned, while the
overall N-way join is partitioned. We can not try partition-wise join
when one or both of the joining relations are not partitioned in one
of the pairs of joining relation. Skip partition-wise join for that
pair.

An example is A IJ B LJ C where B has an extra partition compared to A
and C. Join B LJ C requires dummy relation to join the extra partition
from B to a missing partition from C, and hence can not use
partition-wise join right now and hence BC is not partitioned.  The
extra partition in B gets eliminated in A IJ B and thus it can use
partition-wise join and is partitioned. Hence (AB)C can use
partition-wise join but not A(BC).

Not every pair of joining relation (including the one presented to
build_joinrel_partition_info()) for the same joinrel can use
partition-wise join or has both the relations partitioned. Hence we
calculate the partition bounds for the join relation in
try_partition_wise_join() instead of doing it here.

The partition bounds produced for the first pair that can use
partition-wise are saved in the RelOptInfo of the joinrel. Partition
bounds produced for subsequent pairs should match that produced by the
first pair.

Support for default partition in list partitioned tables
========================================================

When a list value is present in one of the joining relations and not
the other, and the other relation has default partition, match (join)
the partition containing that list value with the default partition.
If there are multiple matches, we can not proceed with the
partition-wise join similar to the case of matching non-default
partition. If the default partition happens to be on the outer side of
the join, the resulting join partition acts as a default partition as
it will contain all the values from the default partition. If
the partition containing the list value happens to be on the outer
side of the join, the resulting join partition is associated with the
list value, since no other partition key value from the default
partition makes it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join.  Such a
default partition may get joined to a partition from the inner side,
if inner side has a list value that is not present in the outer side.
But if such a join partner does not exist, we won't be able to use
partition-wise join since there is no (dummy) relation on the inner
side to join to.

Support for matching default partition in range partitioned tables
==================================================================

When a range is present in one of the joining relations and not the
other, and the other relation has default partition, match (join) the
partition corresponding range with the default partition. If two
ranges overlap but their ranges don't match exactly, the
non-overlapping portion from one range may join the previous
(non-overlapping portion near the lower bound, if exists), next
(non-overlapping portion near the upper bound, if exists) ranges from
the other relation or default partition from the other relation. This
causes multiple partitions on the other side to be joined with a
single partition on the first side; a case we don't support. Any range
from one side which doesn't overlap with any range on the other side,
may find its join partner in the default partition and the
corresponding range partition will need to be joined with the default
partition.  If the default partition happens to be on the outer side
of the join, the resulting join partition acts as a default partition
as it will contain all the values from the default partition. If the
non-partition corresponding to the range happens to be on the outer
side of the join, the resulting join partition is associated with that
range, since partition key values from the default partition outside
that range won't make it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join.  Such a
default partition may get joined to a non-default partition from the
inner side, if inner side has a range missing in the outer side.  But
if such a join partner does not exist, we won't be able to use
partition-wise join since there is no (dummy) relation on the inner
side to join to.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/optimizer/path/joinrels.c |   95 +-
 src/backend/optimizer/util/relnode.c  |   33 +-
 src/backend/partitioning/partbounds.c | 1677 ++++++++++++++++++++++++-
 src/include/partitioning/partbounds.h |    7 +
 4 files changed, 1766 insertions(+), 46 deletions(-)

diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index dfbbfdac6d..ca4d261bc1 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1320,25 +1320,31 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	PartitionScheme part_scheme;
+	PartitionBoundInfo join_boundinfo;
+	List	   *parts1;
+	List	   *parts2;
+	ListCell   *lc1;
+	ListCell   *lc2;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1351,32 +1357,63 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	 */
 	Assert(joinrel->part_scheme == rel1->part_scheme &&
 		   joinrel->part_scheme == rel2->part_scheme);
+	part_scheme = joinrel->part_scheme;
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * Get the list of matching partitions to be joined along with the
+	 * partition bounds of the join relation. Because of the restrictions
+	 * imposed by partition matching algorithm, not every pair of joining
+	 * relations for this join will be able to use partition-wise join. But all
+	 * those pairs which can use partition-wise join will produce the same
+	 * partition bounds for the join relation.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	join_boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											part_scheme->partsupfunc,
+											part_scheme->partcollation,
+											rel1, rel2,
+											parent_sjinfo->jointype,
+											&parts1, &parts2);
+
+	if (join_boundinfo == NULL)
+		return;
 
-	nparts = joinrel->nparts;
+	if (joinrel->boundinfo == NULL)
+	{
+		Assert(joinrel->nparts == 0 && joinrel->part_rels == NULL);
+		joinrel->boundinfo = join_boundinfo;
+		joinrel->nparts = list_length(parts1);
+		Assert(joinrel->nparts == list_length(parts2));
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) *
+									joinrel->nparts);
+	}
+	else
+	{
+		Assert(partition_bounds_equal(part_scheme->partnatts,
+									  part_scheme->parttyplen,
+									  part_scheme->parttypbyval,
+									  join_boundinfo, joinrel->boundinfo));
+		/*
+		 * Every pair of joining relations should result in the same number
+		 * of child-joins.
+		 */
+		Assert(joinrel->nparts == list_length(parts1));
+		Assert(joinrel->nparts == list_length(parts2));
+		Assert(joinrel->part_rels);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	cnt_parts = 0;
+	forboth(lc1, parts1, lc2, parts2)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
+		int			part1 = lfirst_int(lc1);
+		int			part2 = lfirst_int(lc2);
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1384,6 +1421,10 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		Assert(part1 >= 0 && part2 >= 0);
+		child_rel1 = rel1->part_rels[part1];
+		child_rel2 = rel2->part_rels[part2];
+
 		/*
 		 * If a child table has consider_partitionwise_join=false, it means
 		 * that it's a dummy relation for which we skipped setting up tlist
@@ -1438,12 +1479,24 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 			joinrel->part_rels[cnt_parts] = child_joinrel;
 		}
 
+		/*
+		 * For every pair of joining relations, the set of matching partitions
+		 * would change. However, the base relation partitions constituting
+		 * the given child should remain same for all the joining pairs. Since
+		 * the order in which children are stored in the array of child-joins,
+		 * depends upon partition bounds of the join, which are same for all
+		 * the joining pairs, every joining pair yields the child-joins in the
+		 * same order.
+		 */
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
 
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
+		cnt_parts++;
 	}
+
+	Assert(cnt_parts == joinrel->nparts);
 }
 
 /*
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index f04c6b76f4..a1561dd066 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1585,7 +1585,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1598,24 +1598,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1627,17 +1609,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index c492685d11..38e35f33fc 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -22,6 +22,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/pathnodes.h"
 #include "parser/parse_coerce.h"
 #include "partitioning/partprune.h"
 #include "partitioning/partbounds.h"
@@ -65,6 +66,12 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+typedef struct PartitionMap
+{
+	int from;
+	int to;
+} PartitionMap;
+
 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);
@@ -106,7 +113,58 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 static List *get_range_nulltest(PartitionKey key);
 static bool partition_hbounds_equal(PartitionBoundInfo b1,
 									PartitionBoundInfo b2);
-
+static PartitionBoundInfo partition_range_bounds_merge(
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 List **outer_parts, List **inner_parts,
+							 JoinType jointype, int partnatts,
+							 FmgrInfo *supfuncs, Oid *collations);
+static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype);
+static PartitionBoundInfo partition_hash_bounds_merge(RelOptInfo *outer_rel,
+							RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype);
+static void generate_matching_part_pairs(PartitionMap *outer_maps,
+										 PartitionMap *inner_maps,
+										 int nparts1, int nparts2,
+										 JoinType jointype, int nparts,
+										 List **parts1, List **parts2);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int map_and_merge_partitions(PartitionMap *outer_maps,
+										PartitionMap *inner_maps,
+										int index1, int index2, int *next_index);
+static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *collations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2);
+static bool partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
+						   Oid *collations, PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, int *ub_cmpval,
+						   int *lb_cmpval);
+static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes);
+static bool merge_default_partitions(PartitionBoundInfo outer_bi,
+						 			 PartitionBoundInfo inner_bi,
+						 			 PartitionMap *outer_maps,
+						 			 PartitionMap *inner_maps,
+						 			 JoinType jointype,
+									 int *next_index, int *default_index);
+static bool merge_null_partitions(PartitionBoundInfo outer_bi,
+								  PartitionBoundInfo inner_bi,
+								  PartitionMap *outer_maps,
+								  PartitionMap *inner_maps,
+								  JoinType jointype,
+								  int *next_index, int *null_index,
+								  int *default_index);
 
 /*
  * get_qual_from_partbound
@@ -2941,3 +2999,1620 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * The function produces the partition bounds for a join between two relations
+ * whose partition bounds are given. The function also returns two lists of
+ * partition indexes one for each of the joining relations. Both the lists
+ * contain the same number of elements. The partition indexes at the same
+ * positions in the lists indicate the pair partitions, one from each side, to
+ * be joined and the position itself corresponds to the index of partition
+ * produced by that child-join in the partitioned join.
+ *
+ * The function returns NULL if we can not find the matching pair of
+ * partitions. This happens if 1. multiple partitions on one side match with
+ * one partition on the other side. 2. a given partition on the outer side
+ * doesn't have a matching partition on the inner side. We can not support the
+ * first case since we don't have a way to represent multiple partitions as a
+ * single relation (RelOptInfo) and then perform join using the ganged
+ * relation. We can not support the second case since the missing inner
+ * partition needs to be represented as an empty relation and we don't have a
+ * way to introduce empty relation during join planning after creating paths
+ * for all the base relations.
+ */
+extern PartitionBoundInfo
+partition_bounds_merge(int partnatts, FmgrInfo *partsupfunc,
+					   Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
+{
+	PartitionBoundInfo 	merged_bounds;
+	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
+					   	inner_binfo = inner_rel->boundinfo;
+	char				strategy = outer_binfo->strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partsupfunc,
+														partcollation,
+														outer_rel, inner_rel,
+														outer_parts, inner_parts,
+														jointype);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(outer_rel, inner_rel,
+														 outer_parts, inner_parts,
+														 jointype, partnatts,
+														 partsupfunc,
+														 partcollation);
+			break;
+
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = partition_hash_bounds_merge(outer_rel, inner_rel,
+														outer_parts, inner_parts,
+														jointype);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	Assert((*outer_parts == NIL || *inner_parts != NIL) &&
+		   (*inner_parts == NIL || *outer_parts != NIL));
+
+	return merged_bounds;
+}
+
+/*
+ * partition_get_range_bounds
+ *
+ * Given the index of lower bound in datums array, return lower and upper
+ * bounds and the index of the partition with that lower bound.
+ */
+static int
+partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
+						   PartitionRangeBound *lower,
+						   PartitionRangeBound *upper)
+{
+	int			part_index;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The lower bound should correspond to a valid partition. */
+	part_index = bi->indexes[lb_index + 1];
+	Assert(part_index >= 0);
+
+	lower->kind = bi->kind[lb_index];
+	lower->datums = bi->datums[lb_index];
+	lower->lower = true;
+	upper->kind = bi->kind[lb_index + 1];
+	upper->datums = bi->datums[lb_index + 1];
+	upper->lower = false;
+
+	return part_index;
+}
+
+/*
+ * partition_range_get_next_lb_index
+ *
+ * Given the index of lower bound in datums array return the
+ * index of lower bound of the next partition. When the given index corresponds
+ * to the last partition, return number of datums (ndatums).
+ */
+static int
+partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
+{
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The partition index corresponding to the upper bound should be valid. */
+	Assert(bi->indexes[lb_index + 1] >= 0);
+
+	/*
+	 * If there are no bounds left beyond the upper bound, we have reached the
+	 * last partition.
+	 */
+	if (lb_index + 2 < bi->ndatums)
+	{
+		/*
+		 * If the bound next to the upper bound corresponds to no partition,
+		 * that's the next lower bound of the next partition. Otherwise, the
+		 * current upper bound is the lower bound of the next partition.
+		 */
+		if (bi->indexes[lb_index + 2] < 0)
+			return lb_index + 2;
+		else
+			return lb_index + 1;
+	}
+	else
+		return bi->ndatums;
+}
+
+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *partcollations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2)
+{
+	return partition_rbound_cmp(partnatts, partsupfunc, partcollations,
+								bound1->datums, bound1->kind, bound1->lower,
+								bound2);
+}
+
+/*
+ * partition_range_cmp
+ *
+ * Compare the bounds of two range partitions. Set ub_cmpval <, = or > 0, if the
+ * first partition's upper bound is lower than, equal to or higher than the
+ * second partition's upper bound resp. Similarly set lb_cmpval <, =  or > 0,
+ * if the first partition's lower bound is lower than, equal to or higher than
+ * the second partition's lower bound resp.
+ *
+ * Return true, if the ranges overlap, otherwise return false.
+ */
+static bool
+partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
+					PartitionRangeBound *lower_bound1,
+					PartitionRangeBound *upper_bound1,
+					PartitionRangeBound *lower_bound2,
+					PartitionRangeBound *upper_bound2, int *ub_cmpval,
+					int *lb_cmpval)
+{
+	bool		overlap;
+
+	/*
+	 * Compare upper bound of the first partition with the lower bound of the
+	 * second and vice-versa. If lower bound is higher than the upper bound,
+	 * the partitions are not overlapping. All other cases indicate overlapping
+	 * partitions.
+	 */
+	if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+								  lower_bound1, upper_bound2) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = 1;
+		*lb_cmpval = 1;
+	}
+	else if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+									   lower_bound2, upper_bound1) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = -1;
+		*lb_cmpval = -1;
+	}
+	else
+	{
+		overlap = true;
+		*ub_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, upper_bound1,
+											   upper_bound2);
+		*lb_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, lower_bound1,
+											   lower_bound2);
+	}
+
+	return overlap;
+}
+
+/*
+ * partition_range_merge
+ *
+ * Merge the partition bounds of given two partitions such that the join
+ * between the given two partitions fits merged bounds.
+ *
+ * "merged_upper" will be set to one of the given upper bounds and
+ * "merged_lower" will be set to one of the given lower bounds.
+ */
+static void
+partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
+					  Oid *partcollations, JoinType jointype,
+					  PartitionRangeBound *left_lb,
+					  PartitionRangeBound *left_ub,
+					  PartitionRangeBound *right_lb,
+					  PartitionRangeBound *right_ub,
+					  PartitionRangeBound **merged_lb,
+					  PartitionRangeBound **merged_ub)
+{
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = left_ub;
+			*merged_lb = left_lb;
+			break;
+
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) < 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) > 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		case JOIN_FULL:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) > 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) < 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+
+	return;
+}
+
+/*
+ * Add the lower bound of the next range to the list of bounds, if the lower
+ * bound is higher or equal to the previous upper bound. If successful return
+ * true, otherwise false.
+ */
+static bool
+partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
+							  Oid *partcollations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		prev_ub.datums = llast(*merged_datums);
+		prev_ub.kind = llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  next_lb_datums, next_lb_kind, false,
+									  &prev_ub);
+	}
+
+	/*
+	 * The lower bound is lower than the last upper bound, thus does not fit
+	 * the bounds created so far and hence can not be merged with the existing
+	 * bounds.
+	 */
+	if (cmpval < 0)
+		return false;
+
+	/*
+	 * Add bounds of the new merged partition. If the next lower bound is
+	 * higher than the last upper bound, add new range with index
+	 * corresponding to the lower bound as -1. If the merged lower bound
+	 * is same as the last merged upper bound, the last upper bound will be
+	 * reused as the lower bound of the next range.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, next_lb_datums);
+		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	return true;
+}
+
+/*
+ * handle_missing_partition
+ *
+ * If a range appears in one of the joining relations but not the other, a row
+ * in the corresponding partition will not have any join partner in the other
+ * relation, unless the other relation has a default partition. If a given list
+ * value is present in one joining relation but not the other, the default
+ * partition on the other side may contain that value.
+ *
+ * In both these cases, such an extra partition forms a joining pair with the
+ * default partition, if any,  on the other side.
+ *
+ * If the default partition happens to be on the outer side of the join, the
+ * resultant partition will act as the default partition of the join relation.
+ * Otherwise the resultant partition will be associated with the range.
+ *
+ * When the default partition is not present in the other relation, the rows in
+ * the extra partition will be included in the bounds of the join result, if it
+ * appears on the outer side of the join, since all rows from the outer side
+ * are included in the join result.
+ *
+ * This function handles all these cases.
+ *
+ * maps_with_missing and missing_side_default are the partition maps (See
+ * partition_range/list_bounds_merge() for details) and the index of default
+ * partition respectively corresponding the side with missing partition.
+ *
+ * maps_with_extra and extra_part are the partition maps (See
+ * partition_range/list_bounds_merge() for details) and the index of extra
+ * partition respectively corresponding to the side with the extra partition.
+ *
+ * It returns true if the matching succeeds, otherwise returns false.
+ */
+static bool
+handle_missing_partition(PartitionMap *maps_with_missing,
+						 PartitionMap *maps_with_extra,
+						 int missing_side_default,
+						 int extra_part,
+						 bool missing_side_outer,
+						 bool missing_side_inner,
+						 int *next_index, int *default_index,
+						 int *merged_index)
+{
+	bool missing_has_default = (missing_side_default != -1);
+
+	if (missing_has_default)
+	{
+		*merged_index = map_and_merge_partitions(maps_with_missing,
+												 maps_with_extra,
+												 missing_side_default,
+												 extra_part,
+												 next_index);
+		if (*merged_index < 0)
+			return false;
+
+		if (missing_side_outer)
+		{
+			/*
+			 * Default partition on the outer side forms the default
+			 * partition of the join result.
+			 */
+			if (*default_index < 0)
+				*default_index = *merged_index;
+			else if(*default_index != *merged_index)
+			{
+				/*
+				 * Ended up with default partition on the outer side
+				 * being joined with multiple partitions on the inner
+				 * side. We don't support this case.
+				 */
+				return false;
+			}
+
+			/*
+			 * Since the merged partition acts as a default partition, it
+			 * doesn't need a separate index.
+			 */
+			*merged_index = -1;
+		}
+	}
+	else if (missing_side_inner)
+	{
+		/*
+		 * If this partition has already been mapped (say because we
+		 * found an overlapping range earlier), we know where does it
+		 * fit in the join result. Nothing to do in that case. Else
+		 * create a new merged partition.
+		 */
+		PartitionMap *extra_map = &maps_with_extra[extra_part];
+		if (extra_map->to < 0)
+		{
+			extra_map->to = *next_index;
+			*next_index = *next_index + 1;
+			*merged_index = extra_map->to;
+		}
+	}
+
+	return true;
+}
+
+static PartitionMap*
+init_partition_map(RelOptInfo *rel)
+{
+	int i, nparts = rel->nparts;
+	PartitionMap *map;
+
+	map = (PartitionMap *) palloc(sizeof(PartitionMap) * nparts);
+
+	for (i = 0; i < nparts; i++)
+	{
+		map[i].from = -1;
+		map[i].to = -1;
+	}
+
+	return map;
+}
+
+/*
+ * Allocate and initialize partition maps. We maintain four maps, two maps
+ * for each joining relation. pmap[i] gives the partition from the other
+ * relation which would join with ith partition of the given relation.
+ * Partition i from the given relation will join with partition pmap[i]
+ * from the other relation to produce partition mmap[i] of the join (merged
+ * partition).
+ *
+ * pmap[i] = -1 indicates that ith partition of a given relation does not
+ * have a matching partition from the other relation.
+ *
+ * mmap[i] = -1 indicates that ith partition of a given relation does not
+ * contribute to the join result. That can happen only when the given
+ * relation is the inner relation and it doesn't have a matching partition
+ * from the outer relation, hence pmap[i] should be -1.
+ *
+ * In case of an outer join, every partition of the outer join will appear
+ * in the join result, and thus has mmap[i] set for all i. But it's not
+ * necessary that every partition on the outer side will have a matching
+ * partition on the inner side. In such a case, we end up with pmap[i] = -1
+ * and mmap[i] != -1.
+ */
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 List **outer_parts, List **inner_parts,
+							 JoinType jointype, int partnatts,
+							 FmgrInfo *partsupfuncs, Oid *partcollations)
+
+{
+	PartitionMap *outer_maps = NULL;
+	PartitionMap *inner_maps = NULL;
+	int			outer_part = 0;
+	int			inner_part = 0;
+	PartitionBoundInfo merged_bounds = NULL;
+	int			outer_lb_index;
+	int			inner_lb_index;
+	int			next_index;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	List	   *merged_kinds = NIL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			inner_default = inner_bi->default_index;
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	outer_maps = init_partition_map(outer_rel);
+	inner_maps = init_partition_map(inner_rel);
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = 0;
+	inner_lb_index = 0;
+	next_index = 0;
+	while (outer_lb_index < outer_bi->ndatums ||
+		   inner_lb_index < inner_bi->ndatums)
+	{
+		PartitionRangeBound outer_lb, outer_ub,
+							inner_lb, inner_ub,
+							*merged_lb = NULL,
+							*merged_ub = NULL;
+
+		int			merged_index = -1;
+		bool		overlap;
+		bool		finished_outer = false;
+		bool		finished_inner = false;
+
+		/* Result of bounds comparison per partition_rbound_cmp(). */
+		int			ub_cmpval;	/* Upper bounds comparison result. */
+		int			lb_cmpval;	/* Lower bounds comparison result. */
+
+		/* Get the range bounds of the next pair of partitions. */
+		if (outer_lb_index < outer_bi->ndatums)
+			outer_part = partition_get_range_bounds(outer_bi, outer_lb_index,
+												&outer_lb, &outer_ub);
+		else
+			finished_outer = true;
+
+		if (inner_lb_index < inner_bi->ndatums)
+			inner_part = partition_get_range_bounds(inner_bi, inner_lb_index,
+												&inner_lb, &inner_ub);
+		else
+			finished_inner = true;
+
+		Assert(!finished_outer || !finished_inner);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (finished_outer)
+		{
+			overlap = false;
+			ub_cmpval = 1;
+			lb_cmpval = 1;
+		}
+		else if (finished_inner)
+		{
+			overlap = false;
+			ub_cmpval = -1;
+			lb_cmpval = -1;
+		}
+		else
+			overlap = partition_range_cmp(partnatts, partsupfuncs, partcollations,
+										  &outer_lb, &outer_ub, &inner_lb,
+										  &inner_ub, &ub_cmpval, &lb_cmpval);
+
+		if (overlap)
+		{
+			/*
+			 * The rows from overlapping portion of ranges on both sides may
+			 * join, hence the corresponding pair of partitions form a joining
+			 * pair. Match them and produce the bounds of the joint partition
+			 * and its index by merging the bounds according to the type of
+			 * join.
+			 */
+			partition_range_merge(partnatts, partsupfuncs, partcollations,
+								  jointype, &outer_lb, &outer_ub, &inner_lb,
+								  &inner_ub, &merged_lb, &merged_ub);
+
+			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+													outer_part, inner_part,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				/* Failed to match the partitions. */
+				return NULL;
+			}
+
+			/*
+			 * If the ranges overlap but don't exactly match, a row from
+			 * non-overlapping portion of the range from one side of join may
+			 * find its join partner in the previous or next overlapping
+			 * partition or default partition on the other side , if such a
+			 * partition exists. All those cases, if true, will cause one
+			 * partition from that side to match at least two partitions on the
+			 * other side; a case that we do not support now. Previous
+			 * partition has been delt with in the previous iteration of this
+			 * loop, next partition will be delt in the next iteration. We will
+			 * deal with the default partition here.
+			 */
+			if ((lb_cmpval < 0 && inner_has_default) ||
+				/* Non-overlapping range on the lower side of outer range. */
+				(lb_cmpval > 0 && outer_has_default) ||
+				/* Non-overlapping range on the lower side of inner range. */
+				(ub_cmpval < 0 && outer_has_default) ||
+				/* Non-overlapping range on the upper side of inner range. */
+				(ub_cmpval > 0 && inner_has_default))
+				/* Non-overlapping range on the upper side of outer range. */
+				return NULL;
+		}
+
+		if (ub_cmpval == 0)
+		{
+			/* Upper bounds of both the ranges match. */
+			Assert(overlap);
+
+			/* Move to the next pair of partitions. */
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the inner side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &outer_lb;
+				merged_ub = &outer_ub;
+
+				/*
+				 * For a FULL join, inner relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the inner relation acts as
+				 * INNER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_inner = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_outer = (jointype == JOIN_FULL);
+				if (!handle_missing_partition(inner_maps,
+											  outer_maps,
+											  inner_default,
+											  outer_part,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the outer side. */
+			Assert(!finished_outer);
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+		}
+		else
+		{
+			Assert(ub_cmpval > 0);
+
+			/* Upper bound of outer range higher than that of the inner. */
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the outer side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &inner_lb;
+				merged_ub = &inner_ub;
+
+				/*
+				 * For a FULL join, outer relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the outer relation acts as
+				 * OUTER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_outer = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_inner = (jointype == JOIN_FULL);
+
+				if (!handle_missing_partition(outer_maps,
+											  inner_maps,
+											  outer_default,
+											  inner_part,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the inner side. */
+			Assert (!finished_inner);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+
+		if (merged_index < 0)
+		{
+			/* We didn't find a new merged partition. */
+			continue;
+		}
+
+		/*
+		 * We have a valid partition index for the next partition of join. The
+		 * partition should have valid range.
+		 */
+		Assert(merged_lb && merged_ub);
+
+		/* Try merging new lower bound with the last upper bound. */
+		if (!partition_range_merge_next_lb(partnatts, partsupfuncs,
+										   partcollations,
+										   merged_lb->datums,
+										   merged_lb->kind, &merged_datums,
+										   &merged_kinds, &merged_indexes))
+			return NULL;
+
+		/* Add upper bound with the merged partition index. */
+		merged_datums = lappend(merged_datums, merged_ub->datums);
+		merged_kinds = lappend(merged_kinds, merged_ub->kind);
+		merged_indexes = lappend_int(merged_indexes, merged_index);
+	}
+
+	if (!merge_default_partitions(outer_bi, inner_bi,
+										  outer_maps, inner_maps,
+										  jointype, &next_index,
+										  &default_index))
+		return NULL;
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_maps, inner_maps,
+								 outer_nparts, inner_nparts,
+								 jointype, next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes,
+													  merged_kinds,
+													  -1, default_index);
+	}
+
+	/* Free any memory we used in this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype)
+{
+	PartitionMap *outer_maps = NULL;
+	PartitionMap *inner_maps = NULL;
+	int			cnto;
+	int			cnti;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			next_index = 0;
+	int			null_index;
+	int			default_index = -1;
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			      *outer_indexes = outer_bi->indexes;
+	int			      *inner_indexes = inner_bi->indexes;
+	int				   outer_default = outer_bi->default_index;
+	int				   inner_default = inner_bi->default_index;
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	outer_maps = init_partition_map(outer_rel);
+	inner_maps = init_partition_map(inner_rel);
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			o_index;
+		int			i_index;
+		int			cmpval;
+		int			merged_index = -1;
+		Datum	   *merged_datum;
+		bool		finished_inner;
+		bool		finished_outer;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+		{
+			finished_outer = true;
+			odatums = NULL;
+			o_index = -1;
+		}
+		else
+		{
+			finished_outer = false;
+			odatums = outer_bi->datums[cnto];
+			o_index = outer_indexes[cnto];
+		}
+
+		if (cnti >= inner_bi->ndatums)
+		{
+			finished_inner = true;
+			idatums = NULL;
+			i_index = -1;
+		}
+		else
+		{
+			finished_inner = false;
+			idatums = inner_bi->datums[cnti];
+			i_index = inner_indexes[cnti];
+		}
+
+		/* If we exhausted both the sides, we won't enter the loop. */
+		Assert(!finished_inner || !finished_outer);
+
+		if (finished_outer)
+			cmpval = 1;
+		else if (finished_inner)
+			cmpval = -1;
+		else
+		{
+			/* Every list datum should map to a valid partition index. */
+			Assert(o_index >= 0 && i_index >= 0 &&
+				   odatums != NULL && idatums != NULL);
+
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0], idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+													o_index, i_index,
+													&next_index);
+
+			if (merged_index < 0)
+				return NULL;
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			/* A datum missing from the inner side. */
+			merged_index = -1;
+			merged_datum = odatums;
+
+			/*
+			 * For a FULL join, inner relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the inner relation acts as
+			 * INNER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_inner = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_outer = (jointype == JOIN_FULL);
+
+			if (!handle_missing_partition(inner_maps,
+										  outer_maps,
+										  inner_default,
+										  o_index,
+										  missing_side_outer,
+										  missing_side_inner,
+										  &next_index,
+										  &default_index,
+										  &merged_index))
+				return NULL;
+
+			/* Move to the next datum on the outer side. */
+			Assert(!finished_outer);
+			cnto++;
+		}
+		else
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			Assert(cmpval > 0);
+
+			/* A datum missing from the outer side. */
+			merged_index = -1;
+			merged_datum = idatums;
+
+			/*
+			 * For a FULL join, outer relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the outer relation acts as
+			 * OUTER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_outer = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_inner = (jointype == JOIN_FULL);
+
+			if (!handle_missing_partition(outer_maps,
+										  inner_maps,
+										  outer_default,
+										  i_index,
+										  missing_side_outer,
+										  missing_side_inner,
+										  &next_index,
+										  &default_index,
+										  &merged_index))
+				return NULL;
+
+			/* Move to the next datum on the right side. */
+			Assert(!finished_inner);
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	if (!merge_null_partitions(outer_bi, inner_bi,
+							   outer_maps, inner_maps,
+							   jointype, &next_index, &null_index,
+							   &default_index))
+		return NULL;
+
+	if (!merge_default_partitions(outer_bi, inner_bi,
+								  outer_maps, inner_maps,
+								  jointype, &next_index,
+								  &default_index))
+		return NULL;
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_maps, inner_maps,
+								 outer_nparts, inner_nparts,
+								 jointype, next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes, NIL,
+													  null_index, default_index);
+	}
+
+	/* Free up all extra memory before returning from this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_bounds_merge()'s arm for hash partitioned tables.
+ *
+ * If the given two hash bounds are same, the function returns the first one
+ * without any change, alongwith the lists of matching partitions. Otherwise it
+ * returns NULL.
+ *
+ * We could try merging the bounds when both the bounds have same greatest
+ * modulii. But there seems to be hardly any requirement for the same.
+ */
+static PartitionBoundInfo
+partition_hash_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype)
+{
+	int			nparts;
+	int			cnt;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * Hash partitioned table does not have explicit NULL accepting partition
+	 * and also does not have a default partition.
+	 */
+	Assert(!partition_bound_has_default(outer_bi) &&
+		   !partition_bound_has_default(inner_bi));
+	Assert(!partition_bound_accepts_nulls(outer_bi) &&
+		   !partition_bound_accepts_nulls(inner_bi));
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+
+	if (outer_nparts != inner_nparts)
+		return NULL;
+	nparts = outer_nparts;
+
+	if (outer_bi->ndatums != inner_bi->ndatums ||
+		!partition_hbounds_equal(outer_bi, inner_bi))
+		return NULL;
+
+	 /*
+	  * Cook up list of matching partitions. Since bounds are exactly same the
+	  * partitions at the same position from both the relations match.
+	  */
+	for (cnt = 0; cnt < nparts; cnt++)
+	{
+		*outer_parts = lappend_int(*outer_parts, cnt);
+		*inner_parts = lappend_int(*inner_parts, cnt);
+	}
+
+	return outer_bi;
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * If the two given partitions (given by index1 and index2 resp.) are
+ * already mapped to each other return the index of corresponding partition in
+ * the merged set of partitions.  If they do not have a merged partition
+ * associated with them, assign a new merged partition index.  If the
+ * partitions are already mapped and their mapped partitions are different from
+ * each other, they can not be merged, so return -1.
+ *
+ * partmaps1[i] gives the mapping of partitions for both relations. It
+ * describes which partition of relation 2 matches ith partition of relation 1,
+ * and which partition in the merged set matches ith partition of relation 1
+ * maps to. Similarly for partmap2.
+ *
+ * index1 and index2 are the indexes of matching partition from respective
+ * relations.
+ *
+ * *next_index is used and incremented when the given partitions require a new
+ * merged partition.
+ */
+
+static int
+map_and_merge_partitions(PartitionMap *partmaps1, PartitionMap *partmaps2,
+						 int index1, int index2, int *next_index)
+{
+	PartitionMap 	*partmap1 = &partmaps1[index1];
+	PartitionMap 	*partmap2 = &partmaps2[index2];
+	int				merged_index;
+
+	/*
+	 * If both the partitions are not mapped to each other, update the
+	 * maps.
+	 */
+	if (partmap1->from < 0 && partmap2->from < 0)
+	{
+		partmap1->from = index2;
+		partmap2->from = index1;
+	}
+
+	/*
+	 * If the given to partitions map to each other, find the corresponding
+	 * merged partition index .
+	 */
+	if (partmap1->from == index2 && partmap2->from == index1)
+	{
+		/*
+		 * If both the partitions are mapped to the same merged partition, get
+		 * the index of merged partition.
+		 */
+		if (partmap1->to == partmap2->to)
+		{
+			merged_index = partmap1->to;
+
+			/*
+			 * If the given two partitions do not have a merged partition
+			 * associated with them, allocate a new merged partition.
+			 */
+			if (merged_index < 0)
+			{
+				merged_index = *next_index;
+				*next_index = *next_index + 1;
+				partmap1->to = merged_index;
+				partmap2->to = merged_index;
+			}
+		}
+
+		/*
+		 * If partition from one relation was mapped to a merged partition but
+		 * not the partition from the other relation, map the same merged
+		 * partition to the partition from other relation, since matching
+		 * partitions map to the same merged partition.
+		 */
+		else if (partmap1->to >= 0 && partmap2->to < 0)
+		{
+			partmap2->to = partmap1->to;
+			merged_index = partmap1->to;
+		}
+		else if (partmap1->to < 0 && partmap2->to >= 0)
+		{
+			partmap1->to = partmap2->to;
+			merged_index = partmap2->to;
+		}
+		else
+		{
+			Assert(partmap1->to != partmap2->to &&
+				   partmap1->to >= 0 && partmap2->to >= 0);
+
+			/*
+			 * Both the partitions map to different merged partitions. This
+			 * means that multiple partitions from one relation matches to one
+			 * partition from the other relation. Partition-wise join does not
+			 * handle this case right now, since it requires ganging multiple
+			 * partitions together (into one RelOptInfo).
+			 */
+			merged_index = -1;
+		}
+	}
+	else
+	{
+		/*
+		 * Multiple partitions from one relation map to one partition from the
+		 * other relation. Partition-wise join does not handle this case right
+		 * now, since it requires ganging multiple partitions together (into
+		 * one RelOptInfo).
+		 */
+		merged_index = -1;
+	}
+
+	return merged_index;
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * partmaps1 map each partition from either side of the join to a merged
+ * partition resp. E.g. partmaps1[i].to gives the merged partition to which ith
+ * partition of first relation maps. Similarly for partmap2. If
+ * partmaps1[i].to == partmaps2[j].to, i and j form the matching pair of
+ * partitions.
+ *
+ * Given these maps this function produces the list pairs of partitions which
+ * when joined produce the merged partitions in the order of merged partition
+ * indexes.
+ *
+ * nparts1 and nparts2 are the number of partitions of the joining relations
+ * resp.
+ *
+ * nparts is the number of merged partitions.
+ *
+ * If successful, the pairs of partitions are returned as two separate lists,
+ * parts1 and parts2 resp., one for each side. Otherwise, those lists will be
+ * set to NIL.
+ */
+static void
+generate_matching_part_pairs(PartitionMap *partmaps1, PartitionMap *partmaps2,
+							 int nparts1, int nparts2,
+							 JoinType jointype, int nparts,
+							 List **matched_parts1, List **matched_parts2)
+{
+	bool	merged = true;
+	int		*matching1 = (int *) palloc(sizeof(int) * nparts),
+			*matching2 = (int *) palloc(sizeof(int) * nparts);
+	int 	i;
+
+	*matched_parts1 = NIL;
+	*matched_parts2 = NIL;
+
+	/* Set pairs of matching partitions. */
+	for (i = 0; i < nparts; i++)
+	{
+		if (i >= nparts1)
+			matching1[i] = -1;
+		else
+		{
+			PartitionMap outer_map = partmaps1[i];
+
+			if (outer_map.to >= 0)
+			{
+				Assert(outer_map.to < nparts);
+				matching1[outer_map.to] = i;
+			}
+		}
+
+		if (i >= nparts2)
+			matching2[i] = -1;
+		else
+		{
+			PartitionMap inner_map = partmaps2[i];
+
+			if (inner_map.to >= 0)
+			{
+				Assert(inner_map.to < nparts);
+				matching2[inner_map.to] = i;
+			}
+		}
+	}
+
+	/*
+	 * If we have a partition missing on an inner side, we need to add a dummy
+	 * relation which joins with the outer partition. If the inner relation
+	 * happens to be a base relation, it will require adding a dummy child
+	 * base relation during join processing. Right now, we freeze the base
+	 * relation arrays like PlannerInfo::simple_rte_array after planning for
+	 * base relations. Adding a new (dummy) base relation would require some
+	 * changes to that. So, right now, we do not implement partition-wise join
+	 * in such cases.
+	 */
+	for (i = 0; i < nparts; i++)
+	{
+		int			part1 = matching1[i];
+		int			part2 = matching2[i];
+
+		/* At least one of the partitions should exist. */
+		Assert(part1 >= 0 || part2 >= 0);
+
+		switch (jointype)
+		{
+			case JOIN_INNER:
+			case JOIN_SEMI:
+
+				/*
+				 * An inner or semi join can not return any row when the
+				 * matching partition on either side is missing. We should
+				 * have eliminated all such cases while merging the bounds.
+				 */
+				Assert(part1 >= 0 && part2 >= 0);
+				break;
+
+			case JOIN_LEFT:
+			case JOIN_ANTI:
+				Assert(part1 >= 0);
+				if (part2 < 0)
+					merged = false;
+				break;
+
+			case JOIN_FULL:
+				if (part1 < 0 || part2 < 0)
+					merged = false;
+				break;
+
+			default:
+				elog(ERROR, "unrecognized join type: %d", (int) jointype);
+		}
+
+		if (!merged)
+			break;
+
+		*matched_parts1 = lappend_int(*matched_parts1, part1);
+		*matched_parts2 = lappend_int(*matched_parts2, part2);
+	}
+
+	pfree(matching1);
+	pfree(matching2);
+
+	if (!merged)
+	{
+		list_free(*matched_parts1);
+		list_free(*matched_parts2);
+		*matched_parts1 = NIL;
+		*matched_parts2 = NIL;
+	}
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * Merge default partitions from both sides, if any, and assign the default
+ * partition for the join result, if necessary.
+ *
+ * If both the relations have default partitions, try mapping those to each
+ * other. If the mapping succeeds corresponding merged partition will act as
+ * the default partition of the join result.
+ *
+ * If inner side of the join has default but not the outer side, rows in it
+ * won't appear in the join result. So don't create a default partition. If
+ * outer side of the join has default but not the inner side, rows in it will
+ * appear in the join result, so create a default merged partition.
+ */
+static bool
+merge_default_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+						 PartitionMap *outer_maps, PartitionMap *inner_maps,
+						 JoinType jointype, int *next_index, int *default_index)
+{
+	int				outer_default = outer_bi->default_index;
+	int				inner_default = inner_bi->default_index;
+	bool			outer_has_default = partition_bound_has_default(outer_bi);
+	bool			inner_has_default = partition_bound_has_default(inner_bi);
+	bool			merged = true;
+	PartitionMap 	*outer_default_map = NULL;
+	PartitionMap 	*inner_default_map = NULL;
+
+	if (outer_has_default)
+		outer_default_map = &outer_maps[outer_default];
+
+	if (inner_has_default)
+		inner_default_map = &inner_maps[inner_default];
+
+	if (!outer_has_default && !inner_has_default)
+		Assert(*default_index < 0);
+	else if (outer_default_map != NULL && inner_default_map == NULL)
+	{
+		if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+			jointype == JOIN_ANTI)
+		{
+			if (outer_default_map->to < 0)
+			{
+				outer_default_map->to = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = outer_default_map->to;
+			}
+			else
+				Assert(*default_index == outer_default_map->to);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else if (outer_default_map == NULL && inner_default_map != NULL)
+	{
+		if (jointype == JOIN_FULL)
+		{
+			if (inner_default_map->to < 0)
+			{
+				inner_default_map->to = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = inner_default_map->to;
+			}
+			else
+				Assert(*default_index == inner_default_map->to);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		*default_index = map_and_merge_partitions(outer_maps, inner_maps,
+												  outer_default, inner_default,
+												  next_index);
+
+		if (*default_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
+
+/*
+ * merge_null_partitions
+ *
+ * Merge NULL partitions, i.e. a partition that can hold NULL values for a list
+ * partitioned table, if any. Find the index of merged partition to which the
+ * NULL values would belong in the join result. If one joining relation has a
+ * NULL partition but not the other, try matching it with the default partition
+ * from the other relation since the default partition may have rows with NULL
+ * partition key. We can eliminate a NULL partition when it appears only on the
+ * inner side of the join and the outer side doesn't have a default partition.
+ *
+ * When the equality operator used for join is strict, two NULL values will not
+ * be considered as equal, and thus a NULL partition can be eliminated for an
+ * inner join. But we don't check the strictness operator here.
+ */
+static bool
+merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_maps, PartitionMap *inner_maps,
+					  JoinType jointype, int *next_index,
+					  int *null_index, int *default_index)
+{
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	int			outer_ni = outer_bi->null_index;
+	int			inner_ni = inner_bi->null_index;
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	bool		merged = true;
+
+	if (!outer_has_null && !inner_has_null)
+		*null_index = -1;
+	else if (outer_has_null && !inner_has_null)
+	{
+		int			merged_index;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, inner relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the inner relation acts as
+		 * INNER relation. For INNER and SEMI join OUTER and INNER
+		 * differentiation is immaterial.
+		 */
+		missing_side_inner = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_outer = (jointype == JOIN_FULL);
+
+		merged = handle_missing_partition(inner_maps,
+										  outer_maps,
+										  inner_default,
+										  outer_ni,
+										  missing_side_outer,
+										  missing_side_inner, next_index,
+										  default_index, &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join to which the outer null partition maps
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = outer_maps[outer_ni].to;
+	}
+	else if (!outer_has_null && inner_has_null)
+	{
+		int			merged_index;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, outer relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the outer relation acts as OUTER
+		 * relation. For INNER and SEMI join OUTER and INNER differentiation is
+		 * immaterial.
+		 */
+		missing_side_outer = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_inner = (jointype == JOIN_FULL);
+		merged = handle_missing_partition(outer_maps,
+										  inner_maps,
+										  outer_default,
+										  inner_ni,
+										  missing_side_outer,
+										  missing_side_inner,
+										  next_index, default_index,
+										  &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join, to which the outer side null partition maps,
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = inner_maps[inner_ni].to;
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		*null_index = map_and_merge_partitions(outer_maps,
+											   inner_maps,
+											   outer_ni,
+											   inner_ni,
+											   next_index);
+		if (*null_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index b1ae39ad63..fb2dc1cbb1 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -16,6 +16,7 @@
 #include "nodes/pg_list.h"
 #include "partitioning/partdefs.h"
 #include "utils/relcache.h"
+struct RelOptInfo;				/* avoid including pathnodes.h here */
 
 
 /*
@@ -107,5 +108,11 @@ extern int partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 							  int nvalues, Datum *values, bool *is_equal);
 extern int partition_hash_bsearch(PartitionBoundInfo boundinfo,
 					   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   struct RelOptInfo *outer_rel,
+					   struct RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
-- 
2.19.2

0003-Tests-for-0-1-1-1-and-1-0-partition-matching-v17.patchtext/x-patch; name=0003-Tests-for-0-1-1-1-and-1-0-partition-matching-v17.patchDownload
From 03bb2b2dc4b9d24c7f47a4eaeb16b6d3d166c742 Mon Sep 17 00:00:00 2001
From: Etsuro Fujita <efujita@postgresql.org>
Date: Thu, 31 Jan 2019 20:31:17 +0900
Subject: [PATCH 3/3] Tests for 0:1, 1:1 and 1:0 partition matching

Rajkumar Raghuvanshi and Ashutosh Bapat.
---
 src/test/regress/expected/partition_join.out | 4131 +++++++++++++++---
 src/test/regress/sql/partition_join.sql      |  427 +-
 2 files changed, 3855 insertions(+), 703 deletions(-)

diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index c55de5d476..f19611c853 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -8,59 +8,86 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-(4 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+(7 rows)
 
 -- left outer join, with whole-row reference; partitionwise join does not apply
 EXPLAIN (COSTS OFF)
@@ -72,35 +99,50 @@ SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER
    ->  Hash Right Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(22 rows)
 
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-      t1      |      t2      
---------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
- (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
- (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
- (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
- (550,0,0550) | 
-(12 rows)
+       t1       |       t2       
+----------------+----------------
+ (-250,0,-0250) | 
+ (-200,0,-0200) | 
+ (-150,0,-0150) | (0,-150,-0150)
+ (-100,0,-0100) | 
+ (-50,0,-0050)  | 
+ (0,0,0000)     | (0,0,0000)
+ (50,0,0050)    | 
+ (100,0,0100)   | 
+ (150,0,0150)   | (0,150,0150)
+ (200,0,0200)   | 
+ (250,0,0250)   | 
+ (300,0,0300)   | (0,300,0300)
+ (350,0,0350)   | 
+ (400,0,0400)   | 
+ (450,0,0450)   | (0,450,0450)
+ (500,0,0500)   | 
+ (550,0,0550)   | 
+ (600,0,0600)   | (0,600,0600)
+ (650,0,0650)   | 
+ (700,0,0700)   | 
+ (750,0,0750)   | (0,750,0750)
+(21 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
@@ -112,35 +154,53 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
    ->  Append
          ->  Hash Right Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Right Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t2_2.b)
-(20 rows)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+      |       | -225 | -0225
+      |       |  -75 | -0075
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(14 rows)
 
 -- full outer join, with placeholder vars
 EXPLAIN (COSTS OFF)
@@ -148,8 +208,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                             QUERY PLAN                            
 ------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               Filter: (((50) = prt1_p0.a) OR ((75) = prt2_p0.b))
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
@@ -174,7 +242,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               Filter: (((50) = prt1_p4.a) OR ((75) = prt2_p4.b))
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
@@ -211,8 +287,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Hash Left Join
+               Hash Cond: (prt1_p0.a = b)
+               ->  Seq Scan on prt1_p0
+                     Filter: ((a < 450) AND (b = 0))
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
          ->  Hash Left Join
                Hash Cond: (prt1_p1.a = b)
                ->  Seq Scan on prt1_p1
@@ -227,29 +310,42 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                ->  Hash
                      ->  Seq Scan on prt1_p2
                            Filter: ((a < 450) AND (b = 0))
-(17 rows)
+(24 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-(9 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+(14 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
                          QUERY PLAN                         
 ------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = b)
+               Filter: ((prt1_p0.b = 0) OR (a = 0))
+               ->  Seq Scan on prt1_p0
+                     Filter: (a < 450)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = b)
                Filter: ((prt1_p1.b = 0) OR (a = 0))
@@ -274,64 +370,153 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                ->  Hash
                      ->  Result
                            One-Time Filter: false
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt2_p4.b = a)
+               Filter: ((b = 0) OR (prt2_p4.a = 0))
+               ->  Seq Scan on prt2_p4
+                     Filter: (b > 250)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
-(12 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+      |       | 375 | 0375
+      |       | 450 | 0450
+      |       | 525 | 0525
+      |       | 600 | 0600
+      |       | 675 | 0675
+      |       | 750 | 0750
+(20 rows)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Semi Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Semi Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                            Filter: (a = 0)
-         ->  Nested Loop Semi Join
-               Join Filter: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
-               ->  Materialize
-                     ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
-(24 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt1_p3 t2_3
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+                           Filter: (b = 0)
+(37 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
 
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
@@ -342,27 +527,82 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
    ->  Append
          ->  Hash Anti Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Hash Anti Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Hash Anti Join
                Hash Cond: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
-(17 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(27 rows)
 
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
-  sum  |         avg          | sum  |         avg         
--------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+  sum  |         avg          | sum  |        avg         
+-------+----------------------+------+--------------------
+ 95550 | 273.0000000000000000 | 2200 | 6.2857142857142857
 (1 row)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Append
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p0 t1
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+               Index Cond: (a = t1.b)
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.b = t2_1.a)
+         ->  Seq Scan on prt2_p1 t1_1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p1 t2_1
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p2 t1_2
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+               Index Cond: (a = t1_2.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p3 t1_3
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+               Index Cond: (a = t1_3.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p4 t1_4
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+               Index Cond: (a = t1_4.b)
+(27 rows)
+
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+  b   |   c   
+------+-------
+ -225 | -0225
+  -75 | -0075
+   75 | 0075
+  225 | 0225
+  375 | 0375
+  525 | 0525
+  675 | 0675
+(7 rows)
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -374,49 +614,74 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
+                     ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
                            Index Cond: (a = t1.a)
-                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t3
                            Index Cond: (b = t2.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_1
+                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
                            Index Cond: (a = t1_1.a)
-                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3_1
                            Index Cond: (b = t2_1.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_2
+                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
                            Index Cond: (a = t1_2.a)
-                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_2
                            Index Cond: (b = t2_2.a)
-(27 rows)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                           Index Cond: (a = t1_3.a)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_3
+                           Index Cond: (b = t2_3.a)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                           Index Cond: (a = t1_4.a)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t3_4
+                           Index Cond: (b = t2_4.a)
+(43 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   | t2a | t3a | least 
------+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
- 100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
- 250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
- 400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
- 550 | 0 | 0550 |     |     |      
-(12 rows)
+  a   | b |   c   | t2a  | t3a | least 
+------+---+-------+------+-----+-------
+ -250 | 0 | -0250 |      |     |      
+ -200 | 0 | -0200 |      |     |      
+ -150 | 0 | -0150 | -150 |   0 |  -150
+ -100 | 0 | -0100 |      |     |      
+  -50 | 0 | -0050 |      |     |      
+    0 | 0 | 0000  |    0 |   0 |     0
+   50 | 0 | 0050  |      |     |      
+  100 | 0 | 0100  |      |     |      
+  150 | 0 | 0150  |  150 |   0 |   150
+  200 | 0 | 0200  |      |     |      
+  250 | 0 | 0250  |      |     |      
+  300 | 0 | 0300  |  300 |   0 |   300
+  350 | 0 | 0350  |      |     |      
+  400 | 0 | 0400  |      |     |      
+  450 | 0 | 0450  |  450 |   0 |   450
+  500 | 0 | 0500  |      |     |      
+  550 | 0 | 0550  |      |     |      
+  600 | 0 | 0600  |  600 |   0 |   600
+  650 | 0 | 0650  |      |     |      
+  700 | 0 | 0700  |      |     |      
+  750 | 0 | 0750  |  750 |   0 |   750
+(21 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
@@ -430,64 +695,95 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
          Hash Cond: ((t1.c)::text = (t2.c)::text)
          Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Seq Scan on prt1_p1 t1
-               ->  Seq Scan on prt1_p2 t1_1
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
          ->  Hash
                ->  Append
                      ->  Hash Join
                            Hash Cond: (t2.a = t3.b)
-                           ->  Seq Scan on prt1_p1 t2
+                           ->  Seq Scan on prt1_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t3
+                                 ->  Seq Scan on prt2_p0 t3
                      ->  Hash Join
                            Hash Cond: (t2_1.a = t3_1.b)
-                           ->  Seq Scan on prt1_p2 t2_1
+                           ->  Seq Scan on prt1_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t3_1
+                                 ->  Seq Scan on prt2_p1 t3_1
                      ->  Hash Join
                            Hash Cond: (t2_2.a = t3_2.b)
-                           ->  Seq Scan on prt1_p3 t2_2
+                           ->  Seq Scan on prt1_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p3 t3_2
-(26 rows)
+                                 ->  Seq Scan on prt2_p2 t3_2
+                     ->  Hash Join
+                           Hash Cond: (t2_3.a = t3_3.b)
+                           ->  Seq Scan on prt1_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t3_3
+                     ->  Hash Join
+                           Hash Cond: (t2_4.a = t3_4.b)
+                           ->  Seq Scan on prt1_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t3_4
+(38 rows)
 
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, t2.b t2b, t2.c t2c, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
-  a  | t2a | t2c  
------+-----+------
-   0 |   0 | 0000
-  50 |     | 
- 100 |     | 
- 150 | 150 | 0150
- 200 |     | 
- 250 |     | 
- 300 | 300 | 0300
- 350 |     | 
- 400 |     | 
- 450 | 450 | 0450
- 500 |     | 
- 550 |     | 
-(12 rows)
+  a   | t2a  |  t2c  
+------+------+-------
+ -250 |      | 
+ -200 |      | 
+ -150 | -150 | -0150
+ -100 |      | 
+  -50 |      | 
+    0 |    0 | 0000
+   50 |      | 
+  100 |      | 
+  150 |  150 | 0150
+  200 |      | 
+  250 |      | 
+  300 |  300 | 0300
+  350 |      | 
+  400 |      | 
+  450 |  450 | 0450
+  500 |      | 
+  550 |      | 
+  600 |  600 | 0600
+  650 |      | 
+  700 |      | 
+  750 |  750 | 0750
+(21 rows)
 
 --
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
@@ -498,32 +794,49 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
    ->  Append
          ->  Hash Join
                Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
-               ->  Seq Scan on prt2_e_p1 t2
+               ->  Seq Scan on prt2_e_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 t1
+                     ->  Seq Scan on prt1_e_p0 t1
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
-               ->  Seq Scan on prt2_e_p2 t2_1
+               Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+               ->  Seq Scan on prt1_e_p1 t1_1
+                     Filter: (c = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 t1_1
-                           Filter: (c = 0)
+                     ->  Seq Scan on prt2_e_p1 t2_1
          ->  Hash Join
                Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
-               ->  Seq Scan on prt2_e_p3 t2_2
+               ->  Seq Scan on prt2_e_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 t1_2
+                     ->  Seq Scan on prt1_e_p2 t1_2
                            Filter: (c = 0)
-(21 rows)
+         ->  Hash Join
+               Hash Cond: (((t2_3.b + t2_3.a) / 2) = ((t1_3.a + t1_3.b) / 2))
+               ->  Seq Scan on prt2_e_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p3 t1_3
+                           Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (((t2_4.b + t2_4.a) / 2) = ((t1_4.a + t1_4.b) / 2))
+               ->  Seq Scan on prt2_e_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4 t1_4
+                           Filter: (c = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
- 150 | 0 | 150 | 0
- 300 | 0 | 300 | 0
- 450 | 0 | 450 | 0
-(4 rows)
+  a   | c |  b   | c 
+------+---+------+---
+ -250 | 0 | -250 | 0
+ -100 | 0 | -100 | 0
+    0 | 0 |    0 | 0
+    0 | 0 |    0 | 0
+  150 | 0 |  150 | 0
+  300 | 0 |  300 | 0
+  450 | 0 |  450 | 0
+  600 | 0 |  600 | 0
+  750 | 0 |  750 | 0
+(9 rows)
 
 --
 -- N-way join
@@ -536,154 +849,232 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = ((t3.a + t3.b) / 2))
+               Join Filter: (t1.a = t2.b)
                ->  Hash Join
-                     Hash Cond: (t2.b = t1.a)
-                     ->  Seq Scan on prt2_p1 t2
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on prt1_e_p0 t3
                      ->  Hash
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                     Index Cond: (((a + b) / 2) = t2.b)
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
+                     Index Cond: (b = ((t3.a + t3.b) / 2))
          ->  Nested Loop
                Join Filter: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_1.b = t1_1.a)
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
+               ->  Index Scan using iprt1_e_p4_ab2 on prt1_e_p1 t3_1
                      Index Cond: (((a + b) / 2) = t2_1.b)
          ->  Nested Loop
                Join Filter: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_2.b = t1_2.a)
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
+               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_2
                      Index Cond: (((a + b) / 2) = t2_2.b)
-(33 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = ((t3_3.a + t3_3.b) / 2))
+               ->  Nested Loop
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t2_3.b)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t2_4.b)
+               ->  Hash Join
+                     Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                     ->  Seq Scan on prt1_e_p4 t3_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_p4 t1_4
+                                 Filter: (b = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = ((t3_4.a + t3_4.b) / 2))
+(52 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
-(4 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(8 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                          QUERY PLAN                          
---------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Hash Right Join
                Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
-               ->  Seq Scan on prt1_e_p1 t3
+               ->  Seq Scan on prt1_e_p0 t3
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2.b = t1.a)
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_1.a)
-               ->  Seq Scan on prt1_e_p2 t3_1
+               ->  Seq Scan on prt1_e_p1 t3_1
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_1.b = t1_1.a)
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_2.a)
-               ->  Seq Scan on prt1_e_p3 t3_2
+               ->  Seq Scan on prt1_e_p2 t3_2
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_2.b = t1_2.a)
-                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                                        Filter: (b = 0)
-(33 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (t1_3.a = b)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (t1_3.a = ((a + b) / 2))
+         ->  Hash Right Join
+               Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+               ->  Seq Scan on prt1_e_p4 t3_4
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_4.b = t1_4.a)
+                           ->  Seq Scan on prt2_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_p4 t1_4
+                                       Filter: (b = 0)
+(51 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                            QUERY PLAN                             
--------------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p1 t3
+                           ->  Seq Scan on prt1_e_p0 t3
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p1_b on prt2_p1 t2
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
                      Index Cond: (t1.a = b)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p2 t3_1
+                           ->  Seq Scan on prt1_e_p1 t3_1
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
+               ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
                      Index Cond: (t1_1.a = b)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p3 t3_2
+                           ->  Seq Scan on prt1_e_p2 t3_2
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
                      Index Cond: (t1_2.a = b)
-(30 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_e_p3 t3_3
+                           Filter: (c = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                           Index Cond: (a = ((t3_3.a + t3_3.b) / 2))
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (t1_3.a = b)
+         ->  Nested Loop Left Join
+               ->  Hash Right Join
+                     Hash Cond: (t1_4.a = ((t3_4.a + t3_4.b) / 2))
+                     ->  Seq Scan on prt1_p4 t1_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p4 t3_4
+                                 Filter: (c = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (t1_4.a = b)
+(47 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -692,21 +1083,34 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                                                    QUERY PLAN                                                   
 ----------------------------------------------------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   Sort Key: prt1_p0.a, prt2_p0.b, ((prt1_e_p0.a + prt1_e_p0.b))
    ->  Append
          ->  Hash Full Join
-               Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
-               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               Hash Cond: (prt1_p0.a = ((prt1_e_p0.a + prt1_e_p0.b) / 2))
+               Filter: ((prt1_p0.a = (50)) OR (prt2_p0.b = (75)) OR (((prt1_e_p0.a + prt1_e_p0.b) / 2) = (50)))
                ->  Hash Full Join
-                     Hash Cond: (prt1_p1.a = prt2_p1.b)
-                     ->  Seq Scan on prt1_p1
+                     Hash Cond: (prt1_p0.a = prt2_p0.b)
+                     ->  Seq Scan on prt1_p0
                            Filter: (b = 0)
                      ->  Hash
-                           ->  Seq Scan on prt2_p1
+                           ->  Seq Scan on prt2_p0
                                  Filter: (a = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1
+                     ->  Seq Scan on prt1_e_p0
                            Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = prt1_p1.a)
+               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               ->  Seq Scan on prt1_e_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on prt1_p1
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1
+                                       Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
                Filter: ((prt1_p2.a = (50)) OR (prt2_p2.b = (75)) OR (((prt1_e_p2.a + prt1_e_p2.b) / 2) = (50)))
@@ -733,7 +1137,20 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                ->  Hash
                      ->  Seq Scan on prt1_e_p3
                            Filter: (c = 0)
-(42 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = ((prt1_e_p4.a + prt1_e_p4.b) / 2))
+               Filter: ((prt1_p4.a = (50)) OR (prt2_p4.b = (75)) OR (((prt1_e_p4.a + prt1_e_p4.b) / 2) = (50)))
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = prt2_p4.b)
+                     ->  Seq Scan on prt1_p4
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4
+                                 Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4
+                           Filter: (c = 0)
+(68 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
@@ -751,172 +1168,260 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = t1_3.b)
+               Join Filter: (t1.a = t1_5.b)
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Join
-                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
-                           ->  Seq Scan on prt1_e_p1 t2
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t1_3
+                                 ->  Seq Scan on prt2_p0 t1_5
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
                      Index Cond: (a = ((t2.a + t2.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_1.a = t1_4.b)
+               Join Filter: (t1_1.a = t1_6.b)
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Join
-                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
-                           ->  Seq Scan on prt1_e_p2 t2_1
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_6.b)
+                           ->  Seq Scan on prt1_e_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t1_4
+                                 ->  Seq Scan on prt2_p1 t1_6
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
                      Index Cond: (a = ((t2_1.a + t2_1.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_2.a = t1_5.b)
+               Join Filter: (t1_2.a = t1_7.b)
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
+                           ->  Seq Scan on prt2_p2 t1_7
                                  Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
+                           ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t2_2
+                                 Index Cond: (((a + b) / 2) = t1_7.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t1_8.b)
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Nested Loop
+                           ->  Seq Scan on prt2_p3 t1_8
+                                 Filter: (a = 0)
+                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_3
+                                 Index Cond: (((a + b) / 2) = t1_8.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = ((t2_3.a + t2_3.b) / 2))
+                     Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t1_9.b)
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Join
+                           Hash Cond: (((t2_4.a + t2_4.b) / 2) = t1_9.b)
+                           ->  Seq Scan on prt1_e_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t1_9
+                                       Filter: (a = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = ((t2_4.a + t2_4.b) / 2))
+                     Filter: (b = 0)
+(66 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                               QUERY PLAN                                
--------------------------------------------------------------------------
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
-                           ->  Seq Scan on prt2_p1 t1_3
+                           Hash Cond: (t1_5.b = ((t1_10.a + t1_10.b) / 2))
+                           ->  Seq Scan on prt2_p0 t1_5
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t1_6
+                                 ->  Seq Scan on prt1_e_p0 t1_10
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
-                     Index Cond: (a = t1_3.b)
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t1_5.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
-                           ->  Seq Scan on prt2_p2 t1_4
+                           Hash Cond: (t1_6.b = ((t1_11.a + t1_11.b) / 2))
+                           ->  Seq Scan on prt2_p1 t1_6
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t1_7
+                                 ->  Seq Scan on prt1_e_p1 t1_11
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
-                     Index Cond: (a = t1_4.b)
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
+                     Index Cond: (a = t1_6.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
-                           ->  Seq Scan on prt2_p3 t1_5
+                           Hash Cond: (t1_7.b = ((t1_12.a + t1_12.b) / 2))
+                           ->  Seq Scan on prt2_p2 t1_7
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p3 t1_8
+                                 ->  Seq Scan on prt1_e_p2 t1_12
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t1_5.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t1_7.b)
                      Filter: (b = 0)
-(39 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_8.b = ((t1_13.a + t1_13.b) / 2))
+                           ->  Seq Scan on prt2_p3 t1_8
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p3 t1_13
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t1_8.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_9.b = ((t1_14.a + t1_14.b) / 2))
+                           ->  Seq Scan on prt2_p4 t1_9
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t1_14
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = t1_9.b)
+                     Filter: (b = 0)
+(63 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 -- test merge joins
 SET enable_hashjoin TO off;
 SET enable_nestloop TO off;
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                           QUERY PLAN                           
-----------------------------------------------------------------
+                            QUERY PLAN                            
+------------------------------------------------------------------
  Merge Append
    Sort Key: t1.a
    ->  Merge Semi Join
-         Merge Cond: (t1.a = t1_3.b)
+         Merge Cond: (t1.a = t1_5.b)
          ->  Sort
                Sort Key: t1.a
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+               Merge Cond: (t1_5.b = (((t1_10.a + t1_10.b) / 2)))
                ->  Sort
-                     Sort Key: t1_3.b
-                     ->  Seq Scan on prt2_p1 t1_3
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on prt2_p0 t1_5
                ->  Sort
-                     Sort Key: (((t1_6.a + t1_6.b) / 2))
-                     ->  Seq Scan on prt1_e_p1 t1_6
+                     Sort Key: (((t1_10.a + t1_10.b) / 2))
+                     ->  Seq Scan on prt1_e_p0 t1_10
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_1.a = t1_4.b)
+         Merge Cond: (t1_1.a = t1_6.b)
          ->  Sort
                Sort Key: t1_1.a
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+               Merge Cond: (t1_6.b = (((t1_11.a + t1_11.b) / 2)))
                ->  Sort
-                     Sort Key: t1_4.b
-                     ->  Seq Scan on prt2_p2 t1_4
+                     Sort Key: t1_6.b
+                     ->  Seq Scan on prt2_p1 t1_6
                ->  Sort
-                     Sort Key: (((t1_7.a + t1_7.b) / 2))
-                     ->  Seq Scan on prt1_e_p2 t1_7
+                     Sort Key: (((t1_11.a + t1_11.b) / 2))
+                     ->  Seq Scan on prt1_e_p1 t1_11
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_2.a = t1_5.b)
+         Merge Cond: (t1_2.a = t1_7.b)
          ->  Sort
                Sort Key: t1_2.a
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+               Merge Cond: (t1_7.b = (((t1_12.a + t1_12.b) / 2)))
                ->  Sort
-                     Sort Key: t1_5.b
-                     ->  Seq Scan on prt2_p3 t1_5
+                     Sort Key: t1_7.b
+                     ->  Seq Scan on prt2_p2 t1_7
                ->  Sort
-                     Sort Key: (((t1_8.a + t1_8.b) / 2))
-                     ->  Seq Scan on prt1_e_p3 t1_8
+                     Sort Key: (((t1_12.a + t1_12.b) / 2))
+                     ->  Seq Scan on prt1_e_p2 t1_12
                            Filter: (c = 0)
-(47 rows)
+   ->  Merge Semi Join
+         Merge Cond: (t1_3.a = t1_8.b)
+         ->  Sort
+               Sort Key: t1_3.a
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_8.b = (((t1_13.a + t1_13.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_8.b
+                     ->  Seq Scan on prt2_p3 t1_8
+               ->  Sort
+                     Sort Key: (((t1_13.a + t1_13.b) / 2))
+                     ->  Seq Scan on prt1_e_p3 t1_13
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_4.a = t1_9.b)
+         ->  Sort
+               Sort Key: t1_4.a
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_9.b = (((t1_14.a + t1_14.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_9.b
+                     ->  Seq Scan on prt2_p4 t1_9
+               ->  Sort
+                     Sort Key: (((t1_14.a + t1_14.b) / 2))
+                     ->  Seq Scan on prt1_e_p4 t1_14
+                           Filter: (c = 0)
+(77 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
@@ -933,14 +1438,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
                            ->  Sort
                                  Sort Key: (((t3.a + t3.b) / 2))
-                                 ->  Seq Scan on prt1_e_p1 t3
+                                 ->  Seq Scan on prt1_e_p0 t3
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1.a
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                ->  Sort
                      Sort Key: t2.b
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Merge Left Join
                Merge Cond: (t1_1.a = t2_1.b)
                ->  Sort
@@ -949,14 +1454,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
                            ->  Sort
                                  Sort Key: (((t3_1.a + t3_1.b) / 2))
-                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                 ->  Seq Scan on prt1_e_p1 t3_1
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_1.a
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                ->  Sort
                      Sort Key: t2_1.b
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Merge Left Join
                Merge Cond: (t1_2.a = t2_2.b)
                ->  Sort
@@ -965,32 +1470,74 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
                            ->  Sort
                                  Sort Key: (((t3_2.a + t3_2.b) / 2))
-                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                 ->  Seq Scan on prt1_e_p2 t3_2
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_2.a
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                ->  Sort
                      Sort Key: t2_2.b
-                     ->  Seq Scan on prt2_p3 t2_2
-(51 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Merge Left Join
+               Merge Cond: (t1_3.a = t2_3.b)
+               ->  Sort
+                     Sort Key: t1_3.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_3.a + t3_3.b) / 2)) = t1_3.a)
+                           ->  Sort
+                                 Sort Key: (((t3_3.a + t3_3.b) / 2))
+                                 ->  Seq Scan on prt1_e_p3 t3_3
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_3.a
+                                 ->  Seq Scan on prt1_p3 t1_3
+               ->  Sort
+                     Sort Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Merge Left Join
+               Merge Cond: (t1_4.a = t2_4.b)
+               ->  Sort
+                     Sort Key: t1_4.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_4.a + t3_4.b) / 2)) = t1_4.a)
+                           ->  Sort
+                                 Sort Key: (((t3_4.a + t3_4.b) / 2))
+                                 ->  Seq Scan on prt1_e_p4 t3_4
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_4.a
+                                 ->  Seq Scan on prt1_p4 t1_4
+               ->  Sort
+                     Sort Key: t2_4.b
+                     ->  Seq Scan on prt2_p4 t2_4
+(83 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- MergeAppend on nullable column
 EXPLAIN (COSTS OFF)
@@ -998,8 +1545,18 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Merge Left Join
+               Merge Cond: (prt1_p0.a = b)
+               ->  Sort
+                     Sort Key: prt1_p0.a
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
+               ->  Sort
+                     Sort Key: b
+                     ->  Result
+                           One-Time Filter: false
          ->  Merge Left Join
                Merge Cond: (prt1_p1.a = b)
                ->  Sort
@@ -1020,21 +1577,26 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                      Sort Key: prt2_p2.b
                      ->  Seq Scan on prt2_p2
                            Filter: (b > 250)
-(23 rows)
+(33 rows)
 
 SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  b  
------+-----
-   0 |    
-  50 |    
- 100 |    
- 150 |    
- 200 |    
- 250 |    
- 300 | 300
- 350 |    
- 400 |    
-(9 rows)
+  a   |  b  
+------+-----
+ -250 |    
+ -200 |    
+ -150 |    
+ -100 |    
+  -50 |    
+    0 |    
+   50 |    
+  100 |    
+  150 |    
+  200 |    
+  250 |    
+  300 | 300
+  350 |    
+  400 |    
+(14 rows)
 
 -- merge join when expression with whole-row reference needs to be sorted;
 -- partitionwise join does not apply
@@ -1048,175 +1610,2412 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
          Sort Key: t1.a, ((((t1.*)::prt1))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
    ->  Sort
          Sort Key: t2.b, ((((t2.*)::prt2))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt2_p1 t2
-                     ->  Seq Scan on prt2_p2 t2_1
-                     ->  Seq Scan on prt2_p3 t2_2
-(16 rows)
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+(20 rows)
 
 SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.b ORDER BY t1.a;
- a  | b  
-----+----
-  0 |  0
-  6 |  6
- 12 | 12
- 18 | 18
- 24 | 24
-(5 rows)
+  a  |  b  
+-----+-----
+ -24 | -24
+ -18 | -18
+ -12 | -12
+  -6 |  -6
+   0 |   0
+   6 |   6
+  12 |  12
+  18 |  18
+  24 |  24
+(9 rows)
 
 RESET enable_hashjoin;
 RESET enable_nestloop;
---
--- partitioned by multiple columns
---
-CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
-CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
-ANALYZE prt1_m;
-CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
-CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
-ANALYZE prt2_m;
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-                                                             QUERY PLAN                                                             
-------------------------------------------------------------------------------------------------------------------------------------
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p4 t2_3
+               ->  Seq Scan on prt2_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_4
+                           Filter: (b = 0)
+(22 rows)
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p3 t1_2
+                           Filter: (b = 0)
+(21 rows)
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -150 | -0150 | 0 | -0150
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+(7 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (t1_3.a = b)
+         ->  Hash Right Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 |   | 
+ -200 | -0200 |   | 
+ -150 | -0150 | 0 | -0150
+ -100 | -0100 |   | 
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+(21 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -200 | 0 | -0200
+ -100 | 0 | -0100
+  -50 | 0 | -0050
+   50 | 0 | 0050
+  100 | 0 | 0100
+  200 | 0 | 0200
+  250 | 0 | 0250
+  350 | 0 | 0350
+  400 | 0 | 0400
+  500 | 0 | 0500
+  550 | 0 | 0550
+  650 | 0 | 0650
+  700 | 0 | 0700
+(14 rows)
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+                             QUERY PLAN                              
+---------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t3.c
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p0_a on prt1_p0 t3
+                           Index Cond: (a = t2.b)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t2.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_1.a = t2_1.b)
+                           ->  Seq Scan on prt1_p1 t3_1
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1 t2_1
+                                       Filter: (a = 0)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p2_a on prt1_p2 t3_2
+                           Index Cond: (a = t2_2.b)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t3_3
+                           Index Cond: (a = t2_3.b)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_4.a = t2_4.b)
+                           ->  Seq Scan on prt1_p4 t3_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t2_4
+                                       Filter: (a = 0)
+(47 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+  a   | a |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.b
+   ->  Hash Right Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p5 t2_5
+                           Filter: (a = 0)
+(24 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: (t1.a = t2.b)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.a, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
+                     Filter: (a = 0)
+(22 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
+               ->  Seq Scan on prt1_m_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p1
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
+               ->  Seq Scan on prt1_m_p2
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p2
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
+               ->  Seq Scan on prt1_m_p3
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p3
+                           Filter: (c = 0)
+(24 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+  a  | c |  b  | c 
+-----+---+-----+---
+   0 | 0 |   0 | 0
+  50 | 0 |     |  
+ 100 | 0 |     |  
+ 150 | 0 | 150 | 0
+ 200 | 0 |     |  
+ 250 | 0 |     |  
+ 300 | 0 | 300 | 0
+ 350 | 0 |     |  
+ 400 | 0 |     |  
+ 450 | 0 | 450 | 0
+ 500 | 0 |     |  
+ 550 | 0 |     |  
+     |   |  75 | 0
+     |   | 225 | 0
+     |   | 375 | 0
+     |   | 525 | 0
+(16 rows)
+
+--
+-- tests for list partitioned tables.
+--
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
+--
+-- list partitioned by expression
+--
+CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
+ANALYZE plt1_e;
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+                                           QUERY PLAN                                           
+------------------------------------------------------------------------------------------------
+ GroupAggregate
+   Group Key: t1.c, t2.c, t3.c
+   ->  Sort
+         Sort Key: t1.c, t3.c
+         ->  Append
+               ->  Hash Join
+                     Hash Cond: ((t1.c)::text = ltrim(t3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2.b = t1.b) AND ((t2.c)::text = (t1.c)::text))
+                           ->  Seq Scan on plt2_p4 t2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p4 t1
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p4 t3
+               ->  Hash Join
+                     Hash Cond: ((t1_1.c)::text = ltrim(t3_1.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t1_1.b = t2_1.b) AND ((t1_1.c)::text = (t2_1.c)::text))
+                           ->  Seq Scan on plt1_p1 t1_1
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p1 t2_1
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p1 t3_1
+               ->  Hash Join
+                     Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t1_2.b = t2_2.b) AND ((t1_2.c)::text = (t2_2.c)::text))
+                           ->  Seq Scan on plt1_p2 t1_2
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p2 t2_2
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p2 t3_2
+               ->  Hash Join
+                     Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2_3.b = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text))
+                           ->  Seq Scan on plt2_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p3 t3_3
+(41 rows)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p5 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_3
+                                 ->  Seq Scan on plt2_p3 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
  Sort
-   Sort Key: prt1_m_p1.a, prt2_m_p1.b
-   ->  Append
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
-               ->  Seq Scan on prt1_m_p1
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p1
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
-               ->  Seq Scan on prt1_m_p2
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p2
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
-               ->  Seq Scan on prt1_m_p3
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p3
-                           Filter: (c = 0)
-(24 rows)
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
 
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
-  50 | 0 |     |  
- 100 | 0 |     |  
- 150 | 0 | 150 | 0
- 200 | 0 |     |  
- 250 | 0 |     |  
- 300 | 0 | 300 | 0
- 350 | 0 |     |  
- 400 | 0 |     |  
- 450 | 0 | 450 | 0
- 500 | 0 |     |  
- 550 | 0 |     |  
-     |   |  75 | 0
-     |   | 225 | 0
-     |   | 375 | 0
-     |   | 525 | 0
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
 (16 rows)
 
---
--- tests for list partitioned tables.
---
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
---
--- list partitioned by expression
---
-CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1_e;
--- test partition matching with N-way join
+-- semi join
 EXPLAIN (COSTS OFF)
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                   QUERY PLAN                                   
---------------------------------------------------------------------------------
- GroupAggregate
-   Group Key: t1.c, t2.c, t3.c
-   ->  Sort
-         Sort Key: t1.c, t3.c
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
          ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.c = ltrim(t3.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1.b = t2.b) AND (t1.c = t2.c))
-                           ->  Seq Scan on plt1_p1 t1
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.c = ltrim(t3_1.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
-                           ->  Seq Scan on plt1_p2 t1_1
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.c = ltrim(t3_2.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
-                           ->  Seq Scan on plt1_p3 t1_2
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p3 t3_2
-(32 rows)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Seq Scan on plt2_p2 t2_2
+                                 ->  Seq Scan on plt2_p3 t2_3
+(22 rows)
 
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  75.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 123.0000000000000000 | 123.0000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 225.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 273.0000000000000000 | 273.0000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 375.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 423.0000000000000000 | 423.0000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 525.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 573.0000000000000000 | 573.0000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1241,22 +4040,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
 --------------------------------------------------
  Hash Left Join
    Hash Cond: (t2.b = a)
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t3.a = t2.b)
-               ->  Seq Scan on prt1_p1 t3
-               ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
-         ->  Hash Join
-               Hash Cond: (t3_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t3_1
-               ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
-         ->  Hash Join
-               Hash Cond: (t3_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t3_2
-               ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
+   ->  Hash Join
+         Hash Cond: (t3.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t3
+               ->  Seq Scan on prt1_p1 t3_1
+               ->  Seq Scan on prt1_p2 t3_2
+               ->  Seq Scan on prt1_p3 t3_3
+               ->  Seq Scan on prt1_p4 t3_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
    ->  Hash
          ->  Result
                One-Time Filter: false
@@ -1271,16 +4070,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 --
 -- tests for hash partitioned tables.
@@ -1356,41 +4161,9 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  273.0000000000000000 | 273.0000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
 (6 rows)
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
- Sort
-   Sort Key: t1.a
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
-               ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
-               ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
-                           Filter: (b = 0)
-(21 rows)
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
@@ -1405,26 +4178,24 @@ SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c
    Sort Key: t1.c
    ->  HashAggregate
          Group Key: t1.c, t2.c
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t2.c = t1.c)
-                     ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p1 t1
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on plt1_p4 t1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_1.c = t1_1.c)
-                     ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_p2 t1_1
+                           ->  Seq Scan on plt1_p1 t1_1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_2.c = t1_2.c)
-                     ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Seq Scan on plt1_p2 t1_2
                                  Filter: ((a % 25) = 0)
-(23 rows)
+                           ->  Seq Scan on plt1_p3 t1_3
+                                 Filter: ((a % 25) = 0)
+(21 rows)
 
 --
 -- multiple levels of partitioning
@@ -1826,64 +4597,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
  Hash Join
    Hash Cond: (t1.a = t2.a)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt4_n_p1 t2
                ->  Seq Scan on prt4_n_p2 t2_1
                ->  Seq Scan on prt4_n_p3 t2_2
-(11 rows)
+(13 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a and t1.a = t3.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                        QUERY PLAN                        
+----------------------------------------------------------
  Hash Join
-   Hash Cond: (t2.a = t1.a)
+   Hash Cond: (t3.b = t1.a)
    ->  Append
-         ->  Seq Scan on prt4_n_p1 t2
-         ->  Seq Scan on prt4_n_p2 t2_1
-         ->  Seq Scan on prt4_n_p3 t2_2
+         ->  Seq Scan on prt2_p0 t3
+         ->  Seq Scan on prt2_p1 t3_1
+         ->  Seq Scan on prt2_p2 t3_2
+         ->  Seq Scan on prt2_p3 t3_3
+         ->  Seq Scan on prt2_p4 t3_4
+         ->  Seq Scan on prt2_p5 t3_5
    ->  Hash
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.a = t3_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_2.b)
-                     ->  Seq Scan on prt1_p3 t1_2
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3 t3_2
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt4_n_p1 t2
+                           ->  Seq Scan on prt4_n_p2 t2_1
+                           ->  Seq Scan on prt4_n_p3 t2_2
 (23 rows)
 
 -- partitionwise join can not be applied if there are no equi-join conditions
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (t1.a < b)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (t1.a < b)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (t1.a < b)
-(12 rows)
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+(16 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partitionwise join
@@ -1969,16 +4746,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
-                     ->  Seq Scan on plt1_p1 t3
-                     ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3
+                     ->  Seq Scan on plt1_p1 t3_1
+                     ->  Seq Scan on plt1_p2 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partitionwise join can not be applied for a join between list and range
 -- partitioned table
@@ -1989,14 +4767,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
  Hash Full Join
    Hash Cond: ((t2.c)::text = (t1.c)::text)
    ->  Append
-         ->  Seq Scan on prt1_p1 t2
-         ->  Seq Scan on prt1_p2 t2_1
-         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p0 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p2 t2_2
+         ->  Seq Scan on prt1_p3 t2_3
+         ->  Seq Scan on prt1_p4 t2_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt1_n_p1 t1
                ->  Seq Scan on prt1_n_p2 t1_1
-(10 rows)
+(12 rows)
 
 -- partitionwise join can not be applied if only one of joining table has
 -- default partition
@@ -2012,16 +4792,23 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
    ->  Hash Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index c1c9859651..0e884835fb 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -10,25 +10,39 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -67,11 +81,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -93,20 +115,30 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 
 EXPLAIN (COSTS OFF)
@@ -169,6 +201,128 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -193,28 +347,79 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
 -- test partition matching with N-way join
@@ -222,6 +427,175 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
@@ -267,27 +641,18 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
 ALTER TABLE plt2 DETACH PARTITION plt2_p3;
 ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
 ANALYZE plt2;
-
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.a % 25 = 0 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
 --
 -- multiple levels of partitioning
 --
-- 
2.19.2

#62amul sul
sulamul@gmail.com
In reply to: Etsuro Fujita (#61)
3 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

There are few whitespaces in 0002 patch that I have fixed in the attached
version.
Rest of the patches are untouched.

Ill continue my review and testing.

Regards,
Amul

On Thu, Jan 31, 2019 at 5:26 PM Etsuro Fujita <fujita.etsuro@lab.ntt.co.jp>
wrote:

Show quoted text

(2019/01/22 21:38), Etsuro Fujita wrote:

Will continue to review.

I rebased the patch set against the latest HEAD. Attached is a new
version. I'll move this to the next CF, and continue to review it.

Best regards,
Etsuro Fujita

Attachments:

0001-Hash-partition-bound-equality-refactoring-v18.patchapplication/octet-stream; name=0001-Hash-partition-bound-equality-refactoring-v18.patchDownload
From a4b338f1225bcf9d5d3636f73537a5d1d2bee000 Mon Sep 17 00:00:00 2001
From: Etsuro Fujita <efujita@postgresql.org>
Date: Thu, 31 Jan 2019 19:18:18 +0900
Subject: [PATCH 1/3] Hash partition bound equality refactoring.

Separate the code to check whether two given hash bounds are equal into a
separate function, so that it can be called from multiple places. Right now
it's only caller is partition_bounds_equal() but later we will use it for
merging partition bounds.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/partitioning/partbounds.c | 95 +++++++++++++++++----------
 1 file changed, 61 insertions(+), 34 deletions(-)

diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index d478ae7e19..c492685d11 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -104,6 +104,9 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 						 Expr **keyCol,
 						 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static bool partition_hbounds_equal(PartitionBoundInfo b1,
+									PartitionBoundInfo b2);
+
 
 /*
  * get_qual_from_partbound
@@ -652,6 +655,63 @@ create_range_bounds(PartitionBoundSpec **boundspecs, int nparts,
 	return boundinfo;
 }
 
+/*
+ * Are two hash partition bound collections logically equal?
+ *
+ * Hash partition bounds store modulus and remainder in datums array which are
+ * always integers irrespective of the number of partition keys and their data
+ * types. Hence we can compare the hash bound collection without any partition
+ * key specific information. Separating this logic in a function which does not
+ * require partition key specific information allows it be called from places
+ * where the partition key specific information is not completely available.
+ */
+static bool
+partition_hbounds_equal(PartitionBoundInfo b1, PartitionBoundInfo b2)
+{
+	int			greatest_modulus = get_hash_partition_greatest_modulus(b1);
+	int			i;
+
+	Assert(b1->strategy == PARTITION_STRATEGY_HASH &&
+		   b2->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * If two hash partitioned tables have different greatest moduli,
+	 * their partition schemes don't match.  For hash partitioned table,
+	 * the greatest modulus is given by the last datum and number of
+	 * partitions is given by ndatums.
+	 */
+	if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
+		return false;
+
+	/*
+	 * We arrange the partitions in the ascending order of their modulus and
+	 * remainders.  Also every modulus is factor of next larger modulus.
+	 * Therefore we can safely store index of a given partition in indexes
+	 * array at remainder of that partition.  Also entries at (remainder + N *
+	 * modulus) positions in indexes array are all same for (modulus,
+	 * remainder) specification for any partition.  Thus datums array from both
+	 * the given bounds are same, if and only if their indexes array will be
+	 * same.  So, it suffices to compare indexes array.
+	 */
+	for (i = 0; i < greatest_modulus; i++)
+		if (b1->indexes[i] != b2->indexes[i])
+			return false;
+
+#ifdef USE_ASSERT_CHECKING
+
+	/*
+	 * Nonetheless make sure that the bounds are indeed same when the indexes
+	 * match.  Hash partition bound stores modulus and remainder at
+	 * b1->datums[i][0] and b1->datums[i][1] position respectively.
+	 */
+	for (i = 0; i < b1->ndatums; i++)
+		Assert((b1->datums[i][0] == b2->datums[i][0] &&
+				b1->datums[i][1] == b2->datums[i][1]));
+#endif
+
+	return true;
+}
+
 /*
  * Are two partition bound collections logically equal?
  *
@@ -680,41 +740,8 @@ partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
 
 	if (b1->strategy == PARTITION_STRATEGY_HASH)
 	{
-		int			greatest_modulus = get_hash_partition_greatest_modulus(b1);
-
-		/*
-		 * If two hash partitioned tables have different greatest moduli,
-		 * their partition schemes don't match.
-		 */
-		if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
+		if (!partition_hbounds_equal(b1, b2))
 			return false;
-
-		/*
-		 * We arrange the partitions in the ascending order of their moduli
-		 * and remainders.  Also every modulus is factor of next larger
-		 * modulus.  Therefore we can safely store index of a given partition
-		 * in indexes array at remainder of that partition.  Also entries at
-		 * (remainder + N * modulus) positions in indexes array are all same
-		 * for (modulus, remainder) specification for any partition.  Thus
-		 * datums array from both the given bounds are same, if and only if
-		 * their indexes array will be same.  So, it suffices to compare
-		 * indexes array.
-		 */
-		for (i = 0; i < greatest_modulus; i++)
-			if (b1->indexes[i] != b2->indexes[i])
-				return false;
-
-#ifdef USE_ASSERT_CHECKING
-
-		/*
-		 * Nonetheless make sure that the bounds are indeed same when the
-		 * indexes match.  Hash partition bound stores modulus and remainder
-		 * at b1->datums[i][0] and b1->datums[i][1] position respectively.
-		 */
-		for (i = 0; i < b1->ndatums; i++)
-			Assert((b1->datums[i][0] == b2->datums[i][0] &&
-					b1->datums[i][1] == b2->datums[i][1]));
-#endif
 	}
 	else
 	{
-- 
2.18.0

0002-Partition-wise-join-for-1-1-1-0-0-1-partition-matchi-v18.patchapplication/octet-stream; name=0002-Partition-wise-join-for-1-1-1-0-0-1-partition-matchi-v18.patchDownload
From 292460a9c6f490b74405e8ac44a2de275472b1ed Mon Sep 17 00:00:00 2001
From: Amul Sul <amul.sul@enterprisedb.com>
Date: Sun, 3 Feb 2019 23:51:15 -0500
Subject: [PATCH 2/3] Partition-wise join for 1:1, 1:0, 0:1 partition matching.

Earlier version of partition-wise join implementation allowed
partition-wise join between two relations with exactly same partition
bounds. This commit allows partition-wise join to be applied under
following conditions

1. the partition bounds of joining relations are such that rows from
given partition on one side join can join with rows from maximum one
partition on the other side i.e. bounds of a given partition on one
side match/overlap with those of maximum one partition on the other
side. If the mapping happens to be m:n where m > 1 or n > 1, we have
to gang multiple partition relations together into a single relation.
This means that we have to add simple relations during join
processing, something which is not supported right now.  ALso, in such
a case, different pairs of joining relations can produce different
partition bounds for the same join relation, which again is not
supported right now.

2. For every partition on outer side that can contribute to the result
of an OUTER join, there exists at least one (taken along with item 1,
it means exactly one)  matching partition on the inner side. To
support partition-wise join when the inner matching partition doesn't
exist, we have to add a dummy base relation corresponding to the
non-existent inner partition. We don't have support to add base
relations during join processing.

3. For hash partitioned tables however, we support partition-wise join
only when the bounds exactly match. For hash partitioning it's unusual
to have missing partitions and hence generic partition matching is not
required.

With advanced partition matching, we won't be able to apply
partition-wise join when there are missing matching partitions. So for
a given N-way join, some sub-joins may not be partitioned, while the
overall N-way join is partitioned. We can not try partition-wise join
when one or both of the joining relations are not partitioned in one
of the pairs of joining relation. Skip partition-wise join for that
pair.

An example is A IJ B LJ C where B has an extra partition compared to A
and C. Join B LJ C requires dummy relation to join the extra partition
from B to a missing partition from C, and hence can not use
partition-wise join right now and hence BC is not partitioned.  The
extra partition in B gets eliminated in A IJ B and thus it can use
partition-wise join and is partitioned. Hence (AB)C can use
partition-wise join but not A(BC).

Not every pair of joining relation (including the one presented to
build_joinrel_partition_info()) for the same joinrel can use
partition-wise join or has both the relations partitioned. Hence we
calculate the partition bounds for the join relation in
try_partition_wise_join() instead of doing it here.

The partition bounds produced for the first pair that can use
partition-wise are saved in the RelOptInfo of the joinrel. Partition
bounds produced for subsequent pairs should match that produced by the
first pair.

Support for default partition in list partitioned tables
========================================================

When a list value is present in one of the joining relations and not
the other, and the other relation has default partition, match (join)
the partition containing that list value with the default partition.
If there are multiple matches, we can not proceed with the
partition-wise join similar to the case of matching non-default
partition. If the default partition happens to be on the outer side of
the join, the resulting join partition acts as a default partition as
it will contain all the values from the default partition. If
the partition containing the list value happens to be on the outer
side of the join, the resulting join partition is associated with the
list value, since no other partition key value from the default
partition makes it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join.  Such a
default partition may get joined to a partition from the inner side,
if inner side has a list value that is not present in the outer side.
But if such a join partner does not exist, we won't be able to use
partition-wise join since there is no (dummy) relation on the inner
side to join to.

Support for matching default partition in range partitioned tables
==================================================================

When a range is present in one of the joining relations and not the
other, and the other relation has default partition, match (join) the
partition corresponding range with the default partition. If two
ranges overlap but their ranges don't match exactly, the
non-overlapping portion from one range may join the previous
(non-overlapping portion near the lower bound, if exists), next
(non-overlapping portion near the upper bound, if exists) ranges from
the other relation or default partition from the other relation. This
causes multiple partitions on the other side to be joined with a
single partition on the first side; a case we don't support. Any range
from one side which doesn't overlap with any range on the other side,
may find its join partner in the default partition and the
corresponding range partition will need to be joined with the default
partition.  If the default partition happens to be on the outer side
of the join, the resulting join partition acts as a default partition
as it will contain all the values from the default partition. If the
non-partition corresponding to the range happens to be on the outer
side of the join, the resulting join partition is associated with that
range, since partition key values from the default partition outside
that range won't make it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join.  Such a
default partition may get joined to a non-default partition from the
inner side, if inner side has a range missing in the outer side.  But
if such a join partner does not exist, we won't be able to use
partition-wise join since there is no (dummy) relation on the inner
side to join to.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/optimizer/path/joinrels.c |   95 +-
 src/backend/optimizer/util/relnode.c  |   33 +-
 src/backend/partitioning/partbounds.c | 1677 ++++++++++++++++++++++++-
 src/include/partitioning/partbounds.h |    7 +
 4 files changed, 1766 insertions(+), 46 deletions(-)

diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index dfbbfdac6d..ca4d261bc1 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1320,25 +1320,31 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	PartitionScheme part_scheme;
+	PartitionBoundInfo join_boundinfo;
+	List	   *parts1;
+	List	   *parts2;
+	ListCell   *lc1;
+	ListCell   *lc2;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1351,32 +1357,63 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	 */
 	Assert(joinrel->part_scheme == rel1->part_scheme &&
 		   joinrel->part_scheme == rel2->part_scheme);
+	part_scheme = joinrel->part_scheme;
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * Get the list of matching partitions to be joined along with the
+	 * partition bounds of the join relation. Because of the restrictions
+	 * imposed by partition matching algorithm, not every pair of joining
+	 * relations for this join will be able to use partition-wise join. But all
+	 * those pairs which can use partition-wise join will produce the same
+	 * partition bounds for the join relation.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	join_boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											part_scheme->partsupfunc,
+											part_scheme->partcollation,
+											rel1, rel2,
+											parent_sjinfo->jointype,
+											&parts1, &parts2);
+
+	if (join_boundinfo == NULL)
+		return;
 
-	nparts = joinrel->nparts;
+	if (joinrel->boundinfo == NULL)
+	{
+		Assert(joinrel->nparts == 0 && joinrel->part_rels == NULL);
+		joinrel->boundinfo = join_boundinfo;
+		joinrel->nparts = list_length(parts1);
+		Assert(joinrel->nparts == list_length(parts2));
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) *
+									joinrel->nparts);
+	}
+	else
+	{
+		Assert(partition_bounds_equal(part_scheme->partnatts,
+									  part_scheme->parttyplen,
+									  part_scheme->parttypbyval,
+									  join_boundinfo, joinrel->boundinfo));
+		/*
+		 * Every pair of joining relations should result in the same number
+		 * of child-joins.
+		 */
+		Assert(joinrel->nparts == list_length(parts1));
+		Assert(joinrel->nparts == list_length(parts2));
+		Assert(joinrel->part_rels);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	cnt_parts = 0;
+	forboth(lc1, parts1, lc2, parts2)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
+		int			part1 = lfirst_int(lc1);
+		int			part2 = lfirst_int(lc2);
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1384,6 +1421,10 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		Assert(part1 >= 0 && part2 >= 0);
+		child_rel1 = rel1->part_rels[part1];
+		child_rel2 = rel2->part_rels[part2];
+
 		/*
 		 * If a child table has consider_partitionwise_join=false, it means
 		 * that it's a dummy relation for which we skipped setting up tlist
@@ -1438,12 +1479,24 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 			joinrel->part_rels[cnt_parts] = child_joinrel;
 		}
 
+		/*
+		 * For every pair of joining relations, the set of matching partitions
+		 * would change. However, the base relation partitions constituting
+		 * the given child should remain same for all the joining pairs. Since
+		 * the order in which children are stored in the array of child-joins,
+		 * depends upon partition bounds of the join, which are same for all
+		 * the joining pairs, every joining pair yields the child-joins in the
+		 * same order.
+		 */
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
 
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
+		cnt_parts++;
 	}
+
+	Assert(cnt_parts == joinrel->nparts);
 }
 
 /*
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index f04c6b76f4..a1561dd066 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1585,7 +1585,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1598,24 +1598,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1627,17 +1609,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index c492685d11..42287100e4 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -22,6 +22,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/pathnodes.h"
 #include "parser/parse_coerce.h"
 #include "partitioning/partprune.h"
 #include "partitioning/partbounds.h"
@@ -65,6 +66,12 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+typedef struct PartitionMap
+{
+	int from;
+	int to;
+} PartitionMap;
+
 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);
@@ -106,7 +113,58 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 static List *get_range_nulltest(PartitionKey key);
 static bool partition_hbounds_equal(PartitionBoundInfo b1,
 									PartitionBoundInfo b2);
-
+static PartitionBoundInfo partition_range_bounds_merge(
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 List **outer_parts, List **inner_parts,
+							 JoinType jointype, int partnatts,
+							 FmgrInfo *supfuncs, Oid *collations);
+static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype);
+static PartitionBoundInfo partition_hash_bounds_merge(RelOptInfo *outer_rel,
+							RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype);
+static void generate_matching_part_pairs(PartitionMap *outer_maps,
+										 PartitionMap *inner_maps,
+										 int nparts1, int nparts2,
+										 JoinType jointype, int nparts,
+										 List **parts1, List **parts2);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int map_and_merge_partitions(PartitionMap *outer_maps,
+										PartitionMap *inner_maps,
+										int index1, int index2, int *next_index);
+static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *collations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2);
+static bool partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
+						   Oid *collations, PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, int *ub_cmpval,
+						   int *lb_cmpval);
+static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes);
+static bool merge_default_partitions(PartitionBoundInfo outer_bi,
+						 PartitionBoundInfo inner_bi,
+						 PartitionMap *outer_maps,
+						 PartitionMap *inner_maps,
+						 JoinType jointype,
+						 int *next_index, int *default_index);
+static bool merge_null_partitions(PartitionBoundInfo outer_bi,
+								  PartitionBoundInfo inner_bi,
+								  PartitionMap *outer_maps,
+								  PartitionMap *inner_maps,
+								  JoinType jointype,
+								  int *next_index, int *null_index,
+								  int *default_index);
 
 /*
  * get_qual_from_partbound
@@ -2941,3 +2999,1620 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * The function produces the partition bounds for a join between two relations
+ * whose partition bounds are given. The function also returns two lists of
+ * partition indexes one for each of the joining relations. Both the lists
+ * contain the same number of elements. The partition indexes at the same
+ * positions in the lists indicate the pair partitions, one from each side, to
+ * be joined and the position itself corresponds to the index of partition
+ * produced by that child-join in the partitioned join.
+ *
+ * The function returns NULL if we can not find the matching pair of
+ * partitions. This happens if 1. multiple partitions on one side match with
+ * one partition on the other side. 2. a given partition on the outer side
+ * doesn't have a matching partition on the inner side. We can not support the
+ * first case since we don't have a way to represent multiple partitions as a
+ * single relation (RelOptInfo) and then perform join using the ganged
+ * relation. We can not support the second case since the missing inner
+ * partition needs to be represented as an empty relation and we don't have a
+ * way to introduce empty relation during join planning after creating paths
+ * for all the base relations.
+ */
+extern PartitionBoundInfo
+partition_bounds_merge(int partnatts, FmgrInfo *partsupfunc,
+					   Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
+{
+	PartitionBoundInfo 	merged_bounds;
+	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
+						inner_binfo = inner_rel->boundinfo;
+	char				strategy = outer_binfo->strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partsupfunc,
+														partcollation,
+														outer_rel, inner_rel,
+														outer_parts, inner_parts,
+														jointype);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(outer_rel, inner_rel,
+														 outer_parts, inner_parts,
+														 jointype, partnatts,
+														 partsupfunc,
+														 partcollation);
+			break;
+
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = partition_hash_bounds_merge(outer_rel, inner_rel,
+														outer_parts, inner_parts,
+														jointype);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	Assert((*outer_parts == NIL || *inner_parts != NIL) &&
+		   (*inner_parts == NIL || *outer_parts != NIL));
+
+	return merged_bounds;
+}
+
+/*
+ * partition_get_range_bounds
+ *
+ * Given the index of lower bound in datums array, return lower and upper
+ * bounds and the index of the partition with that lower bound.
+ */
+static int
+partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
+						   PartitionRangeBound *lower,
+						   PartitionRangeBound *upper)
+{
+	int			part_index;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The lower bound should correspond to a valid partition. */
+	part_index = bi->indexes[lb_index + 1];
+	Assert(part_index >= 0);
+
+	lower->kind = bi->kind[lb_index];
+	lower->datums = bi->datums[lb_index];
+	lower->lower = true;
+	upper->kind = bi->kind[lb_index + 1];
+	upper->datums = bi->datums[lb_index + 1];
+	upper->lower = false;
+
+	return part_index;
+}
+
+/*
+ * partition_range_get_next_lb_index
+ *
+ * Given the index of lower bound in datums array return the
+ * index of lower bound of the next partition. When the given index corresponds
+ * to the last partition, return number of datums (ndatums).
+ */
+static int
+partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
+{
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The partition index corresponding to the upper bound should be valid. */
+	Assert(bi->indexes[lb_index + 1] >= 0);
+
+	/*
+	 * If there are no bounds left beyond the upper bound, we have reached the
+	 * last partition.
+	 */
+	if (lb_index + 2 < bi->ndatums)
+	{
+		/*
+		 * If the bound next to the upper bound corresponds to no partition,
+		 * that's the next lower bound of the next partition. Otherwise, the
+		 * current upper bound is the lower bound of the next partition.
+		 */
+		if (bi->indexes[lb_index + 2] < 0)
+			return lb_index + 2;
+		else
+			return lb_index + 1;
+	}
+	else
+		return bi->ndatums;
+}
+
+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *partcollations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2)
+{
+	return partition_rbound_cmp(partnatts, partsupfunc, partcollations,
+								bound1->datums, bound1->kind, bound1->lower,
+								bound2);
+}
+
+/*
+ * partition_range_cmp
+ *
+ * Compare the bounds of two range partitions. Set ub_cmpval <, = or > 0, if the
+ * first partition's upper bound is lower than, equal to or higher than the
+ * second partition's upper bound resp. Similarly set lb_cmpval <, =  or > 0,
+ * if the first partition's lower bound is lower than, equal to or higher than
+ * the second partition's lower bound resp.
+ *
+ * Return true, if the ranges overlap, otherwise return false.
+ */
+static bool
+partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
+					PartitionRangeBound *lower_bound1,
+					PartitionRangeBound *upper_bound1,
+					PartitionRangeBound *lower_bound2,
+					PartitionRangeBound *upper_bound2, int *ub_cmpval,
+					int *lb_cmpval)
+{
+	bool		overlap;
+
+	/*
+	 * Compare upper bound of the first partition with the lower bound of the
+	 * second and vice-versa. If lower bound is higher than the upper bound,
+	 * the partitions are not overlapping. All other cases indicate overlapping
+	 * partitions.
+	 */
+	if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+								  lower_bound1, upper_bound2) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = 1;
+		*lb_cmpval = 1;
+	}
+	else if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+									   lower_bound2, upper_bound1) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = -1;
+		*lb_cmpval = -1;
+	}
+	else
+	{
+		overlap = true;
+		*ub_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, upper_bound1,
+											   upper_bound2);
+		*lb_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, lower_bound1,
+											   lower_bound2);
+	}
+
+	return overlap;
+}
+
+/*
+ * partition_range_merge
+ *
+ * Merge the partition bounds of given two partitions such that the join
+ * between the given two partitions fits merged bounds.
+ *
+ * "merged_upper" will be set to one of the given upper bounds and
+ * "merged_lower" will be set to one of the given lower bounds.
+ */
+static void
+partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
+					  Oid *partcollations, JoinType jointype,
+					  PartitionRangeBound *left_lb,
+					  PartitionRangeBound *left_ub,
+					  PartitionRangeBound *right_lb,
+					  PartitionRangeBound *right_ub,
+					  PartitionRangeBound **merged_lb,
+					  PartitionRangeBound **merged_ub)
+{
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = left_ub;
+			*merged_lb = left_lb;
+			break;
+
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) < 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) > 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		case JOIN_FULL:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) > 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) < 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+
+	return;
+}
+
+/*
+ * Add the lower bound of the next range to the list of bounds, if the lower
+ * bound is higher or equal to the previous upper bound. If successful return
+ * true, otherwise false.
+ */
+static bool
+partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
+							  Oid *partcollations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		prev_ub.datums = llast(*merged_datums);
+		prev_ub.kind = llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  next_lb_datums, next_lb_kind, false,
+									  &prev_ub);
+	}
+
+	/*
+	 * The lower bound is lower than the last upper bound, thus does not fit
+	 * the bounds created so far and hence can not be merged with the existing
+	 * bounds.
+	 */
+	if (cmpval < 0)
+		return false;
+
+	/*
+	 * Add bounds of the new merged partition. If the next lower bound is
+	 * higher than the last upper bound, add new range with index
+	 * corresponding to the lower bound as -1. If the merged lower bound
+	 * is same as the last merged upper bound, the last upper bound will be
+	 * reused as the lower bound of the next range.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, next_lb_datums);
+		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	return true;
+}
+
+/*
+ * handle_missing_partition
+ *
+ * If a range appears in one of the joining relations but not the other, a row
+ * in the corresponding partition will not have any join partner in the other
+ * relation, unless the other relation has a default partition. If a given list
+ * value is present in one joining relation but not the other, the default
+ * partition on the other side may contain that value.
+ *
+ * In both these cases, such an extra partition forms a joining pair with the
+ * default partition, if any,  on the other side.
+ *
+ * If the default partition happens to be on the outer side of the join, the
+ * resultant partition will act as the default partition of the join relation.
+ * Otherwise the resultant partition will be associated with the range.
+ *
+ * When the default partition is not present in the other relation, the rows in
+ * the extra partition will be included in the bounds of the join result, if it
+ * appears on the outer side of the join, since all rows from the outer side
+ * are included in the join result.
+ *
+ * This function handles all these cases.
+ *
+ * maps_with_missing and missing_side_default are the partition maps (See
+ * partition_range/list_bounds_merge() for details) and the index of default
+ * partition respectively corresponding the side with missing partition.
+ *
+ * maps_with_extra and extra_part are the partition maps (See
+ * partition_range/list_bounds_merge() for details) and the index of extra
+ * partition respectively corresponding to the side with the extra partition.
+ *
+ * It returns true if the matching succeeds, otherwise returns false.
+ */
+static bool
+handle_missing_partition(PartitionMap *maps_with_missing,
+						 PartitionMap *maps_with_extra,
+						 int missing_side_default,
+						 int extra_part,
+						 bool missing_side_outer,
+						 bool missing_side_inner,
+						 int *next_index, int *default_index,
+						 int *merged_index)
+{
+	bool missing_has_default = (missing_side_default != -1);
+
+	if (missing_has_default)
+	{
+		*merged_index = map_and_merge_partitions(maps_with_missing,
+												 maps_with_extra,
+												 missing_side_default,
+												 extra_part,
+												 next_index);
+		if (*merged_index < 0)
+			return false;
+
+		if (missing_side_outer)
+		{
+			/*
+			 * Default partition on the outer side forms the default
+			 * partition of the join result.
+			 */
+			if (*default_index < 0)
+				*default_index = *merged_index;
+			else if(*default_index != *merged_index)
+			{
+				/*
+				 * Ended up with default partition on the outer side
+				 * being joined with multiple partitions on the inner
+				 * side. We don't support this case.
+				 */
+				return false;
+			}
+
+			/*
+			 * Since the merged partition acts as a default partition, it
+			 * doesn't need a separate index.
+			 */
+			*merged_index = -1;
+		}
+	}
+	else if (missing_side_inner)
+	{
+		/*
+		 * If this partition has already been mapped (say because we
+		 * found an overlapping range earlier), we know where does it
+		 * fit in the join result. Nothing to do in that case. Else
+		 * create a new merged partition.
+		 */
+		PartitionMap *extra_map = &maps_with_extra[extra_part];
+		if (extra_map->to < 0)
+		{
+			extra_map->to = *next_index;
+			*next_index = *next_index + 1;
+			*merged_index = extra_map->to;
+		}
+	}
+
+	return true;
+}
+
+static PartitionMap*
+init_partition_map(RelOptInfo *rel)
+{
+	int i, nparts = rel->nparts;
+	PartitionMap *map;
+
+	map = (PartitionMap *) palloc(sizeof(PartitionMap) * nparts);
+
+	for (i = 0; i < nparts; i++)
+	{
+		map[i].from = -1;
+		map[i].to = -1;
+	}
+
+	return map;
+}
+
+/*
+ * Allocate and initialize partition maps. We maintain four maps, two maps
+ * for each joining relation. pmap[i] gives the partition from the other
+ * relation which would join with ith partition of the given relation.
+ * Partition i from the given relation will join with partition pmap[i]
+ * from the other relation to produce partition mmap[i] of the join (merged
+ * partition).
+ *
+ * pmap[i] = -1 indicates that ith partition of a given relation does not
+ * have a matching partition from the other relation.
+ *
+ * mmap[i] = -1 indicates that ith partition of a given relation does not
+ * contribute to the join result. That can happen only when the given
+ * relation is the inner relation and it doesn't have a matching partition
+ * from the outer relation, hence pmap[i] should be -1.
+ *
+ * In case of an outer join, every partition of the outer join will appear
+ * in the join result, and thus has mmap[i] set for all i. But it's not
+ * necessary that every partition on the outer side will have a matching
+ * partition on the inner side. In such a case, we end up with pmap[i] = -1
+ * and mmap[i] != -1.
+ */
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 List **outer_parts, List **inner_parts,
+							 JoinType jointype, int partnatts,
+							 FmgrInfo *partsupfuncs, Oid *partcollations)
+
+{
+	PartitionMap *outer_maps = NULL;
+	PartitionMap *inner_maps = NULL;
+	int			outer_part = 0;
+	int			inner_part = 0;
+	PartitionBoundInfo merged_bounds = NULL;
+	int			outer_lb_index;
+	int			inner_lb_index;
+	int			next_index;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	List	   *merged_kinds = NIL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			inner_default = inner_bi->default_index;
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	outer_maps = init_partition_map(outer_rel);
+	inner_maps = init_partition_map(inner_rel);
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = 0;
+	inner_lb_index = 0;
+	next_index = 0;
+	while (outer_lb_index < outer_bi->ndatums ||
+		   inner_lb_index < inner_bi->ndatums)
+	{
+		PartitionRangeBound outer_lb, outer_ub,
+							inner_lb, inner_ub,
+							*merged_lb = NULL,
+							*merged_ub = NULL;
+
+		int			merged_index = -1;
+		bool		overlap;
+		bool		finished_outer = false;
+		bool		finished_inner = false;
+
+		/* Result of bounds comparison per partition_rbound_cmp(). */
+		int			ub_cmpval;	/* Upper bounds comparison result. */
+		int			lb_cmpval;	/* Lower bounds comparison result. */
+
+		/* Get the range bounds of the next pair of partitions. */
+		if (outer_lb_index < outer_bi->ndatums)
+			outer_part = partition_get_range_bounds(outer_bi, outer_lb_index,
+												&outer_lb, &outer_ub);
+		else
+			finished_outer = true;
+
+		if (inner_lb_index < inner_bi->ndatums)
+			inner_part = partition_get_range_bounds(inner_bi, inner_lb_index,
+												&inner_lb, &inner_ub);
+		else
+			finished_inner = true;
+
+		Assert(!finished_outer || !finished_inner);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (finished_outer)
+		{
+			overlap = false;
+			ub_cmpval = 1;
+			lb_cmpval = 1;
+		}
+		else if (finished_inner)
+		{
+			overlap = false;
+			ub_cmpval = -1;
+			lb_cmpval = -1;
+		}
+		else
+			overlap = partition_range_cmp(partnatts, partsupfuncs, partcollations,
+										  &outer_lb, &outer_ub, &inner_lb,
+										  &inner_ub, &ub_cmpval, &lb_cmpval);
+
+		if (overlap)
+		{
+			/*
+			 * The rows from overlapping portion of ranges on both sides may
+			 * join, hence the corresponding pair of partitions form a joining
+			 * pair. Match them and produce the bounds of the joint partition
+			 * and its index by merging the bounds according to the type of
+			 * join.
+			 */
+			partition_range_merge(partnatts, partsupfuncs, partcollations,
+								  jointype, &outer_lb, &outer_ub, &inner_lb,
+								  &inner_ub, &merged_lb, &merged_ub);
+
+			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+													outer_part, inner_part,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				/* Failed to match the partitions. */
+				return NULL;
+			}
+
+			/*
+			 * If the ranges overlap but don't exactly match, a row from
+			 * non-overlapping portion of the range from one side of join may
+			 * find its join partner in the previous or next overlapping
+			 * partition or default partition on the other side , if such a
+			 * partition exists. All those cases, if true, will cause one
+			 * partition from that side to match at least two partitions on the
+			 * other side; a case that we do not support now. Previous
+			 * partition has been delt with in the previous iteration of this
+			 * loop, next partition will be delt in the next iteration. We will
+			 * deal with the default partition here.
+			 */
+			if ((lb_cmpval < 0 && inner_has_default) ||
+				/* Non-overlapping range on the lower side of outer range. */
+				(lb_cmpval > 0 && outer_has_default) ||
+				/* Non-overlapping range on the lower side of inner range. */
+				(ub_cmpval < 0 && outer_has_default) ||
+				/* Non-overlapping range on the upper side of inner range. */
+				(ub_cmpval > 0 && inner_has_default))
+				/* Non-overlapping range on the upper side of outer range. */
+				return NULL;
+		}
+
+		if (ub_cmpval == 0)
+		{
+			/* Upper bounds of both the ranges match. */
+			Assert(overlap);
+
+			/* Move to the next pair of partitions. */
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the inner side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &outer_lb;
+				merged_ub = &outer_ub;
+
+				/*
+				 * For a FULL join, inner relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the inner relation acts as
+				 * INNER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_inner = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_outer = (jointype == JOIN_FULL);
+				if (!handle_missing_partition(inner_maps,
+											  outer_maps,
+											  inner_default,
+											  outer_part,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the outer side. */
+			Assert(!finished_outer);
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+		}
+		else
+		{
+			Assert(ub_cmpval > 0);
+
+			/* Upper bound of outer range higher than that of the inner. */
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the outer side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &inner_lb;
+				merged_ub = &inner_ub;
+
+				/*
+				 * For a FULL join, outer relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the outer relation acts as
+				 * OUTER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_outer = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_inner = (jointype == JOIN_FULL);
+
+				if (!handle_missing_partition(outer_maps,
+											  inner_maps,
+											  outer_default,
+											  inner_part,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the inner side. */
+			Assert (!finished_inner);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+
+		if (merged_index < 0)
+		{
+			/* We didn't find a new merged partition. */
+			continue;
+		}
+
+		/*
+		 * We have a valid partition index for the next partition of join. The
+		 * partition should have valid range.
+		 */
+		Assert(merged_lb && merged_ub);
+
+		/* Try merging new lower bound with the last upper bound. */
+		if (!partition_range_merge_next_lb(partnatts, partsupfuncs,
+										   partcollations,
+										   merged_lb->datums,
+										   merged_lb->kind, &merged_datums,
+										   &merged_kinds, &merged_indexes))
+			return NULL;
+
+		/* Add upper bound with the merged partition index. */
+		merged_datums = lappend(merged_datums, merged_ub->datums);
+		merged_kinds = lappend(merged_kinds, merged_ub->kind);
+		merged_indexes = lappend_int(merged_indexes, merged_index);
+	}
+
+	if (!merge_default_partitions(outer_bi, inner_bi,
+										  outer_maps, inner_maps,
+										  jointype, &next_index,
+										  &default_index))
+		return NULL;
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_maps, inner_maps,
+								 outer_nparts, inner_nparts,
+								 jointype, next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes,
+													  merged_kinds,
+													  -1, default_index);
+	}
+
+	/* Free any memory we used in this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype)
+{
+	PartitionMap *outer_maps = NULL;
+	PartitionMap *inner_maps = NULL;
+	int			cnto;
+	int			cnti;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			next_index = 0;
+	int			null_index;
+	int			default_index = -1;
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			      *outer_indexes = outer_bi->indexes;
+	int			      *inner_indexes = inner_bi->indexes;
+	int				   outer_default = outer_bi->default_index;
+	int				   inner_default = inner_bi->default_index;
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	outer_maps = init_partition_map(outer_rel);
+	inner_maps = init_partition_map(inner_rel);
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			o_index;
+		int			i_index;
+		int			cmpval;
+		int			merged_index = -1;
+		Datum	   *merged_datum;
+		bool		finished_inner;
+		bool		finished_outer;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+		{
+			finished_outer = true;
+			odatums = NULL;
+			o_index = -1;
+		}
+		else
+		{
+			finished_outer = false;
+			odatums = outer_bi->datums[cnto];
+			o_index = outer_indexes[cnto];
+		}
+
+		if (cnti >= inner_bi->ndatums)
+		{
+			finished_inner = true;
+			idatums = NULL;
+			i_index = -1;
+		}
+		else
+		{
+			finished_inner = false;
+			idatums = inner_bi->datums[cnti];
+			i_index = inner_indexes[cnti];
+		}
+
+		/* If we exhausted both the sides, we won't enter the loop. */
+		Assert(!finished_inner || !finished_outer);
+
+		if (finished_outer)
+			cmpval = 1;
+		else if (finished_inner)
+			cmpval = -1;
+		else
+		{
+			/* Every list datum should map to a valid partition index. */
+			Assert(o_index >= 0 && i_index >= 0 &&
+				   odatums != NULL && idatums != NULL);
+
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0], idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+													o_index, i_index,
+													&next_index);
+
+			if (merged_index < 0)
+				return NULL;
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			/* A datum missing from the inner side. */
+			merged_index = -1;
+			merged_datum = odatums;
+
+			/*
+			 * For a FULL join, inner relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the inner relation acts as
+			 * INNER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_inner = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_outer = (jointype == JOIN_FULL);
+
+			if (!handle_missing_partition(inner_maps,
+										  outer_maps,
+										  inner_default,
+										  o_index,
+										  missing_side_outer,
+										  missing_side_inner,
+										  &next_index,
+										  &default_index,
+										  &merged_index))
+				return NULL;
+
+			/* Move to the next datum on the outer side. */
+			Assert(!finished_outer);
+			cnto++;
+		}
+		else
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			Assert(cmpval > 0);
+
+			/* A datum missing from the outer side. */
+			merged_index = -1;
+			merged_datum = idatums;
+
+			/*
+			 * For a FULL join, outer relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the outer relation acts as
+			 * OUTER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_outer = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_inner = (jointype == JOIN_FULL);
+
+			if (!handle_missing_partition(outer_maps,
+										  inner_maps,
+										  outer_default,
+										  i_index,
+										  missing_side_outer,
+										  missing_side_inner,
+										  &next_index,
+										  &default_index,
+										  &merged_index))
+				return NULL;
+
+			/* Move to the next datum on the right side. */
+			Assert(!finished_inner);
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	if (!merge_null_partitions(outer_bi, inner_bi,
+							   outer_maps, inner_maps,
+							   jointype, &next_index, &null_index,
+							   &default_index))
+		return NULL;
+
+	if (!merge_default_partitions(outer_bi, inner_bi,
+								  outer_maps, inner_maps,
+								  jointype, &next_index,
+								  &default_index))
+		return NULL;
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_maps, inner_maps,
+								 outer_nparts, inner_nparts,
+								 jointype, next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes, NIL,
+													  null_index, default_index);
+	}
+
+	/* Free up all extra memory before returning from this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_bounds_merge()'s arm for hash partitioned tables.
+ *
+ * If the given two hash bounds are same, the function returns the first one
+ * without any change, alongwith the lists of matching partitions. Otherwise it
+ * returns NULL.
+ *
+ * We could try merging the bounds when both the bounds have same greatest
+ * modulii. But there seems to be hardly any requirement for the same.
+ */
+static PartitionBoundInfo
+partition_hash_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype)
+{
+	int			nparts;
+	int			cnt;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * Hash partitioned table does not have explicit NULL accepting partition
+	 * and also does not have a default partition.
+	 */
+	Assert(!partition_bound_has_default(outer_bi) &&
+		   !partition_bound_has_default(inner_bi));
+	Assert(!partition_bound_accepts_nulls(outer_bi) &&
+		   !partition_bound_accepts_nulls(inner_bi));
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+
+	if (outer_nparts != inner_nparts)
+		return NULL;
+	nparts = outer_nparts;
+
+	if (outer_bi->ndatums != inner_bi->ndatums ||
+		!partition_hbounds_equal(outer_bi, inner_bi))
+		return NULL;
+
+	 /*
+	  * Cook up list of matching partitions. Since bounds are exactly same the
+	  * partitions at the same position from both the relations match.
+	  */
+	for (cnt = 0; cnt < nparts; cnt++)
+	{
+		*outer_parts = lappend_int(*outer_parts, cnt);
+		*inner_parts = lappend_int(*inner_parts, cnt);
+	}
+
+	return outer_bi;
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * If the two given partitions (given by index1 and index2 resp.) are
+ * already mapped to each other return the index of corresponding partition in
+ * the merged set of partitions.  If they do not have a merged partition
+ * associated with them, assign a new merged partition index.  If the
+ * partitions are already mapped and their mapped partitions are different from
+ * each other, they can not be merged, so return -1.
+ *
+ * partmaps1[i] gives the mapping of partitions for both relations. It
+ * describes which partition of relation 2 matches ith partition of relation 1,
+ * and which partition in the merged set matches ith partition of relation 1
+ * maps to. Similarly for partmap2.
+ *
+ * index1 and index2 are the indexes of matching partition from respective
+ * relations.
+ *
+ * *next_index is used and incremented when the given partitions require a new
+ * merged partition.
+ */
+
+static int
+map_and_merge_partitions(PartitionMap *partmaps1, PartitionMap *partmaps2,
+						 int index1, int index2, int *next_index)
+{
+	PartitionMap 	*partmap1 = &partmaps1[index1];
+	PartitionMap 	*partmap2 = &partmaps2[index2];
+	int				merged_index;
+
+	/*
+	 * If both the partitions are not mapped to each other, update the
+	 * maps.
+	 */
+	if (partmap1->from < 0 && partmap2->from < 0)
+	{
+		partmap1->from = index2;
+		partmap2->from = index1;
+	}
+
+	/*
+	 * If the given to partitions map to each other, find the corresponding
+	 * merged partition index .
+	 */
+	if (partmap1->from == index2 && partmap2->from == index1)
+	{
+		/*
+		 * If both the partitions are mapped to the same merged partition, get
+		 * the index of merged partition.
+		 */
+		if (partmap1->to == partmap2->to)
+		{
+			merged_index = partmap1->to;
+
+			/*
+			 * If the given two partitions do not have a merged partition
+			 * associated with them, allocate a new merged partition.
+			 */
+			if (merged_index < 0)
+			{
+				merged_index = *next_index;
+				*next_index = *next_index + 1;
+				partmap1->to = merged_index;
+				partmap2->to = merged_index;
+			}
+		}
+
+		/*
+		 * If partition from one relation was mapped to a merged partition but
+		 * not the partition from the other relation, map the same merged
+		 * partition to the partition from other relation, since matching
+		 * partitions map to the same merged partition.
+		 */
+		else if (partmap1->to >= 0 && partmap2->to < 0)
+		{
+			partmap2->to = partmap1->to;
+			merged_index = partmap1->to;
+		}
+		else if (partmap1->to < 0 && partmap2->to >= 0)
+		{
+			partmap1->to = partmap2->to;
+			merged_index = partmap2->to;
+		}
+		else
+		{
+			Assert(partmap1->to != partmap2->to &&
+				   partmap1->to >= 0 && partmap2->to >= 0);
+
+			/*
+			 * Both the partitions map to different merged partitions. This
+			 * means that multiple partitions from one relation matches to one
+			 * partition from the other relation. Partition-wise join does not
+			 * handle this case right now, since it requires ganging multiple
+			 * partitions together (into one RelOptInfo).
+			 */
+			merged_index = -1;
+		}
+	}
+	else
+	{
+		/*
+		 * Multiple partitions from one relation map to one partition from the
+		 * other relation. Partition-wise join does not handle this case right
+		 * now, since it requires ganging multiple partitions together (into
+		 * one RelOptInfo).
+		 */
+		merged_index = -1;
+	}
+
+	return merged_index;
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * partmaps1 map each partition from either side of the join to a merged
+ * partition resp. E.g. partmaps1[i].to gives the merged partition to which ith
+ * partition of first relation maps. Similarly for partmap2. If
+ * partmaps1[i].to == partmaps2[j].to, i and j form the matching pair of
+ * partitions.
+ *
+ * Given these maps this function produces the list pairs of partitions which
+ * when joined produce the merged partitions in the order of merged partition
+ * indexes.
+ *
+ * nparts1 and nparts2 are the number of partitions of the joining relations
+ * resp.
+ *
+ * nparts is the number of merged partitions.
+ *
+ * If successful, the pairs of partitions are returned as two separate lists,
+ * parts1 and parts2 resp., one for each side. Otherwise, those lists will be
+ * set to NIL.
+ */
+static void
+generate_matching_part_pairs(PartitionMap *partmaps1, PartitionMap *partmaps2,
+							 int nparts1, int nparts2,
+							 JoinType jointype, int nparts,
+							 List **matched_parts1, List **matched_parts2)
+{
+	bool	merged = true;
+	int		*matching1 = (int *) palloc(sizeof(int) * nparts),
+			*matching2 = (int *) palloc(sizeof(int) * nparts);
+	int 	i;
+
+	*matched_parts1 = NIL;
+	*matched_parts2 = NIL;
+
+	/* Set pairs of matching partitions. */
+	for (i = 0; i < nparts; i++)
+	{
+		if (i >= nparts1)
+			matching1[i] = -1;
+		else
+		{
+			PartitionMap outer_map = partmaps1[i];
+
+			if (outer_map.to >= 0)
+			{
+				Assert(outer_map.to < nparts);
+				matching1[outer_map.to] = i;
+			}
+		}
+
+		if (i >= nparts2)
+			matching2[i] = -1;
+		else
+		{
+			PartitionMap inner_map = partmaps2[i];
+
+			if (inner_map.to >= 0)
+			{
+				Assert(inner_map.to < nparts);
+				matching2[inner_map.to] = i;
+			}
+		}
+	}
+
+	/*
+	 * If we have a partition missing on an inner side, we need to add a dummy
+	 * relation which joins with the outer partition. If the inner relation
+	 * happens to be a base relation, it will require adding a dummy child
+	 * base relation during join processing. Right now, we freeze the base
+	 * relation arrays like PlannerInfo::simple_rte_array after planning for
+	 * base relations. Adding a new (dummy) base relation would require some
+	 * changes to that. So, right now, we do not implement partition-wise join
+	 * in such cases.
+	 */
+	for (i = 0; i < nparts; i++)
+	{
+		int			part1 = matching1[i];
+		int			part2 = matching2[i];
+
+		/* At least one of the partitions should exist. */
+		Assert(part1 >= 0 || part2 >= 0);
+
+		switch (jointype)
+		{
+			case JOIN_INNER:
+			case JOIN_SEMI:
+
+				/*
+				 * An inner or semi join can not return any row when the
+				 * matching partition on either side is missing. We should
+				 * have eliminated all such cases while merging the bounds.
+				 */
+				Assert(part1 >= 0 && part2 >= 0);
+				break;
+
+			case JOIN_LEFT:
+			case JOIN_ANTI:
+				Assert(part1 >= 0);
+				if (part2 < 0)
+					merged = false;
+				break;
+
+			case JOIN_FULL:
+				if (part1 < 0 || part2 < 0)
+					merged = false;
+				break;
+
+			default:
+				elog(ERROR, "unrecognized join type: %d", (int) jointype);
+		}
+
+		if (!merged)
+			break;
+
+		*matched_parts1 = lappend_int(*matched_parts1, part1);
+		*matched_parts2 = lappend_int(*matched_parts2, part2);
+	}
+
+	pfree(matching1);
+	pfree(matching2);
+
+	if (!merged)
+	{
+		list_free(*matched_parts1);
+		list_free(*matched_parts2);
+		*matched_parts1 = NIL;
+		*matched_parts2 = NIL;
+	}
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * Merge default partitions from both sides, if any, and assign the default
+ * partition for the join result, if necessary.
+ *
+ * If both the relations have default partitions, try mapping those to each
+ * other. If the mapping succeeds corresponding merged partition will act as
+ * the default partition of the join result.
+ *
+ * If inner side of the join has default but not the outer side, rows in it
+ * won't appear in the join result. So don't create a default partition. If
+ * outer side of the join has default but not the inner side, rows in it will
+ * appear in the join result, so create a default merged partition.
+ */
+static bool
+merge_default_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+						 PartitionMap *outer_maps, PartitionMap *inner_maps,
+						 JoinType jointype, int *next_index, int *default_index)
+{
+	int				outer_default = outer_bi->default_index;
+	int				inner_default = inner_bi->default_index;
+	bool			outer_has_default = partition_bound_has_default(outer_bi);
+	bool			inner_has_default = partition_bound_has_default(inner_bi);
+	bool			merged = true;
+	PartitionMap 	*outer_default_map = NULL;
+	PartitionMap 	*inner_default_map = NULL;
+
+	if (outer_has_default)
+		outer_default_map = &outer_maps[outer_default];
+
+	if (inner_has_default)
+		inner_default_map = &inner_maps[inner_default];
+
+	if (!outer_has_default && !inner_has_default)
+		Assert(*default_index < 0);
+	else if (outer_default_map != NULL && inner_default_map == NULL)
+	{
+		if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+			jointype == JOIN_ANTI)
+		{
+			if (outer_default_map->to < 0)
+			{
+				outer_default_map->to = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = outer_default_map->to;
+			}
+			else
+				Assert(*default_index == outer_default_map->to);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else if (outer_default_map == NULL && inner_default_map != NULL)
+	{
+		if (jointype == JOIN_FULL)
+		{
+			if (inner_default_map->to < 0)
+			{
+				inner_default_map->to = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = inner_default_map->to;
+			}
+			else
+				Assert(*default_index == inner_default_map->to);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		*default_index = map_and_merge_partitions(outer_maps, inner_maps,
+												  outer_default, inner_default,
+												  next_index);
+
+		if (*default_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
+
+/*
+ * merge_null_partitions
+ *
+ * Merge NULL partitions, i.e. a partition that can hold NULL values for a list
+ * partitioned table, if any. Find the index of merged partition to which the
+ * NULL values would belong in the join result. If one joining relation has a
+ * NULL partition but not the other, try matching it with the default partition
+ * from the other relation since the default partition may have rows with NULL
+ * partition key. We can eliminate a NULL partition when it appears only on the
+ * inner side of the join and the outer side doesn't have a default partition.
+ *
+ * When the equality operator used for join is strict, two NULL values will not
+ * be considered as equal, and thus a NULL partition can be eliminated for an
+ * inner join. But we don't check the strictness operator here.
+ */
+static bool
+merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_maps, PartitionMap *inner_maps,
+					  JoinType jointype, int *next_index,
+					  int *null_index, int *default_index)
+{
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	int			outer_ni = outer_bi->null_index;
+	int			inner_ni = inner_bi->null_index;
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	bool		merged = true;
+
+	if (!outer_has_null && !inner_has_null)
+		*null_index = -1;
+	else if (outer_has_null && !inner_has_null)
+	{
+		int			merged_index;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, inner relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the inner relation acts as
+		 * INNER relation. For INNER and SEMI join OUTER and INNER
+		 * differentiation is immaterial.
+		 */
+		missing_side_inner = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_outer = (jointype == JOIN_FULL);
+
+		merged = handle_missing_partition(inner_maps,
+										  outer_maps,
+										  inner_default,
+										  outer_ni,
+										  missing_side_outer,
+										  missing_side_inner, next_index,
+										  default_index, &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join to which the outer null partition maps
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = outer_maps[outer_ni].to;
+	}
+	else if (!outer_has_null && inner_has_null)
+	{
+		int			merged_index;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, outer relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the outer relation acts as OUTER
+		 * relation. For INNER and SEMI join OUTER and INNER differentiation is
+		 * immaterial.
+		 */
+		missing_side_outer = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_inner = (jointype == JOIN_FULL);
+		merged = handle_missing_partition(outer_maps,
+										  inner_maps,
+										  outer_default,
+										  inner_ni,
+										  missing_side_outer,
+										  missing_side_inner,
+										  next_index, default_index,
+										  &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join, to which the outer side null partition maps,
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = inner_maps[inner_ni].to;
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		*null_index = map_and_merge_partitions(outer_maps,
+											   inner_maps,
+											   outer_ni,
+											   inner_ni,
+											   next_index);
+		if (*null_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index b1ae39ad63..fb2dc1cbb1 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -16,6 +16,7 @@
 #include "nodes/pg_list.h"
 #include "partitioning/partdefs.h"
 #include "utils/relcache.h"
+struct RelOptInfo;				/* avoid including pathnodes.h here */
 
 
 /*
@@ -107,5 +108,11 @@ extern int partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 							  int nvalues, Datum *values, bool *is_equal);
 extern int partition_hash_bsearch(PartitionBoundInfo boundinfo,
 					   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   struct RelOptInfo *outer_rel,
+					   struct RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
-- 
2.18.0

0003-Tests-for-0-1-1-1-and-1-0-partition-matching-v18.patchapplication/octet-stream; name=0003-Tests-for-0-1-1-1-and-1-0-partition-matching-v18.patchDownload
From 2529b4613ffcf2e7683e3d279f3a363c7d8647ca Mon Sep 17 00:00:00 2001
From: Etsuro Fujita <efujita@postgresql.org>
Date: Thu, 31 Jan 2019 20:31:17 +0900
Subject: [PATCH 3/3] Tests for 0:1, 1:1 and 1:0 partition matching

Rajkumar Raghuvanshi and Ashutosh Bapat.
---
 src/test/regress/expected/partition_join.out | 3955 +++++++++++++++---
 src/test/regress/sql/partition_join.sql      |  427 +-
 2 files changed, 3767 insertions(+), 615 deletions(-)

diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index c55de5d476..f19611c853 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -8,59 +8,86 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-(4 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+(7 rows)
 
 -- left outer join, with whole-row reference; partitionwise join does not apply
 EXPLAIN (COSTS OFF)
@@ -72,35 +99,50 @@ SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER
    ->  Hash Right Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(22 rows)
 
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-      t1      |      t2      
---------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
- (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
- (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
- (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
- (550,0,0550) | 
-(12 rows)
+       t1       |       t2       
+----------------+----------------
+ (-250,0,-0250) | 
+ (-200,0,-0200) | 
+ (-150,0,-0150) | (0,-150,-0150)
+ (-100,0,-0100) | 
+ (-50,0,-0050)  | 
+ (0,0,0000)     | (0,0,0000)
+ (50,0,0050)    | 
+ (100,0,0100)   | 
+ (150,0,0150)   | (0,150,0150)
+ (200,0,0200)   | 
+ (250,0,0250)   | 
+ (300,0,0300)   | (0,300,0300)
+ (350,0,0350)   | 
+ (400,0,0400)   | 
+ (450,0,0450)   | (0,450,0450)
+ (500,0,0500)   | 
+ (550,0,0550)   | 
+ (600,0,0600)   | (0,600,0600)
+ (650,0,0650)   | 
+ (700,0,0700)   | 
+ (750,0,0750)   | (0,750,0750)
+(21 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
@@ -112,35 +154,53 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
    ->  Append
          ->  Hash Right Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Right Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t2_2.b)
-(20 rows)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+      |       | -225 | -0225
+      |       |  -75 | -0075
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(14 rows)
 
 -- full outer join, with placeholder vars
 EXPLAIN (COSTS OFF)
@@ -148,8 +208,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                             QUERY PLAN                            
 ------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               Filter: (((50) = prt1_p0.a) OR ((75) = prt2_p0.b))
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
@@ -174,7 +242,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               Filter: (((50) = prt1_p4.a) OR ((75) = prt2_p4.b))
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
@@ -211,8 +287,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Hash Left Join
+               Hash Cond: (prt1_p0.a = b)
+               ->  Seq Scan on prt1_p0
+                     Filter: ((a < 450) AND (b = 0))
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
          ->  Hash Left Join
                Hash Cond: (prt1_p1.a = b)
                ->  Seq Scan on prt1_p1
@@ -227,29 +310,42 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                ->  Hash
                      ->  Seq Scan on prt1_p2
                            Filter: ((a < 450) AND (b = 0))
-(17 rows)
+(24 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-(9 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+(14 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
                          QUERY PLAN                         
 ------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = b)
+               Filter: ((prt1_p0.b = 0) OR (a = 0))
+               ->  Seq Scan on prt1_p0
+                     Filter: (a < 450)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = b)
                Filter: ((prt1_p1.b = 0) OR (a = 0))
@@ -274,64 +370,153 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                ->  Hash
                      ->  Result
                            One-Time Filter: false
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt2_p4.b = a)
+               Filter: ((b = 0) OR (prt2_p4.a = 0))
+               ->  Seq Scan on prt2_p4
+                     Filter: (b > 250)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
-(12 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+      |       | 375 | 0375
+      |       | 450 | 0450
+      |       | 525 | 0525
+      |       | 600 | 0600
+      |       | 675 | 0675
+      |       | 750 | 0750
+(20 rows)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Semi Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Semi Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                            Filter: (a = 0)
-         ->  Nested Loop Semi Join
-               Join Filter: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
-               ->  Materialize
-                     ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
-(24 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt1_p3 t2_3
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+                           Filter: (b = 0)
+(37 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
 
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
@@ -342,27 +527,82 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
    ->  Append
          ->  Hash Anti Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Hash Anti Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Hash Anti Join
                Hash Cond: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
-(17 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(27 rows)
 
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
-  sum  |         avg          | sum  |         avg         
--------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+  sum  |         avg          | sum  |        avg         
+-------+----------------------+------+--------------------
+ 95550 | 273.0000000000000000 | 2200 | 6.2857142857142857
 (1 row)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Append
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p0 t1
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+               Index Cond: (a = t1.b)
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.b = t2_1.a)
+         ->  Seq Scan on prt2_p1 t1_1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p1 t2_1
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p2 t1_2
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+               Index Cond: (a = t1_2.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p3 t1_3
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+               Index Cond: (a = t1_3.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p4 t1_4
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+               Index Cond: (a = t1_4.b)
+(27 rows)
+
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+  b   |   c   
+------+-------
+ -225 | -0225
+  -75 | -0075
+   75 | 0075
+  225 | 0225
+  375 | 0375
+  525 | 0525
+  675 | 0675
+(7 rows)
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -374,49 +614,74 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
+                     ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
                            Index Cond: (a = t1.a)
-                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t3
                            Index Cond: (b = t2.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_1
+                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
                            Index Cond: (a = t1_1.a)
-                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3_1
                            Index Cond: (b = t2_1.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_2
+                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
                            Index Cond: (a = t1_2.a)
-                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_2
                            Index Cond: (b = t2_2.a)
-(27 rows)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                           Index Cond: (a = t1_3.a)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_3
+                           Index Cond: (b = t2_3.a)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                           Index Cond: (a = t1_4.a)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t3_4
+                           Index Cond: (b = t2_4.a)
+(43 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   | t2a | t3a | least 
------+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
- 100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
- 250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
- 400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
- 550 | 0 | 0550 |     |     |      
-(12 rows)
+  a   | b |   c   | t2a  | t3a | least 
+------+---+-------+------+-----+-------
+ -250 | 0 | -0250 |      |     |      
+ -200 | 0 | -0200 |      |     |      
+ -150 | 0 | -0150 | -150 |   0 |  -150
+ -100 | 0 | -0100 |      |     |      
+  -50 | 0 | -0050 |      |     |      
+    0 | 0 | 0000  |    0 |   0 |     0
+   50 | 0 | 0050  |      |     |      
+  100 | 0 | 0100  |      |     |      
+  150 | 0 | 0150  |  150 |   0 |   150
+  200 | 0 | 0200  |      |     |      
+  250 | 0 | 0250  |      |     |      
+  300 | 0 | 0300  |  300 |   0 |   300
+  350 | 0 | 0350  |      |     |      
+  400 | 0 | 0400  |      |     |      
+  450 | 0 | 0450  |  450 |   0 |   450
+  500 | 0 | 0500  |      |     |      
+  550 | 0 | 0550  |      |     |      
+  600 | 0 | 0600  |  600 |   0 |   600
+  650 | 0 | 0650  |      |     |      
+  700 | 0 | 0700  |      |     |      
+  750 | 0 | 0750  |  750 |   0 |   750
+(21 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
@@ -430,64 +695,95 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
          Hash Cond: ((t1.c)::text = (t2.c)::text)
          Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Seq Scan on prt1_p1 t1
-               ->  Seq Scan on prt1_p2 t1_1
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
          ->  Hash
                ->  Append
                      ->  Hash Join
                            Hash Cond: (t2.a = t3.b)
-                           ->  Seq Scan on prt1_p1 t2
+                           ->  Seq Scan on prt1_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t3
+                                 ->  Seq Scan on prt2_p0 t3
                      ->  Hash Join
                            Hash Cond: (t2_1.a = t3_1.b)
-                           ->  Seq Scan on prt1_p2 t2_1
+                           ->  Seq Scan on prt1_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t3_1
+                                 ->  Seq Scan on prt2_p1 t3_1
                      ->  Hash Join
                            Hash Cond: (t2_2.a = t3_2.b)
-                           ->  Seq Scan on prt1_p3 t2_2
+                           ->  Seq Scan on prt1_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p3 t3_2
-(26 rows)
+                                 ->  Seq Scan on prt2_p2 t3_2
+                     ->  Hash Join
+                           Hash Cond: (t2_3.a = t3_3.b)
+                           ->  Seq Scan on prt1_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t3_3
+                     ->  Hash Join
+                           Hash Cond: (t2_4.a = t3_4.b)
+                           ->  Seq Scan on prt1_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t3_4
+(38 rows)
 
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, t2.b t2b, t2.c t2c, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
-  a  | t2a | t2c  
------+-----+------
-   0 |   0 | 0000
-  50 |     | 
- 100 |     | 
- 150 | 150 | 0150
- 200 |     | 
- 250 |     | 
- 300 | 300 | 0300
- 350 |     | 
- 400 |     | 
- 450 | 450 | 0450
- 500 |     | 
- 550 |     | 
-(12 rows)
+  a   | t2a  |  t2c  
+------+------+-------
+ -250 |      | 
+ -200 |      | 
+ -150 | -150 | -0150
+ -100 |      | 
+  -50 |      | 
+    0 |    0 | 0000
+   50 |      | 
+  100 |      | 
+  150 |  150 | 0150
+  200 |      | 
+  250 |      | 
+  300 |  300 | 0300
+  350 |      | 
+  400 |      | 
+  450 |  450 | 0450
+  500 |      | 
+  550 |      | 
+  600 |  600 | 0600
+  650 |      | 
+  700 |      | 
+  750 |  750 | 0750
+(21 rows)
 
 --
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
@@ -498,32 +794,49 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
    ->  Append
          ->  Hash Join
                Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
-               ->  Seq Scan on prt2_e_p1 t2
+               ->  Seq Scan on prt2_e_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 t1
+                     ->  Seq Scan on prt1_e_p0 t1
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
-               ->  Seq Scan on prt2_e_p2 t2_1
+               Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+               ->  Seq Scan on prt1_e_p1 t1_1
+                     Filter: (c = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 t1_1
-                           Filter: (c = 0)
+                     ->  Seq Scan on prt2_e_p1 t2_1
          ->  Hash Join
                Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
-               ->  Seq Scan on prt2_e_p3 t2_2
+               ->  Seq Scan on prt2_e_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 t1_2
+                     ->  Seq Scan on prt1_e_p2 t1_2
                            Filter: (c = 0)
-(21 rows)
+         ->  Hash Join
+               Hash Cond: (((t2_3.b + t2_3.a) / 2) = ((t1_3.a + t1_3.b) / 2))
+               ->  Seq Scan on prt2_e_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p3 t1_3
+                           Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (((t2_4.b + t2_4.a) / 2) = ((t1_4.a + t1_4.b) / 2))
+               ->  Seq Scan on prt2_e_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4 t1_4
+                           Filter: (c = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
- 150 | 0 | 150 | 0
- 300 | 0 | 300 | 0
- 450 | 0 | 450 | 0
-(4 rows)
+  a   | c |  b   | c 
+------+---+------+---
+ -250 | 0 | -250 | 0
+ -100 | 0 | -100 | 0
+    0 | 0 |    0 | 0
+    0 | 0 |    0 | 0
+  150 | 0 |  150 | 0
+  300 | 0 |  300 | 0
+  450 | 0 |  450 | 0
+  600 | 0 |  600 | 0
+  750 | 0 |  750 | 0
+(9 rows)
 
 --
 -- N-way join
@@ -536,154 +849,232 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = ((t3.a + t3.b) / 2))
+               Join Filter: (t1.a = t2.b)
                ->  Hash Join
-                     Hash Cond: (t2.b = t1.a)
-                     ->  Seq Scan on prt2_p1 t2
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on prt1_e_p0 t3
                      ->  Hash
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                     Index Cond: (((a + b) / 2) = t2.b)
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
+                     Index Cond: (b = ((t3.a + t3.b) / 2))
          ->  Nested Loop
                Join Filter: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_1.b = t1_1.a)
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
+               ->  Index Scan using iprt1_e_p4_ab2 on prt1_e_p1 t3_1
                      Index Cond: (((a + b) / 2) = t2_1.b)
          ->  Nested Loop
                Join Filter: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_2.b = t1_2.a)
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
+               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_2
                      Index Cond: (((a + b) / 2) = t2_2.b)
-(33 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = ((t3_3.a + t3_3.b) / 2))
+               ->  Nested Loop
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t2_3.b)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t2_4.b)
+               ->  Hash Join
+                     Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                     ->  Seq Scan on prt1_e_p4 t3_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_p4 t1_4
+                                 Filter: (b = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = ((t3_4.a + t3_4.b) / 2))
+(52 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
-(4 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(8 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                          QUERY PLAN                          
---------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Hash Right Join
                Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
-               ->  Seq Scan on prt1_e_p1 t3
+               ->  Seq Scan on prt1_e_p0 t3
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2.b = t1.a)
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_1.a)
-               ->  Seq Scan on prt1_e_p2 t3_1
+               ->  Seq Scan on prt1_e_p1 t3_1
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_1.b = t1_1.a)
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_2.a)
-               ->  Seq Scan on prt1_e_p3 t3_2
+               ->  Seq Scan on prt1_e_p2 t3_2
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_2.b = t1_2.a)
-                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                                        Filter: (b = 0)
-(33 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (t1_3.a = b)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (t1_3.a = ((a + b) / 2))
+         ->  Hash Right Join
+               Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+               ->  Seq Scan on prt1_e_p4 t3_4
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_4.b = t1_4.a)
+                           ->  Seq Scan on prt2_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_p4 t1_4
+                                       Filter: (b = 0)
+(51 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                            QUERY PLAN                             
--------------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p1 t3
+                           ->  Seq Scan on prt1_e_p0 t3
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p1_b on prt2_p1 t2
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
                      Index Cond: (t1.a = b)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p2 t3_1
+                           ->  Seq Scan on prt1_e_p1 t3_1
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
+               ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
                      Index Cond: (t1_1.a = b)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p3 t3_2
+                           ->  Seq Scan on prt1_e_p2 t3_2
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
                      Index Cond: (t1_2.a = b)
-(30 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_e_p3 t3_3
+                           Filter: (c = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                           Index Cond: (a = ((t3_3.a + t3_3.b) / 2))
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (t1_3.a = b)
+         ->  Nested Loop Left Join
+               ->  Hash Right Join
+                     Hash Cond: (t1_4.a = ((t3_4.a + t3_4.b) / 2))
+                     ->  Seq Scan on prt1_p4 t1_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p4 t3_4
+                                 Filter: (c = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (t1_4.a = b)
+(47 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -692,21 +1083,34 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                                                    QUERY PLAN                                                   
 ----------------------------------------------------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   Sort Key: prt1_p0.a, prt2_p0.b, ((prt1_e_p0.a + prt1_e_p0.b))
    ->  Append
          ->  Hash Full Join
-               Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
-               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               Hash Cond: (prt1_p0.a = ((prt1_e_p0.a + prt1_e_p0.b) / 2))
+               Filter: ((prt1_p0.a = (50)) OR (prt2_p0.b = (75)) OR (((prt1_e_p0.a + prt1_e_p0.b) / 2) = (50)))
                ->  Hash Full Join
-                     Hash Cond: (prt1_p1.a = prt2_p1.b)
-                     ->  Seq Scan on prt1_p1
+                     Hash Cond: (prt1_p0.a = prt2_p0.b)
+                     ->  Seq Scan on prt1_p0
                            Filter: (b = 0)
                      ->  Hash
-                           ->  Seq Scan on prt2_p1
+                           ->  Seq Scan on prt2_p0
                                  Filter: (a = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1
+                     ->  Seq Scan on prt1_e_p0
                            Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = prt1_p1.a)
+               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               ->  Seq Scan on prt1_e_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on prt1_p1
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1
+                                       Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
                Filter: ((prt1_p2.a = (50)) OR (prt2_p2.b = (75)) OR (((prt1_e_p2.a + prt1_e_p2.b) / 2) = (50)))
@@ -733,7 +1137,20 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                ->  Hash
                      ->  Seq Scan on prt1_e_p3
                            Filter: (c = 0)
-(42 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = ((prt1_e_p4.a + prt1_e_p4.b) / 2))
+               Filter: ((prt1_p4.a = (50)) OR (prt2_p4.b = (75)) OR (((prt1_e_p4.a + prt1_e_p4.b) / 2) = (50)))
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = prt2_p4.b)
+                     ->  Seq Scan on prt1_p4
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4
+                                 Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4
+                           Filter: (c = 0)
+(68 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
@@ -751,172 +1168,260 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = t1_3.b)
+               Join Filter: (t1.a = t1_5.b)
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Join
-                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
-                           ->  Seq Scan on prt1_e_p1 t2
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t1_3
+                                 ->  Seq Scan on prt2_p0 t1_5
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
                      Index Cond: (a = ((t2.a + t2.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_1.a = t1_4.b)
+               Join Filter: (t1_1.a = t1_6.b)
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Join
-                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
-                           ->  Seq Scan on prt1_e_p2 t2_1
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_6.b)
+                           ->  Seq Scan on prt1_e_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t1_4
+                                 ->  Seq Scan on prt2_p1 t1_6
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
                      Index Cond: (a = ((t2_1.a + t2_1.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_2.a = t1_5.b)
+               Join Filter: (t1_2.a = t1_7.b)
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
+                           ->  Seq Scan on prt2_p2 t1_7
                                  Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
+                           ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t2_2
+                                 Index Cond: (((a + b) / 2) = t1_7.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t1_8.b)
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Nested Loop
+                           ->  Seq Scan on prt2_p3 t1_8
+                                 Filter: (a = 0)
+                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_3
+                                 Index Cond: (((a + b) / 2) = t1_8.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = ((t2_3.a + t2_3.b) / 2))
+                     Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t1_9.b)
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Join
+                           Hash Cond: (((t2_4.a + t2_4.b) / 2) = t1_9.b)
+                           ->  Seq Scan on prt1_e_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t1_9
+                                       Filter: (a = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = ((t2_4.a + t2_4.b) / 2))
+                     Filter: (b = 0)
+(66 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                               QUERY PLAN                                
--------------------------------------------------------------------------
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
-                           ->  Seq Scan on prt2_p1 t1_3
+                           Hash Cond: (t1_5.b = ((t1_10.a + t1_10.b) / 2))
+                           ->  Seq Scan on prt2_p0 t1_5
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t1_6
+                                 ->  Seq Scan on prt1_e_p0 t1_10
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
-                     Index Cond: (a = t1_3.b)
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t1_5.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
-                           ->  Seq Scan on prt2_p2 t1_4
+                           Hash Cond: (t1_6.b = ((t1_11.a + t1_11.b) / 2))
+                           ->  Seq Scan on prt2_p1 t1_6
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t1_7
+                                 ->  Seq Scan on prt1_e_p1 t1_11
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
-                     Index Cond: (a = t1_4.b)
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
+                     Index Cond: (a = t1_6.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
-                           ->  Seq Scan on prt2_p3 t1_5
+                           Hash Cond: (t1_7.b = ((t1_12.a + t1_12.b) / 2))
+                           ->  Seq Scan on prt2_p2 t1_7
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p3 t1_8
+                                 ->  Seq Scan on prt1_e_p2 t1_12
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t1_5.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t1_7.b)
                      Filter: (b = 0)
-(39 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_8.b = ((t1_13.a + t1_13.b) / 2))
+                           ->  Seq Scan on prt2_p3 t1_8
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p3 t1_13
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t1_8.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_9.b = ((t1_14.a + t1_14.b) / 2))
+                           ->  Seq Scan on prt2_p4 t1_9
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t1_14
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = t1_9.b)
+                     Filter: (b = 0)
+(63 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 -- test merge joins
 SET enable_hashjoin TO off;
 SET enable_nestloop TO off;
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                           QUERY PLAN                           
-----------------------------------------------------------------
+                            QUERY PLAN                            
+------------------------------------------------------------------
  Merge Append
    Sort Key: t1.a
    ->  Merge Semi Join
-         Merge Cond: (t1.a = t1_3.b)
+         Merge Cond: (t1.a = t1_5.b)
          ->  Sort
                Sort Key: t1.a
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+               Merge Cond: (t1_5.b = (((t1_10.a + t1_10.b) / 2)))
                ->  Sort
-                     Sort Key: t1_3.b
-                     ->  Seq Scan on prt2_p1 t1_3
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on prt2_p0 t1_5
                ->  Sort
-                     Sort Key: (((t1_6.a + t1_6.b) / 2))
-                     ->  Seq Scan on prt1_e_p1 t1_6
+                     Sort Key: (((t1_10.a + t1_10.b) / 2))
+                     ->  Seq Scan on prt1_e_p0 t1_10
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_1.a = t1_4.b)
+         Merge Cond: (t1_1.a = t1_6.b)
          ->  Sort
                Sort Key: t1_1.a
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+               Merge Cond: (t1_6.b = (((t1_11.a + t1_11.b) / 2)))
                ->  Sort
-                     Sort Key: t1_4.b
-                     ->  Seq Scan on prt2_p2 t1_4
+                     Sort Key: t1_6.b
+                     ->  Seq Scan on prt2_p1 t1_6
                ->  Sort
-                     Sort Key: (((t1_7.a + t1_7.b) / 2))
-                     ->  Seq Scan on prt1_e_p2 t1_7
+                     Sort Key: (((t1_11.a + t1_11.b) / 2))
+                     ->  Seq Scan on prt1_e_p1 t1_11
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_2.a = t1_5.b)
+         Merge Cond: (t1_2.a = t1_7.b)
          ->  Sort
                Sort Key: t1_2.a
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+               Merge Cond: (t1_7.b = (((t1_12.a + t1_12.b) / 2)))
                ->  Sort
-                     Sort Key: t1_5.b
-                     ->  Seq Scan on prt2_p3 t1_5
+                     Sort Key: t1_7.b
+                     ->  Seq Scan on prt2_p2 t1_7
                ->  Sort
-                     Sort Key: (((t1_8.a + t1_8.b) / 2))
-                     ->  Seq Scan on prt1_e_p3 t1_8
+                     Sort Key: (((t1_12.a + t1_12.b) / 2))
+                     ->  Seq Scan on prt1_e_p2 t1_12
                            Filter: (c = 0)
-(47 rows)
+   ->  Merge Semi Join
+         Merge Cond: (t1_3.a = t1_8.b)
+         ->  Sort
+               Sort Key: t1_3.a
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_8.b = (((t1_13.a + t1_13.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_8.b
+                     ->  Seq Scan on prt2_p3 t1_8
+               ->  Sort
+                     Sort Key: (((t1_13.a + t1_13.b) / 2))
+                     ->  Seq Scan on prt1_e_p3 t1_13
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_4.a = t1_9.b)
+         ->  Sort
+               Sort Key: t1_4.a
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_9.b = (((t1_14.a + t1_14.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_9.b
+                     ->  Seq Scan on prt2_p4 t1_9
+               ->  Sort
+                     Sort Key: (((t1_14.a + t1_14.b) / 2))
+                     ->  Seq Scan on prt1_e_p4 t1_14
+                           Filter: (c = 0)
+(77 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
@@ -933,14 +1438,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
                            ->  Sort
                                  Sort Key: (((t3.a + t3.b) / 2))
-                                 ->  Seq Scan on prt1_e_p1 t3
+                                 ->  Seq Scan on prt1_e_p0 t3
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1.a
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                ->  Sort
                      Sort Key: t2.b
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Merge Left Join
                Merge Cond: (t1_1.a = t2_1.b)
                ->  Sort
@@ -949,14 +1454,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
                            ->  Sort
                                  Sort Key: (((t3_1.a + t3_1.b) / 2))
-                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                 ->  Seq Scan on prt1_e_p1 t3_1
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_1.a
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                ->  Sort
                      Sort Key: t2_1.b
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Merge Left Join
                Merge Cond: (t1_2.a = t2_2.b)
                ->  Sort
@@ -965,32 +1470,74 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
                            ->  Sort
                                  Sort Key: (((t3_2.a + t3_2.b) / 2))
-                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                 ->  Seq Scan on prt1_e_p2 t3_2
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_2.a
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                ->  Sort
                      Sort Key: t2_2.b
-                     ->  Seq Scan on prt2_p3 t2_2
-(51 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Merge Left Join
+               Merge Cond: (t1_3.a = t2_3.b)
+               ->  Sort
+                     Sort Key: t1_3.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_3.a + t3_3.b) / 2)) = t1_3.a)
+                           ->  Sort
+                                 Sort Key: (((t3_3.a + t3_3.b) / 2))
+                                 ->  Seq Scan on prt1_e_p3 t3_3
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_3.a
+                                 ->  Seq Scan on prt1_p3 t1_3
+               ->  Sort
+                     Sort Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Merge Left Join
+               Merge Cond: (t1_4.a = t2_4.b)
+               ->  Sort
+                     Sort Key: t1_4.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_4.a + t3_4.b) / 2)) = t1_4.a)
+                           ->  Sort
+                                 Sort Key: (((t3_4.a + t3_4.b) / 2))
+                                 ->  Seq Scan on prt1_e_p4 t3_4
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_4.a
+                                 ->  Seq Scan on prt1_p4 t1_4
+               ->  Sort
+                     Sort Key: t2_4.b
+                     ->  Seq Scan on prt2_p4 t2_4
+(83 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- MergeAppend on nullable column
 EXPLAIN (COSTS OFF)
@@ -998,8 +1545,18 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Merge Left Join
+               Merge Cond: (prt1_p0.a = b)
+               ->  Sort
+                     Sort Key: prt1_p0.a
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
+               ->  Sort
+                     Sort Key: b
+                     ->  Result
+                           One-Time Filter: false
          ->  Merge Left Join
                Merge Cond: (prt1_p1.a = b)
                ->  Sort
@@ -1020,21 +1577,26 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                      Sort Key: prt2_p2.b
                      ->  Seq Scan on prt2_p2
                            Filter: (b > 250)
-(23 rows)
+(33 rows)
 
 SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  b  
------+-----
-   0 |    
-  50 |    
- 100 |    
- 150 |    
- 200 |    
- 250 |    
- 300 | 300
- 350 |    
- 400 |    
-(9 rows)
+  a   |  b  
+------+-----
+ -250 |    
+ -200 |    
+ -150 |    
+ -100 |    
+  -50 |    
+    0 |    
+   50 |    
+  100 |    
+  150 |    
+  200 |    
+  250 |    
+  300 | 300
+  350 |    
+  400 |    
+(14 rows)
 
 -- merge join when expression with whole-row reference needs to be sorted;
 -- partitionwise join does not apply
@@ -1048,30 +1610,797 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
          Sort Key: t1.a, ((((t1.*)::prt1))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
    ->  Sort
          Sort Key: t2.b, ((((t2.*)::prt2))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt2_p1 t2
-                     ->  Seq Scan on prt2_p2 t2_1
-                     ->  Seq Scan on prt2_p3 t2_2
-(16 rows)
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+(20 rows)
 
 SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.b ORDER BY t1.a;
- a  | b  
-----+----
-  0 |  0
-  6 |  6
- 12 | 12
- 18 | 18
- 24 | 24
-(5 rows)
+  a  |  b  
+-----+-----
+ -24 | -24
+ -18 | -18
+ -12 | -12
+  -6 |  -6
+   0 |   0
+   6 |   6
+  12 |  12
+  18 |  18
+  24 |  24
+(9 rows)
 
 RESET enable_hashjoin;
 RESET enable_nestloop;
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p4 t2_3
+               ->  Seq Scan on prt2_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_4
+                           Filter: (b = 0)
+(22 rows)
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p3 t1_2
+                           Filter: (b = 0)
+(21 rows)
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -150 | -0150 | 0 | -0150
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+(7 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (t1_3.a = b)
+         ->  Hash Right Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 |   | 
+ -200 | -0200 |   | 
+ -150 | -0150 | 0 | -0150
+ -100 | -0100 |   | 
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+(21 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -200 | 0 | -0200
+ -100 | 0 | -0100
+  -50 | 0 | -0050
+   50 | 0 | 0050
+  100 | 0 | 0100
+  200 | 0 | 0200
+  250 | 0 | 0250
+  350 | 0 | 0350
+  400 | 0 | 0400
+  500 | 0 | 0500
+  550 | 0 | 0550
+  650 | 0 | 0650
+  700 | 0 | 0700
+(14 rows)
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+                             QUERY PLAN                              
+---------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t3.c
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p0_a on prt1_p0 t3
+                           Index Cond: (a = t2.b)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t2.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_1.a = t2_1.b)
+                           ->  Seq Scan on prt1_p1 t3_1
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1 t2_1
+                                       Filter: (a = 0)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p2_a on prt1_p2 t3_2
+                           Index Cond: (a = t2_2.b)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t3_3
+                           Index Cond: (a = t2_3.b)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_4.a = t2_4.b)
+                           ->  Seq Scan on prt1_p4 t3_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t2_4
+                                       Filter: (a = 0)
+(47 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+  a   | a |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.b
+   ->  Hash Right Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p5 t2_5
+                           Filter: (a = 0)
+(24 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: (t1.a = t2.b)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.a, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
+                     Filter: (a = 0)
+(22 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
 --
 -- partitioned by multiple columns
 --
@@ -1141,82 +2470,1552 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 -- test partition matching with N-way join
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                   QUERY PLAN                                   
---------------------------------------------------------------------------------
+                                           QUERY PLAN                                           
+------------------------------------------------------------------------------------------------
  GroupAggregate
    Group Key: t1.c, t2.c, t3.c
    ->  Sort
          Sort Key: t1.c, t3.c
          ->  Append
                ->  Hash Join
-                     Hash Cond: (t1.c = ltrim(t3.c, 'A'::text))
+                     Hash Cond: ((t1.c)::text = ltrim(t3.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1.b = t2.b) AND (t1.c = t2.c))
-                           ->  Seq Scan on plt1_p1 t1
+                           Hash Cond: ((t2.b = t1.b) AND ((t2.c)::text = (t1.c)::text))
+                           ->  Seq Scan on plt2_p4 t2
                            ->  Hash
-                                 ->  Seq Scan on plt2_p1 t2
+                                 ->  Seq Scan on plt1_p4 t1
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p1 t3
+                           ->  Seq Scan on plt1_e_p4 t3
                ->  Hash Join
-                     Hash Cond: (t1_1.c = ltrim(t3_1.c, 'A'::text))
+                     Hash Cond: ((t1_1.c)::text = ltrim(t3_1.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
-                           ->  Seq Scan on plt1_p2 t1_1
+                           Hash Cond: ((t1_1.b = t2_1.b) AND ((t1_1.c)::text = (t2_1.c)::text))
+                           ->  Seq Scan on plt1_p1 t1_1
                            ->  Hash
-                                 ->  Seq Scan on plt2_p2 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p2 t3_1
+                           ->  Seq Scan on plt1_e_p1 t3_1
                ->  Hash Join
-                     Hash Cond: (t1_2.c = ltrim(t3_2.c, 'A'::text))
+                     Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
-                           ->  Seq Scan on plt1_p3 t1_2
+                           Hash Cond: ((t1_2.b = t2_2.b) AND ((t1_2.c)::text = (t2_2.c)::text))
+                           ->  Seq Scan on plt1_p2 t1_2
                            ->  Hash
-                                 ->  Seq Scan on plt2_p3 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p3 t3_2
-(32 rows)
+                           ->  Seq Scan on plt1_e_p2 t3_2
+               ->  Hash Join
+                     Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2_3.b = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text))
+                           ->  Seq Scan on plt2_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p3 t3_3
+(41 rows)
 
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  75.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 123.0000000000000000 | 123.0000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 225.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 273.0000000000000000 | 273.0000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 375.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 423.0000000000000000 | 423.0000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 525.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 573.0000000000000000 | 573.0000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p5 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_3
+                                 ->  Seq Scan on plt2_p3 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Seq Scan on plt2_p2 t2_2
+                                 ->  Seq Scan on plt2_p3 t2_3
+(22 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1241,22 +4040,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
 --------------------------------------------------
  Hash Left Join
    Hash Cond: (t2.b = a)
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t3.a = t2.b)
-               ->  Seq Scan on prt1_p1 t3
-               ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
-         ->  Hash Join
-               Hash Cond: (t3_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t3_1
-               ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
-         ->  Hash Join
-               Hash Cond: (t3_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t3_2
-               ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
+   ->  Hash Join
+         Hash Cond: (t3.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t3
+               ->  Seq Scan on prt1_p1 t3_1
+               ->  Seq Scan on prt1_p2 t3_2
+               ->  Seq Scan on prt1_p3 t3_3
+               ->  Seq Scan on prt1_p4 t3_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
    ->  Hash
          ->  Result
                One-Time Filter: false
@@ -1271,16 +4070,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 --
 -- tests for hash partitioned tables.
@@ -1356,41 +4161,9 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  273.0000000000000000 | 273.0000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
 (6 rows)
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
- Sort
-   Sort Key: t1.a
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
-               ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
-               ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
-                           Filter: (b = 0)
-(21 rows)
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
@@ -1405,26 +4178,24 @@ SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c
    Sort Key: t1.c
    ->  HashAggregate
          Group Key: t1.c, t2.c
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t2.c = t1.c)
-                     ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p1 t1
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on plt1_p4 t1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_1.c = t1_1.c)
-                     ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_p2 t1_1
+                           ->  Seq Scan on plt1_p1 t1_1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_2.c = t1_2.c)
-                     ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Seq Scan on plt1_p2 t1_2
                                  Filter: ((a % 25) = 0)
-(23 rows)
+                           ->  Seq Scan on plt1_p3 t1_3
+                                 Filter: ((a % 25) = 0)
+(21 rows)
 
 --
 -- multiple levels of partitioning
@@ -1826,64 +4597,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
  Hash Join
    Hash Cond: (t1.a = t2.a)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt4_n_p1 t2
                ->  Seq Scan on prt4_n_p2 t2_1
                ->  Seq Scan on prt4_n_p3 t2_2
-(11 rows)
+(13 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a and t1.a = t3.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                        QUERY PLAN                        
+----------------------------------------------------------
  Hash Join
-   Hash Cond: (t2.a = t1.a)
+   Hash Cond: (t3.b = t1.a)
    ->  Append
-         ->  Seq Scan on prt4_n_p1 t2
-         ->  Seq Scan on prt4_n_p2 t2_1
-         ->  Seq Scan on prt4_n_p3 t2_2
+         ->  Seq Scan on prt2_p0 t3
+         ->  Seq Scan on prt2_p1 t3_1
+         ->  Seq Scan on prt2_p2 t3_2
+         ->  Seq Scan on prt2_p3 t3_3
+         ->  Seq Scan on prt2_p4 t3_4
+         ->  Seq Scan on prt2_p5 t3_5
    ->  Hash
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.a = t3_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_2.b)
-                     ->  Seq Scan on prt1_p3 t1_2
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3 t3_2
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt4_n_p1 t2
+                           ->  Seq Scan on prt4_n_p2 t2_1
+                           ->  Seq Scan on prt4_n_p3 t2_2
 (23 rows)
 
 -- partitionwise join can not be applied if there are no equi-join conditions
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (t1.a < b)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (t1.a < b)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (t1.a < b)
-(12 rows)
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+(16 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partitionwise join
@@ -1969,16 +4746,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
-                     ->  Seq Scan on plt1_p1 t3
-                     ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3
+                     ->  Seq Scan on plt1_p1 t3_1
+                     ->  Seq Scan on plt1_p2 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partitionwise join can not be applied for a join between list and range
 -- partitioned table
@@ -1989,14 +4767,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
  Hash Full Join
    Hash Cond: ((t2.c)::text = (t1.c)::text)
    ->  Append
-         ->  Seq Scan on prt1_p1 t2
-         ->  Seq Scan on prt1_p2 t2_1
-         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p0 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p2 t2_2
+         ->  Seq Scan on prt1_p3 t2_3
+         ->  Seq Scan on prt1_p4 t2_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt1_n_p1 t1
                ->  Seq Scan on prt1_n_p2 t1_1
-(10 rows)
+(12 rows)
 
 -- partitionwise join can not be applied if only one of joining table has
 -- default partition
@@ -2012,16 +4792,23 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
    ->  Hash Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index c1c9859651..0e884835fb 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -10,25 +10,39 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -67,11 +81,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -93,20 +115,30 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 
 EXPLAIN (COSTS OFF)
@@ -169,6 +201,128 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -193,28 +347,79 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
 -- test partition matching with N-way join
@@ -222,6 +427,175 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
@@ -267,27 +641,18 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
 ALTER TABLE plt2 DETACH PARTITION plt2_p3;
 ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
 ANALYZE plt2;
-
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.a % 25 = 0 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
 --
 -- multiple levels of partitioning
 --
-- 
2.18.0

#63amul sul
sulamul@gmail.com
In reply to: amul sul (#62)
3 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Attached is the rebased atop of the latest master head(35bc0ec7c8).

Regards,
Amul Sul

On Mon, Feb 4, 2019 at 11:05 AM amul sul <sulamul@gmail.com> wrote:

Show quoted text

There are few whitespaces in 0002 patch that I have fixed in the attached
version.
Rest of the patches are untouched.

Ill continue my review and testing.

Regards,
Amul

On Thu, Jan 31, 2019 at 5:26 PM Etsuro Fujita <fujita.etsuro@lab.ntt.co.jp>
wrote:

(2019/01/22 21:38), Etsuro Fujita wrote:

Will continue to review.

I rebased the patch set against the latest HEAD. Attached is a new
version. I'll move this to the next CF, and continue to review it.

Best regards,
Etsuro Fujita

Attachments:

0001-Hash-partition-bound-equality-refactoring-v19.patchapplication/x-patch; name=0001-Hash-partition-bound-equality-refactoring-v19.patchDownload
From b65bc94d63f453ac0e2e757bcd28661cabc6c549 Mon Sep 17 00:00:00 2001
From: Etsuro Fujita <efujita@postgresql.org>
Date: Thu, 31 Jan 2019 19:18:18 +0900
Subject: [PATCH 1/3] Hash partition bound equality refactoring.

Separate the code to check whether two given hash bounds are equal into a
separate function, so that it can be called from multiple places. Right now
it's only caller is partition_bounds_equal() but later we will use it for
merging partition bounds.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/partitioning/partbounds.c | 95 +++++++++++++++++----------
 1 file changed, 61 insertions(+), 34 deletions(-)

diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index e71eb3793b..637e05e5ca 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -106,6 +106,9 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 						 Expr **keyCol,
 						 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static bool partition_hbounds_equal(PartitionBoundInfo b1,
+									PartitionBoundInfo b2);
+
 
 /*
  * get_qual_from_partbound
@@ -654,6 +657,63 @@ create_range_bounds(PartitionBoundSpec **boundspecs, int nparts,
 	return boundinfo;
 }
 
+/*
+ * Are two hash partition bound collections logically equal?
+ *
+ * Hash partition bounds store modulus and remainder in datums array which are
+ * always integers irrespective of the number of partition keys and their data
+ * types. Hence we can compare the hash bound collection without any partition
+ * key specific information. Separating this logic in a function which does not
+ * require partition key specific information allows it be called from places
+ * where the partition key specific information is not completely available.
+ */
+static bool
+partition_hbounds_equal(PartitionBoundInfo b1, PartitionBoundInfo b2)
+{
+	int			greatest_modulus = get_hash_partition_greatest_modulus(b1);
+	int			i;
+
+	Assert(b1->strategy == PARTITION_STRATEGY_HASH &&
+		   b2->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * If two hash partitioned tables have different greatest moduli,
+	 * their partition schemes don't match.  For hash partitioned table,
+	 * the greatest modulus is given by the last datum and number of
+	 * partitions is given by ndatums.
+	 */
+	if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
+		return false;
+
+	/*
+	 * We arrange the partitions in the ascending order of their modulus and
+	 * remainders.  Also every modulus is factor of next larger modulus.
+	 * Therefore we can safely store index of a given partition in indexes
+	 * array at remainder of that partition.  Also entries at (remainder + N *
+	 * modulus) positions in indexes array are all same for (modulus,
+	 * remainder) specification for any partition.  Thus datums array from both
+	 * the given bounds are same, if and only if their indexes array will be
+	 * same.  So, it suffices to compare indexes array.
+	 */
+	for (i = 0; i < greatest_modulus; i++)
+		if (b1->indexes[i] != b2->indexes[i])
+			return false;
+
+#ifdef USE_ASSERT_CHECKING
+
+	/*
+	 * Nonetheless make sure that the bounds are indeed same when the indexes
+	 * match.  Hash partition bound stores modulus and remainder at
+	 * b1->datums[i][0] and b1->datums[i][1] position respectively.
+	 */
+	for (i = 0; i < b1->ndatums; i++)
+		Assert((b1->datums[i][0] == b2->datums[i][0] &&
+				b1->datums[i][1] == b2->datums[i][1]));
+#endif
+
+	return true;
+}
+
 /*
  * Are two partition bound collections logically equal?
  *
@@ -682,41 +742,8 @@ partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
 
 	if (b1->strategy == PARTITION_STRATEGY_HASH)
 	{
-		int			greatest_modulus = get_hash_partition_greatest_modulus(b1);
-
-		/*
-		 * If two hash partitioned tables have different greatest moduli,
-		 * their partition schemes don't match.
-		 */
-		if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
+		if (!partition_hbounds_equal(b1, b2))
 			return false;
-
-		/*
-		 * We arrange the partitions in the ascending order of their moduli
-		 * and remainders.  Also every modulus is factor of next larger
-		 * modulus.  Therefore we can safely store index of a given partition
-		 * in indexes array at remainder of that partition.  Also entries at
-		 * (remainder + N * modulus) positions in indexes array are all same
-		 * for (modulus, remainder) specification for any partition.  Thus
-		 * datums array from both the given bounds are same, if and only if
-		 * their indexes array will be same.  So, it suffices to compare
-		 * indexes array.
-		 */
-		for (i = 0; i < greatest_modulus; i++)
-			if (b1->indexes[i] != b2->indexes[i])
-				return false;
-
-#ifdef USE_ASSERT_CHECKING
-
-		/*
-		 * Nonetheless make sure that the bounds are indeed same when the
-		 * indexes match.  Hash partition bound stores modulus and remainder
-		 * at b1->datums[i][0] and b1->datums[i][1] position respectively.
-		 */
-		for (i = 0; i < b1->ndatums; i++)
-			Assert((b1->datums[i][0] == b2->datums[i][0] &&
-					b1->datums[i][1] == b2->datums[i][1]));
-#endif
 	}
 	else
 	{
-- 
2.18.0

0002-Partition-wise-join-for-1-1-1-0-0-1-partition-matchi-v19.patchapplication/x-patch; name=0002-Partition-wise-join-for-1-1-1-0-0-1-partition-matchi-v19.patchDownload
From a6fc8476fac4f062e296ad6260207fa144a3a12d Mon Sep 17 00:00:00 2001
From: Amul Sul <amul.sul@enterprisedb.com>
Date: Mon, 4 Mar 2019 23:49:47 -0500
Subject: [PATCH 2/3] Partition-wise join for 1:1, 1:0, 0:1 partition matching.

Earlier version of partition-wise join implementation allowed
partition-wise join between two relations with exactly same partition
bounds. This commit allows partition-wise join to be applied under
following conditions

1. the partition bounds of joining relations are such that rows from
given partition on one side join can join with rows from maximum one
partition on the other side i.e. bounds of a given partition on one
side match/overlap with those of maximum one partition on the other
side. If the mapping happens to be m:n where m > 1 or n > 1, we have
to gang multiple partition relations together into a single relation.
This means that we have to add simple relations during join
processing, something which is not supported right now.  ALso, in such
a case, different pairs of joining relations can produce different
partition bounds for the same join relation, which again is not
supported right now.

2. For every partition on outer side that can contribute to the result
of an OUTER join, there exists at least one (taken along with item 1,
it means exactly one)  matching partition on the inner side. To
support partition-wise join when the inner matching partition doesn't
exist, we have to add a dummy base relation corresponding to the
non-existent inner partition. We don't have support to add base
relations during join processing.

3. For hash partitioned tables however, we support partition-wise join
only when the bounds exactly match. For hash partitioning it's unusual
to have missing partitions and hence generic partition matching is not
required.

With advanced partition matching, we won't be able to apply
partition-wise join when there are missing matching partitions. So for
a given N-way join, some sub-joins may not be partitioned, while the
overall N-way join is partitioned. We can not try partition-wise join
when one or both of the joining relations are not partitioned in one
of the pairs of joining relation. Skip partition-wise join for that
pair.

An example is A IJ B LJ C where B has an extra partition compared to A
and C. Join B LJ C requires dummy relation to join the extra partition
from B to a missing partition from C, and hence can not use
partition-wise join right now and hence BC is not partitioned.  The
extra partition in B gets eliminated in A IJ B and thus it can use
partition-wise join and is partitioned. Hence (AB)C can use
partition-wise join but not A(BC).

Not every pair of joining relation (including the one presented to
build_joinrel_partition_info()) for the same joinrel can use
partition-wise join or has both the relations partitioned. Hence we
calculate the partition bounds for the join relation in
try_partition_wise_join() instead of doing it here.

The partition bounds produced for the first pair that can use
partition-wise are saved in the RelOptInfo of the joinrel. Partition
bounds produced for subsequent pairs should match that produced by the
first pair.

Support for default partition in list partitioned tables
========================================================

When a list value is present in one of the joining relations and not
the other, and the other relation has default partition, match (join)
the partition containing that list value with the default partition.
If there are multiple matches, we can not proceed with the
partition-wise join similar to the case of matching non-default
partition. If the default partition happens to be on the outer side of
the join, the resulting join partition acts as a default partition as
it will contain all the values from the default partition. If
the partition containing the list value happens to be on the outer
side of the join, the resulting join partition is associated with the
list value, since no other partition key value from the default
partition makes it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join.  Such a
default partition may get joined to a partition from the inner side,
if inner side has a list value that is not present in the outer side.
But if such a join partner does not exist, we won't be able to use
partition-wise join since there is no (dummy) relation on the inner
side to join to.

Support for matching default partition in range partitioned tables
==================================================================

When a range is present in one of the joining relations and not the
other, and the other relation has default partition, match (join) the
partition corresponding range with the default partition. If two
ranges overlap but their ranges don't match exactly, the
non-overlapping portion from one range may join the previous
(non-overlapping portion near the lower bound, if exists), next
(non-overlapping portion near the upper bound, if exists) ranges from
the other relation or default partition from the other relation. This
causes multiple partitions on the other side to be joined with a
single partition on the first side; a case we don't support. Any range
from one side which doesn't overlap with any range on the other side,
may find its join partner in the default partition and the
corresponding range partition will need to be joined with the default
partition.  If the default partition happens to be on the outer side
of the join, the resulting join partition acts as a default partition
as it will contain all the values from the default partition. If the
non-partition corresponding to the range happens to be on the outer
side of the join, the resulting join partition is associated with that
range, since partition key values from the default partition outside
that range won't make it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join.  Such a
default partition may get joined to a non-default partition from the
inner side, if inner side has a range missing in the outer side.  But
if such a join partner does not exist, we won't be able to use
partition-wise join since there is no (dummy) relation on the inner
side to join to.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/optimizer/path/joinrels.c |   95 +-
 src/backend/optimizer/util/relnode.c  |   33 +-
 src/backend/partitioning/partbounds.c | 1677 ++++++++++++++++++++++++-
 src/include/partitioning/partbounds.h |    7 +
 4 files changed, 1766 insertions(+), 46 deletions(-)

diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index dfbbfdac6d..ca4d261bc1 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1320,25 +1320,31 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	PartitionScheme part_scheme;
+	PartitionBoundInfo join_boundinfo;
+	List	   *parts1;
+	List	   *parts2;
+	ListCell   *lc1;
+	ListCell   *lc2;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1351,32 +1357,63 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	 */
 	Assert(joinrel->part_scheme == rel1->part_scheme &&
 		   joinrel->part_scheme == rel2->part_scheme);
+	part_scheme = joinrel->part_scheme;
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * Get the list of matching partitions to be joined along with the
+	 * partition bounds of the join relation. Because of the restrictions
+	 * imposed by partition matching algorithm, not every pair of joining
+	 * relations for this join will be able to use partition-wise join. But all
+	 * those pairs which can use partition-wise join will produce the same
+	 * partition bounds for the join relation.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	join_boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											part_scheme->partsupfunc,
+											part_scheme->partcollation,
+											rel1, rel2,
+											parent_sjinfo->jointype,
+											&parts1, &parts2);
+
+	if (join_boundinfo == NULL)
+		return;
 
-	nparts = joinrel->nparts;
+	if (joinrel->boundinfo == NULL)
+	{
+		Assert(joinrel->nparts == 0 && joinrel->part_rels == NULL);
+		joinrel->boundinfo = join_boundinfo;
+		joinrel->nparts = list_length(parts1);
+		Assert(joinrel->nparts == list_length(parts2));
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) *
+									joinrel->nparts);
+	}
+	else
+	{
+		Assert(partition_bounds_equal(part_scheme->partnatts,
+									  part_scheme->parttyplen,
+									  part_scheme->parttypbyval,
+									  join_boundinfo, joinrel->boundinfo));
+		/*
+		 * Every pair of joining relations should result in the same number
+		 * of child-joins.
+		 */
+		Assert(joinrel->nparts == list_length(parts1));
+		Assert(joinrel->nparts == list_length(parts2));
+		Assert(joinrel->part_rels);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	cnt_parts = 0;
+	forboth(lc1, parts1, lc2, parts2)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
+		int			part1 = lfirst_int(lc1);
+		int			part2 = lfirst_int(lc2);
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1384,6 +1421,10 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		Assert(part1 >= 0 && part2 >= 0);
+		child_rel1 = rel1->part_rels[part1];
+		child_rel2 = rel2->part_rels[part2];
+
 		/*
 		 * If a child table has consider_partitionwise_join=false, it means
 		 * that it's a dummy relation for which we skipped setting up tlist
@@ -1438,12 +1479,24 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 			joinrel->part_rels[cnt_parts] = child_joinrel;
 		}
 
+		/*
+		 * For every pair of joining relations, the set of matching partitions
+		 * would change. However, the base relation partitions constituting
+		 * the given child should remain same for all the joining pairs. Since
+		 * the order in which children are stored in the array of child-joins,
+		 * depends upon partition bounds of the join, which are same for all
+		 * the joining pairs, every joining pair yields the child-joins in the
+		 * same order.
+		 */
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
 
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
+		cnt_parts++;
 	}
+
+	Assert(cnt_parts == joinrel->nparts);
 }
 
 /*
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 4130514952..6354213854 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1594,7 +1594,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1607,24 +1607,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1636,17 +1618,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 637e05e5ca..eca5ce946e 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -23,6 +23,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/pathnodes.h"
 #include "parser/parse_coerce.h"
 #include "partitioning/partbounds.h"
 #include "partitioning/partdesc.h"
@@ -67,6 +68,12 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+typedef struct PartitionMap
+{
+	int from;
+	int to;
+} PartitionMap;
+
 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);
@@ -108,7 +115,58 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 static List *get_range_nulltest(PartitionKey key);
 static bool partition_hbounds_equal(PartitionBoundInfo b1,
 									PartitionBoundInfo b2);
-
+static PartitionBoundInfo partition_range_bounds_merge(
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 List **outer_parts, List **inner_parts,
+							 JoinType jointype, int partnatts,
+							 FmgrInfo *supfuncs, Oid *collations);
+static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype);
+static PartitionBoundInfo partition_hash_bounds_merge(RelOptInfo *outer_rel,
+							RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype);
+static void generate_matching_part_pairs(PartitionMap *outer_maps,
+										 PartitionMap *inner_maps,
+										 int nparts1, int nparts2,
+										 JoinType jointype, int nparts,
+										 List **parts1, List **parts2);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int map_and_merge_partitions(PartitionMap *outer_maps,
+										PartitionMap *inner_maps,
+										int index1, int index2, int *next_index);
+static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *collations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2);
+static bool partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
+						   Oid *collations, PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, int *ub_cmpval,
+						   int *lb_cmpval);
+static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes);
+static bool merge_default_partitions(PartitionBoundInfo outer_bi,
+						 PartitionBoundInfo inner_bi,
+						 PartitionMap *outer_maps,
+						 PartitionMap *inner_maps,
+						 JoinType jointype,
+						 int *next_index, int *default_index);
+static bool merge_null_partitions(PartitionBoundInfo outer_bi,
+								  PartitionBoundInfo inner_bi,
+								  PartitionMap *outer_maps,
+								  PartitionMap *inner_maps,
+								  JoinType jointype,
+								  int *next_index, int *null_index,
+								  int *default_index);
 
 /*
  * get_qual_from_partbound
@@ -2943,3 +3001,1620 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * The function produces the partition bounds for a join between two relations
+ * whose partition bounds are given. The function also returns two lists of
+ * partition indexes one for each of the joining relations. Both the lists
+ * contain the same number of elements. The partition indexes at the same
+ * positions in the lists indicate the pair partitions, one from each side, to
+ * be joined and the position itself corresponds to the index of partition
+ * produced by that child-join in the partitioned join.
+ *
+ * The function returns NULL if we can not find the matching pair of
+ * partitions. This happens if 1. multiple partitions on one side match with
+ * one partition on the other side. 2. a given partition on the outer side
+ * doesn't have a matching partition on the inner side. We can not support the
+ * first case since we don't have a way to represent multiple partitions as a
+ * single relation (RelOptInfo) and then perform join using the ganged
+ * relation. We can not support the second case since the missing inner
+ * partition needs to be represented as an empty relation and we don't have a
+ * way to introduce empty relation during join planning after creating paths
+ * for all the base relations.
+ */
+extern PartitionBoundInfo
+partition_bounds_merge(int partnatts, FmgrInfo *partsupfunc,
+					   Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
+{
+	PartitionBoundInfo 	merged_bounds;
+	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
+						inner_binfo = inner_rel->boundinfo;
+	char				strategy = outer_binfo->strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partsupfunc,
+														partcollation,
+														outer_rel, inner_rel,
+														outer_parts, inner_parts,
+														jointype);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(outer_rel, inner_rel,
+														 outer_parts, inner_parts,
+														 jointype, partnatts,
+														 partsupfunc,
+														 partcollation);
+			break;
+
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = partition_hash_bounds_merge(outer_rel, inner_rel,
+														outer_parts, inner_parts,
+														jointype);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	Assert((*outer_parts == NIL || *inner_parts != NIL) &&
+		   (*inner_parts == NIL || *outer_parts != NIL));
+
+	return merged_bounds;
+}
+
+/*
+ * partition_get_range_bounds
+ *
+ * Given the index of lower bound in datums array, return lower and upper
+ * bounds and the index of the partition with that lower bound.
+ */
+static int
+partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
+						   PartitionRangeBound *lower,
+						   PartitionRangeBound *upper)
+{
+	int			part_index;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The lower bound should correspond to a valid partition. */
+	part_index = bi->indexes[lb_index + 1];
+	Assert(part_index >= 0);
+
+	lower->kind = bi->kind[lb_index];
+	lower->datums = bi->datums[lb_index];
+	lower->lower = true;
+	upper->kind = bi->kind[lb_index + 1];
+	upper->datums = bi->datums[lb_index + 1];
+	upper->lower = false;
+
+	return part_index;
+}
+
+/*
+ * partition_range_get_next_lb_index
+ *
+ * Given the index of lower bound in datums array return the
+ * index of lower bound of the next partition. When the given index corresponds
+ * to the last partition, return number of datums (ndatums).
+ */
+static int
+partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
+{
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The partition index corresponding to the upper bound should be valid. */
+	Assert(bi->indexes[lb_index + 1] >= 0);
+
+	/*
+	 * If there are no bounds left beyond the upper bound, we have reached the
+	 * last partition.
+	 */
+	if (lb_index + 2 < bi->ndatums)
+	{
+		/*
+		 * If the bound next to the upper bound corresponds to no partition,
+		 * that's the next lower bound of the next partition. Otherwise, the
+		 * current upper bound is the lower bound of the next partition.
+		 */
+		if (bi->indexes[lb_index + 2] < 0)
+			return lb_index + 2;
+		else
+			return lb_index + 1;
+	}
+	else
+		return bi->ndatums;
+}
+
+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *partcollations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2)
+{
+	return partition_rbound_cmp(partnatts, partsupfunc, partcollations,
+								bound1->datums, bound1->kind, bound1->lower,
+								bound2);
+}
+
+/*
+ * partition_range_cmp
+ *
+ * Compare the bounds of two range partitions. Set ub_cmpval <, = or > 0, if the
+ * first partition's upper bound is lower than, equal to or higher than the
+ * second partition's upper bound resp. Similarly set lb_cmpval <, =  or > 0,
+ * if the first partition's lower bound is lower than, equal to or higher than
+ * the second partition's lower bound resp.
+ *
+ * Return true, if the ranges overlap, otherwise return false.
+ */
+static bool
+partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
+					PartitionRangeBound *lower_bound1,
+					PartitionRangeBound *upper_bound1,
+					PartitionRangeBound *lower_bound2,
+					PartitionRangeBound *upper_bound2, int *ub_cmpval,
+					int *lb_cmpval)
+{
+	bool		overlap;
+
+	/*
+	 * Compare upper bound of the first partition with the lower bound of the
+	 * second and vice-versa. If lower bound is higher than the upper bound,
+	 * the partitions are not overlapping. All other cases indicate overlapping
+	 * partitions.
+	 */
+	if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+								  lower_bound1, upper_bound2) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = 1;
+		*lb_cmpval = 1;
+	}
+	else if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+									   lower_bound2, upper_bound1) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = -1;
+		*lb_cmpval = -1;
+	}
+	else
+	{
+		overlap = true;
+		*ub_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, upper_bound1,
+											   upper_bound2);
+		*lb_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, lower_bound1,
+											   lower_bound2);
+	}
+
+	return overlap;
+}
+
+/*
+ * partition_range_merge
+ *
+ * Merge the partition bounds of given two partitions such that the join
+ * between the given two partitions fits merged bounds.
+ *
+ * "merged_upper" will be set to one of the given upper bounds and
+ * "merged_lower" will be set to one of the given lower bounds.
+ */
+static void
+partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
+					  Oid *partcollations, JoinType jointype,
+					  PartitionRangeBound *left_lb,
+					  PartitionRangeBound *left_ub,
+					  PartitionRangeBound *right_lb,
+					  PartitionRangeBound *right_ub,
+					  PartitionRangeBound **merged_lb,
+					  PartitionRangeBound **merged_ub)
+{
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = left_ub;
+			*merged_lb = left_lb;
+			break;
+
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) < 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) > 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		case JOIN_FULL:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) > 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) < 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+
+	return;
+}
+
+/*
+ * Add the lower bound of the next range to the list of bounds, if the lower
+ * bound is higher or equal to the previous upper bound. If successful return
+ * true, otherwise false.
+ */
+static bool
+partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
+							  Oid *partcollations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		prev_ub.datums = llast(*merged_datums);
+		prev_ub.kind = llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  next_lb_datums, next_lb_kind, false,
+									  &prev_ub);
+	}
+
+	/*
+	 * The lower bound is lower than the last upper bound, thus does not fit
+	 * the bounds created so far and hence can not be merged with the existing
+	 * bounds.
+	 */
+	if (cmpval < 0)
+		return false;
+
+	/*
+	 * Add bounds of the new merged partition. If the next lower bound is
+	 * higher than the last upper bound, add new range with index
+	 * corresponding to the lower bound as -1. If the merged lower bound
+	 * is same as the last merged upper bound, the last upper bound will be
+	 * reused as the lower bound of the next range.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, next_lb_datums);
+		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	return true;
+}
+
+/*
+ * handle_missing_partition
+ *
+ * If a range appears in one of the joining relations but not the other, a row
+ * in the corresponding partition will not have any join partner in the other
+ * relation, unless the other relation has a default partition. If a given list
+ * value is present in one joining relation but not the other, the default
+ * partition on the other side may contain that value.
+ *
+ * In both these cases, such an extra partition forms a joining pair with the
+ * default partition, if any,  on the other side.
+ *
+ * If the default partition happens to be on the outer side of the join, the
+ * resultant partition will act as the default partition of the join relation.
+ * Otherwise the resultant partition will be associated with the range.
+ *
+ * When the default partition is not present in the other relation, the rows in
+ * the extra partition will be included in the bounds of the join result, if it
+ * appears on the outer side of the join, since all rows from the outer side
+ * are included in the join result.
+ *
+ * This function handles all these cases.
+ *
+ * maps_with_missing and missing_side_default are the partition maps (See
+ * partition_range/list_bounds_merge() for details) and the index of default
+ * partition respectively corresponding the side with missing partition.
+ *
+ * maps_with_extra and extra_part are the partition maps (See
+ * partition_range/list_bounds_merge() for details) and the index of extra
+ * partition respectively corresponding to the side with the extra partition.
+ *
+ * It returns true if the matching succeeds, otherwise returns false.
+ */
+static bool
+handle_missing_partition(PartitionMap *maps_with_missing,
+						 PartitionMap *maps_with_extra,
+						 int missing_side_default,
+						 int extra_part,
+						 bool missing_side_outer,
+						 bool missing_side_inner,
+						 int *next_index, int *default_index,
+						 int *merged_index)
+{
+	bool missing_has_default = (missing_side_default != -1);
+
+	if (missing_has_default)
+	{
+		*merged_index = map_and_merge_partitions(maps_with_missing,
+												 maps_with_extra,
+												 missing_side_default,
+												 extra_part,
+												 next_index);
+		if (*merged_index < 0)
+			return false;
+
+		if (missing_side_outer)
+		{
+			/*
+			 * Default partition on the outer side forms the default
+			 * partition of the join result.
+			 */
+			if (*default_index < 0)
+				*default_index = *merged_index;
+			else if(*default_index != *merged_index)
+			{
+				/*
+				 * Ended up with default partition on the outer side
+				 * being joined with multiple partitions on the inner
+				 * side. We don't support this case.
+				 */
+				return false;
+			}
+
+			/*
+			 * Since the merged partition acts as a default partition, it
+			 * doesn't need a separate index.
+			 */
+			*merged_index = -1;
+		}
+	}
+	else if (missing_side_inner)
+	{
+		/*
+		 * If this partition has already been mapped (say because we
+		 * found an overlapping range earlier), we know where does it
+		 * fit in the join result. Nothing to do in that case. Else
+		 * create a new merged partition.
+		 */
+		PartitionMap *extra_map = &maps_with_extra[extra_part];
+		if (extra_map->to < 0)
+		{
+			extra_map->to = *next_index;
+			*next_index = *next_index + 1;
+			*merged_index = extra_map->to;
+		}
+	}
+
+	return true;
+}
+
+static PartitionMap*
+init_partition_map(RelOptInfo *rel)
+{
+	int i, nparts = rel->nparts;
+	PartitionMap *map;
+
+	map = (PartitionMap *) palloc(sizeof(PartitionMap) * nparts);
+
+	for (i = 0; i < nparts; i++)
+	{
+		map[i].from = -1;
+		map[i].to = -1;
+	}
+
+	return map;
+}
+
+/*
+ * Allocate and initialize partition maps. We maintain four maps, two maps
+ * for each joining relation. pmap[i] gives the partition from the other
+ * relation which would join with ith partition of the given relation.
+ * Partition i from the given relation will join with partition pmap[i]
+ * from the other relation to produce partition mmap[i] of the join (merged
+ * partition).
+ *
+ * pmap[i] = -1 indicates that ith partition of a given relation does not
+ * have a matching partition from the other relation.
+ *
+ * mmap[i] = -1 indicates that ith partition of a given relation does not
+ * contribute to the join result. That can happen only when the given
+ * relation is the inner relation and it doesn't have a matching partition
+ * from the outer relation, hence pmap[i] should be -1.
+ *
+ * In case of an outer join, every partition of the outer join will appear
+ * in the join result, and thus has mmap[i] set for all i. But it's not
+ * necessary that every partition on the outer side will have a matching
+ * partition on the inner side. In such a case, we end up with pmap[i] = -1
+ * and mmap[i] != -1.
+ */
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 List **outer_parts, List **inner_parts,
+							 JoinType jointype, int partnatts,
+							 FmgrInfo *partsupfuncs, Oid *partcollations)
+
+{
+	PartitionMap *outer_maps = NULL;
+	PartitionMap *inner_maps = NULL;
+	int			outer_part = 0;
+	int			inner_part = 0;
+	PartitionBoundInfo merged_bounds = NULL;
+	int			outer_lb_index;
+	int			inner_lb_index;
+	int			next_index;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	List	   *merged_kinds = NIL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			inner_default = inner_bi->default_index;
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	outer_maps = init_partition_map(outer_rel);
+	inner_maps = init_partition_map(inner_rel);
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = 0;
+	inner_lb_index = 0;
+	next_index = 0;
+	while (outer_lb_index < outer_bi->ndatums ||
+		   inner_lb_index < inner_bi->ndatums)
+	{
+		PartitionRangeBound outer_lb, outer_ub,
+							inner_lb, inner_ub,
+							*merged_lb = NULL,
+							*merged_ub = NULL;
+
+		int			merged_index = -1;
+		bool		overlap;
+		bool		finished_outer = false;
+		bool		finished_inner = false;
+
+		/* Result of bounds comparison per partition_rbound_cmp(). */
+		int			ub_cmpval;	/* Upper bounds comparison result. */
+		int			lb_cmpval;	/* Lower bounds comparison result. */
+
+		/* Get the range bounds of the next pair of partitions. */
+		if (outer_lb_index < outer_bi->ndatums)
+			outer_part = partition_get_range_bounds(outer_bi, outer_lb_index,
+												&outer_lb, &outer_ub);
+		else
+			finished_outer = true;
+
+		if (inner_lb_index < inner_bi->ndatums)
+			inner_part = partition_get_range_bounds(inner_bi, inner_lb_index,
+												&inner_lb, &inner_ub);
+		else
+			finished_inner = true;
+
+		Assert(!finished_outer || !finished_inner);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (finished_outer)
+		{
+			overlap = false;
+			ub_cmpval = 1;
+			lb_cmpval = 1;
+		}
+		else if (finished_inner)
+		{
+			overlap = false;
+			ub_cmpval = -1;
+			lb_cmpval = -1;
+		}
+		else
+			overlap = partition_range_cmp(partnatts, partsupfuncs, partcollations,
+										  &outer_lb, &outer_ub, &inner_lb,
+										  &inner_ub, &ub_cmpval, &lb_cmpval);
+
+		if (overlap)
+		{
+			/*
+			 * The rows from overlapping portion of ranges on both sides may
+			 * join, hence the corresponding pair of partitions form a joining
+			 * pair. Match them and produce the bounds of the joint partition
+			 * and its index by merging the bounds according to the type of
+			 * join.
+			 */
+			partition_range_merge(partnatts, partsupfuncs, partcollations,
+								  jointype, &outer_lb, &outer_ub, &inner_lb,
+								  &inner_ub, &merged_lb, &merged_ub);
+
+			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+													outer_part, inner_part,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				/* Failed to match the partitions. */
+				return NULL;
+			}
+
+			/*
+			 * If the ranges overlap but don't exactly match, a row from
+			 * non-overlapping portion of the range from one side of join may
+			 * find its join partner in the previous or next overlapping
+			 * partition or default partition on the other side , if such a
+			 * partition exists. All those cases, if true, will cause one
+			 * partition from that side to match at least two partitions on the
+			 * other side; a case that we do not support now. Previous
+			 * partition has been delt with in the previous iteration of this
+			 * loop, next partition will be delt in the next iteration. We will
+			 * deal with the default partition here.
+			 */
+			if ((lb_cmpval < 0 && inner_has_default) ||
+				/* Non-overlapping range on the lower side of outer range. */
+				(lb_cmpval > 0 && outer_has_default) ||
+				/* Non-overlapping range on the lower side of inner range. */
+				(ub_cmpval < 0 && outer_has_default) ||
+				/* Non-overlapping range on the upper side of inner range. */
+				(ub_cmpval > 0 && inner_has_default))
+				/* Non-overlapping range on the upper side of outer range. */
+				return NULL;
+		}
+
+		if (ub_cmpval == 0)
+		{
+			/* Upper bounds of both the ranges match. */
+			Assert(overlap);
+
+			/* Move to the next pair of partitions. */
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the inner side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &outer_lb;
+				merged_ub = &outer_ub;
+
+				/*
+				 * For a FULL join, inner relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the inner relation acts as
+				 * INNER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_inner = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_outer = (jointype == JOIN_FULL);
+				if (!handle_missing_partition(inner_maps,
+											  outer_maps,
+											  inner_default,
+											  outer_part,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the outer side. */
+			Assert(!finished_outer);
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+		}
+		else
+		{
+			Assert(ub_cmpval > 0);
+
+			/* Upper bound of outer range higher than that of the inner. */
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the outer side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &inner_lb;
+				merged_ub = &inner_ub;
+
+				/*
+				 * For a FULL join, outer relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the outer relation acts as
+				 * OUTER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_outer = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_inner = (jointype == JOIN_FULL);
+
+				if (!handle_missing_partition(outer_maps,
+											  inner_maps,
+											  outer_default,
+											  inner_part,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the inner side. */
+			Assert (!finished_inner);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+
+		if (merged_index < 0)
+		{
+			/* We didn't find a new merged partition. */
+			continue;
+		}
+
+		/*
+		 * We have a valid partition index for the next partition of join. The
+		 * partition should have valid range.
+		 */
+		Assert(merged_lb && merged_ub);
+
+		/* Try merging new lower bound with the last upper bound. */
+		if (!partition_range_merge_next_lb(partnatts, partsupfuncs,
+										   partcollations,
+										   merged_lb->datums,
+										   merged_lb->kind, &merged_datums,
+										   &merged_kinds, &merged_indexes))
+			return NULL;
+
+		/* Add upper bound with the merged partition index. */
+		merged_datums = lappend(merged_datums, merged_ub->datums);
+		merged_kinds = lappend(merged_kinds, merged_ub->kind);
+		merged_indexes = lappend_int(merged_indexes, merged_index);
+	}
+
+	if (!merge_default_partitions(outer_bi, inner_bi,
+										  outer_maps, inner_maps,
+										  jointype, &next_index,
+										  &default_index))
+		return NULL;
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_maps, inner_maps,
+								 outer_nparts, inner_nparts,
+								 jointype, next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes,
+													  merged_kinds,
+													  -1, default_index);
+	}
+
+	/* Free any memory we used in this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype)
+{
+	PartitionMap *outer_maps = NULL;
+	PartitionMap *inner_maps = NULL;
+	int			cnto;
+	int			cnti;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			next_index = 0;
+	int			null_index;
+	int			default_index = -1;
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			      *outer_indexes = outer_bi->indexes;
+	int			      *inner_indexes = inner_bi->indexes;
+	int				   outer_default = outer_bi->default_index;
+	int				   inner_default = inner_bi->default_index;
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	outer_maps = init_partition_map(outer_rel);
+	inner_maps = init_partition_map(inner_rel);
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			o_index;
+		int			i_index;
+		int			cmpval;
+		int			merged_index = -1;
+		Datum	   *merged_datum;
+		bool		finished_inner;
+		bool		finished_outer;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+		{
+			finished_outer = true;
+			odatums = NULL;
+			o_index = -1;
+		}
+		else
+		{
+			finished_outer = false;
+			odatums = outer_bi->datums[cnto];
+			o_index = outer_indexes[cnto];
+		}
+
+		if (cnti >= inner_bi->ndatums)
+		{
+			finished_inner = true;
+			idatums = NULL;
+			i_index = -1;
+		}
+		else
+		{
+			finished_inner = false;
+			idatums = inner_bi->datums[cnti];
+			i_index = inner_indexes[cnti];
+		}
+
+		/* If we exhausted both the sides, we won't enter the loop. */
+		Assert(!finished_inner || !finished_outer);
+
+		if (finished_outer)
+			cmpval = 1;
+		else if (finished_inner)
+			cmpval = -1;
+		else
+		{
+			/* Every list datum should map to a valid partition index. */
+			Assert(o_index >= 0 && i_index >= 0 &&
+				   odatums != NULL && idatums != NULL);
+
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0], idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+													o_index, i_index,
+													&next_index);
+
+			if (merged_index < 0)
+				return NULL;
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			/* A datum missing from the inner side. */
+			merged_index = -1;
+			merged_datum = odatums;
+
+			/*
+			 * For a FULL join, inner relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the inner relation acts as
+			 * INNER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_inner = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_outer = (jointype == JOIN_FULL);
+
+			if (!handle_missing_partition(inner_maps,
+										  outer_maps,
+										  inner_default,
+										  o_index,
+										  missing_side_outer,
+										  missing_side_inner,
+										  &next_index,
+										  &default_index,
+										  &merged_index))
+				return NULL;
+
+			/* Move to the next datum on the outer side. */
+			Assert(!finished_outer);
+			cnto++;
+		}
+		else
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			Assert(cmpval > 0);
+
+			/* A datum missing from the outer side. */
+			merged_index = -1;
+			merged_datum = idatums;
+
+			/*
+			 * For a FULL join, outer relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the outer relation acts as
+			 * OUTER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_outer = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_inner = (jointype == JOIN_FULL);
+
+			if (!handle_missing_partition(outer_maps,
+										  inner_maps,
+										  outer_default,
+										  i_index,
+										  missing_side_outer,
+										  missing_side_inner,
+										  &next_index,
+										  &default_index,
+										  &merged_index))
+				return NULL;
+
+			/* Move to the next datum on the right side. */
+			Assert(!finished_inner);
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	if (!merge_null_partitions(outer_bi, inner_bi,
+							   outer_maps, inner_maps,
+							   jointype, &next_index, &null_index,
+							   &default_index))
+		return NULL;
+
+	if (!merge_default_partitions(outer_bi, inner_bi,
+								  outer_maps, inner_maps,
+								  jointype, &next_index,
+								  &default_index))
+		return NULL;
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_maps, inner_maps,
+								 outer_nparts, inner_nparts,
+								 jointype, next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes, NIL,
+													  null_index, default_index);
+	}
+
+	/* Free up all extra memory before returning from this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_bounds_merge()'s arm for hash partitioned tables.
+ *
+ * If the given two hash bounds are same, the function returns the first one
+ * without any change, alongwith the lists of matching partitions. Otherwise it
+ * returns NULL.
+ *
+ * We could try merging the bounds when both the bounds have same greatest
+ * modulii. But there seems to be hardly any requirement for the same.
+ */
+static PartitionBoundInfo
+partition_hash_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype)
+{
+	int			nparts;
+	int			cnt;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * Hash partitioned table does not have explicit NULL accepting partition
+	 * and also does not have a default partition.
+	 */
+	Assert(!partition_bound_has_default(outer_bi) &&
+		   !partition_bound_has_default(inner_bi));
+	Assert(!partition_bound_accepts_nulls(outer_bi) &&
+		   !partition_bound_accepts_nulls(inner_bi));
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+
+	if (outer_nparts != inner_nparts)
+		return NULL;
+	nparts = outer_nparts;
+
+	if (outer_bi->ndatums != inner_bi->ndatums ||
+		!partition_hbounds_equal(outer_bi, inner_bi))
+		return NULL;
+
+	 /*
+	  * Cook up list of matching partitions. Since bounds are exactly same the
+	  * partitions at the same position from both the relations match.
+	  */
+	for (cnt = 0; cnt < nparts; cnt++)
+	{
+		*outer_parts = lappend_int(*outer_parts, cnt);
+		*inner_parts = lappend_int(*inner_parts, cnt);
+	}
+
+	return outer_bi;
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * If the two given partitions (given by index1 and index2 resp.) are
+ * already mapped to each other return the index of corresponding partition in
+ * the merged set of partitions.  If they do not have a merged partition
+ * associated with them, assign a new merged partition index.  If the
+ * partitions are already mapped and their mapped partitions are different from
+ * each other, they can not be merged, so return -1.
+ *
+ * partmaps1[i] gives the mapping of partitions for both relations. It
+ * describes which partition of relation 2 matches ith partition of relation 1,
+ * and which partition in the merged set matches ith partition of relation 1
+ * maps to. Similarly for partmap2.
+ *
+ * index1 and index2 are the indexes of matching partition from respective
+ * relations.
+ *
+ * *next_index is used and incremented when the given partitions require a new
+ * merged partition.
+ */
+
+static int
+map_and_merge_partitions(PartitionMap *partmaps1, PartitionMap *partmaps2,
+						 int index1, int index2, int *next_index)
+{
+	PartitionMap 	*partmap1 = &partmaps1[index1];
+	PartitionMap 	*partmap2 = &partmaps2[index2];
+	int				merged_index;
+
+	/*
+	 * If both the partitions are not mapped to each other, update the
+	 * maps.
+	 */
+	if (partmap1->from < 0 && partmap2->from < 0)
+	{
+		partmap1->from = index2;
+		partmap2->from = index1;
+	}
+
+	/*
+	 * If the given to partitions map to each other, find the corresponding
+	 * merged partition index .
+	 */
+	if (partmap1->from == index2 && partmap2->from == index1)
+	{
+		/*
+		 * If both the partitions are mapped to the same merged partition, get
+		 * the index of merged partition.
+		 */
+		if (partmap1->to == partmap2->to)
+		{
+			merged_index = partmap1->to;
+
+			/*
+			 * If the given two partitions do not have a merged partition
+			 * associated with them, allocate a new merged partition.
+			 */
+			if (merged_index < 0)
+			{
+				merged_index = *next_index;
+				*next_index = *next_index + 1;
+				partmap1->to = merged_index;
+				partmap2->to = merged_index;
+			}
+		}
+
+		/*
+		 * If partition from one relation was mapped to a merged partition but
+		 * not the partition from the other relation, map the same merged
+		 * partition to the partition from other relation, since matching
+		 * partitions map to the same merged partition.
+		 */
+		else if (partmap1->to >= 0 && partmap2->to < 0)
+		{
+			partmap2->to = partmap1->to;
+			merged_index = partmap1->to;
+		}
+		else if (partmap1->to < 0 && partmap2->to >= 0)
+		{
+			partmap1->to = partmap2->to;
+			merged_index = partmap2->to;
+		}
+		else
+		{
+			Assert(partmap1->to != partmap2->to &&
+				   partmap1->to >= 0 && partmap2->to >= 0);
+
+			/*
+			 * Both the partitions map to different merged partitions. This
+			 * means that multiple partitions from one relation matches to one
+			 * partition from the other relation. Partition-wise join does not
+			 * handle this case right now, since it requires ganging multiple
+			 * partitions together (into one RelOptInfo).
+			 */
+			merged_index = -1;
+		}
+	}
+	else
+	{
+		/*
+		 * Multiple partitions from one relation map to one partition from the
+		 * other relation. Partition-wise join does not handle this case right
+		 * now, since it requires ganging multiple partitions together (into
+		 * one RelOptInfo).
+		 */
+		merged_index = -1;
+	}
+
+	return merged_index;
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * partmaps1 map each partition from either side of the join to a merged
+ * partition resp. E.g. partmaps1[i].to gives the merged partition to which ith
+ * partition of first relation maps. Similarly for partmap2. If
+ * partmaps1[i].to == partmaps2[j].to, i and j form the matching pair of
+ * partitions.
+ *
+ * Given these maps this function produces the list pairs of partitions which
+ * when joined produce the merged partitions in the order of merged partition
+ * indexes.
+ *
+ * nparts1 and nparts2 are the number of partitions of the joining relations
+ * resp.
+ *
+ * nparts is the number of merged partitions.
+ *
+ * If successful, the pairs of partitions are returned as two separate lists,
+ * parts1 and parts2 resp., one for each side. Otherwise, those lists will be
+ * set to NIL.
+ */
+static void
+generate_matching_part_pairs(PartitionMap *partmaps1, PartitionMap *partmaps2,
+							 int nparts1, int nparts2,
+							 JoinType jointype, int nparts,
+							 List **matched_parts1, List **matched_parts2)
+{
+	bool	merged = true;
+	int		*matching1 = (int *) palloc(sizeof(int) * nparts),
+			*matching2 = (int *) palloc(sizeof(int) * nparts);
+	int 	i;
+
+	*matched_parts1 = NIL;
+	*matched_parts2 = NIL;
+
+	/* Set pairs of matching partitions. */
+	for (i = 0; i < nparts; i++)
+	{
+		if (i >= nparts1)
+			matching1[i] = -1;
+		else
+		{
+			PartitionMap outer_map = partmaps1[i];
+
+			if (outer_map.to >= 0)
+			{
+				Assert(outer_map.to < nparts);
+				matching1[outer_map.to] = i;
+			}
+		}
+
+		if (i >= nparts2)
+			matching2[i] = -1;
+		else
+		{
+			PartitionMap inner_map = partmaps2[i];
+
+			if (inner_map.to >= 0)
+			{
+				Assert(inner_map.to < nparts);
+				matching2[inner_map.to] = i;
+			}
+		}
+	}
+
+	/*
+	 * If we have a partition missing on an inner side, we need to add a dummy
+	 * relation which joins with the outer partition. If the inner relation
+	 * happens to be a base relation, it will require adding a dummy child
+	 * base relation during join processing. Right now, we freeze the base
+	 * relation arrays like PlannerInfo::simple_rte_array after planning for
+	 * base relations. Adding a new (dummy) base relation would require some
+	 * changes to that. So, right now, we do not implement partition-wise join
+	 * in such cases.
+	 */
+	for (i = 0; i < nparts; i++)
+	{
+		int			part1 = matching1[i];
+		int			part2 = matching2[i];
+
+		/* At least one of the partitions should exist. */
+		Assert(part1 >= 0 || part2 >= 0);
+
+		switch (jointype)
+		{
+			case JOIN_INNER:
+			case JOIN_SEMI:
+
+				/*
+				 * An inner or semi join can not return any row when the
+				 * matching partition on either side is missing. We should
+				 * have eliminated all such cases while merging the bounds.
+				 */
+				Assert(part1 >= 0 && part2 >= 0);
+				break;
+
+			case JOIN_LEFT:
+			case JOIN_ANTI:
+				Assert(part1 >= 0);
+				if (part2 < 0)
+					merged = false;
+				break;
+
+			case JOIN_FULL:
+				if (part1 < 0 || part2 < 0)
+					merged = false;
+				break;
+
+			default:
+				elog(ERROR, "unrecognized join type: %d", (int) jointype);
+		}
+
+		if (!merged)
+			break;
+
+		*matched_parts1 = lappend_int(*matched_parts1, part1);
+		*matched_parts2 = lappend_int(*matched_parts2, part2);
+	}
+
+	pfree(matching1);
+	pfree(matching2);
+
+	if (!merged)
+	{
+		list_free(*matched_parts1);
+		list_free(*matched_parts2);
+		*matched_parts1 = NIL;
+		*matched_parts2 = NIL;
+	}
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * Merge default partitions from both sides, if any, and assign the default
+ * partition for the join result, if necessary.
+ *
+ * If both the relations have default partitions, try mapping those to each
+ * other. If the mapping succeeds corresponding merged partition will act as
+ * the default partition of the join result.
+ *
+ * If inner side of the join has default but not the outer side, rows in it
+ * won't appear in the join result. So don't create a default partition. If
+ * outer side of the join has default but not the inner side, rows in it will
+ * appear in the join result, so create a default merged partition.
+ */
+static bool
+merge_default_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+						 PartitionMap *outer_maps, PartitionMap *inner_maps,
+						 JoinType jointype, int *next_index, int *default_index)
+{
+	int				outer_default = outer_bi->default_index;
+	int				inner_default = inner_bi->default_index;
+	bool			outer_has_default = partition_bound_has_default(outer_bi);
+	bool			inner_has_default = partition_bound_has_default(inner_bi);
+	bool			merged = true;
+	PartitionMap 	*outer_default_map = NULL;
+	PartitionMap 	*inner_default_map = NULL;
+
+	if (outer_has_default)
+		outer_default_map = &outer_maps[outer_default];
+
+	if (inner_has_default)
+		inner_default_map = &inner_maps[inner_default];
+
+	if (!outer_has_default && !inner_has_default)
+		Assert(*default_index < 0);
+	else if (outer_default_map != NULL && inner_default_map == NULL)
+	{
+		if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+			jointype == JOIN_ANTI)
+		{
+			if (outer_default_map->to < 0)
+			{
+				outer_default_map->to = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = outer_default_map->to;
+			}
+			else
+				Assert(*default_index == outer_default_map->to);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else if (outer_default_map == NULL && inner_default_map != NULL)
+	{
+		if (jointype == JOIN_FULL)
+		{
+			if (inner_default_map->to < 0)
+			{
+				inner_default_map->to = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = inner_default_map->to;
+			}
+			else
+				Assert(*default_index == inner_default_map->to);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		*default_index = map_and_merge_partitions(outer_maps, inner_maps,
+												  outer_default, inner_default,
+												  next_index);
+
+		if (*default_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
+
+/*
+ * merge_null_partitions
+ *
+ * Merge NULL partitions, i.e. a partition that can hold NULL values for a list
+ * partitioned table, if any. Find the index of merged partition to which the
+ * NULL values would belong in the join result. If one joining relation has a
+ * NULL partition but not the other, try matching it with the default partition
+ * from the other relation since the default partition may have rows with NULL
+ * partition key. We can eliminate a NULL partition when it appears only on the
+ * inner side of the join and the outer side doesn't have a default partition.
+ *
+ * When the equality operator used for join is strict, two NULL values will not
+ * be considered as equal, and thus a NULL partition can be eliminated for an
+ * inner join. But we don't check the strictness operator here.
+ */
+static bool
+merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_maps, PartitionMap *inner_maps,
+					  JoinType jointype, int *next_index,
+					  int *null_index, int *default_index)
+{
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	int			outer_ni = outer_bi->null_index;
+	int			inner_ni = inner_bi->null_index;
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	bool		merged = true;
+
+	if (!outer_has_null && !inner_has_null)
+		*null_index = -1;
+	else if (outer_has_null && !inner_has_null)
+	{
+		int			merged_index;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, inner relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the inner relation acts as
+		 * INNER relation. For INNER and SEMI join OUTER and INNER
+		 * differentiation is immaterial.
+		 */
+		missing_side_inner = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_outer = (jointype == JOIN_FULL);
+
+		merged = handle_missing_partition(inner_maps,
+										  outer_maps,
+										  inner_default,
+										  outer_ni,
+										  missing_side_outer,
+										  missing_side_inner, next_index,
+										  default_index, &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join to which the outer null partition maps
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = outer_maps[outer_ni].to;
+	}
+	else if (!outer_has_null && inner_has_null)
+	{
+		int			merged_index;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, outer relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the outer relation acts as OUTER
+		 * relation. For INNER and SEMI join OUTER and INNER differentiation is
+		 * immaterial.
+		 */
+		missing_side_outer = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_inner = (jointype == JOIN_FULL);
+		merged = handle_missing_partition(outer_maps,
+										  inner_maps,
+										  outer_default,
+										  inner_ni,
+										  missing_side_outer,
+										  missing_side_inner,
+										  next_index, default_index,
+										  &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join, to which the outer side null partition maps,
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = inner_maps[inner_ni].to;
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		*null_index = map_and_merge_partitions(outer_maps,
+											   inner_maps,
+											   outer_ni,
+											   inner_ni,
+											   next_index);
+		if (*null_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index b1ae39ad63..fb2dc1cbb1 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -16,6 +16,7 @@
 #include "nodes/pg_list.h"
 #include "partitioning/partdefs.h"
 #include "utils/relcache.h"
+struct RelOptInfo;				/* avoid including pathnodes.h here */
 
 
 /*
@@ -107,5 +108,11 @@ extern int partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 							  int nvalues, Datum *values, bool *is_equal);
 extern int partition_hash_bsearch(PartitionBoundInfo boundinfo,
 					   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   struct RelOptInfo *outer_rel,
+					   struct RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
-- 
2.18.0

0003-Tests-for-0-1-1-1-and-1-0-partition-matching-v19.patchapplication/x-patch; name=0003-Tests-for-0-1-1-1-and-1-0-partition-matching-v19.patchDownload
From 445852d92d377d62dac7d1e15446a683baeb9164 Mon Sep 17 00:00:00 2001
From: Amul Sul <amul.sul@enterprisedb.com>
Date: Tue, 5 Mar 2019 00:40:55 -0500
Subject: [PATCH 3/3] Tests for 0:1, 1:1 and 1:0 partition matching

Rajkumar Raghuvanshi and Ashutosh Bapat.
---
 src/test/regress/expected/partition_join.out | 3955 +++++++++++++++---
 src/test/regress/sql/partition_join.sql      |  427 +-
 2 files changed, 3767 insertions(+), 615 deletions(-)

diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index bbdc373782..7954ba2c91 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -8,59 +8,86 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-(4 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+(7 rows)
 
 -- left outer join, with whole-row reference; partitionwise join does not apply
 EXPLAIN (COSTS OFF)
@@ -72,35 +99,50 @@ SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER
    ->  Hash Right Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(22 rows)
 
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-      t1      |      t2      
---------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
- (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
- (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
- (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
- (550,0,0550) | 
-(12 rows)
+       t1       |       t2       
+----------------+----------------
+ (-250,0,-0250) | 
+ (-200,0,-0200) | 
+ (-150,0,-0150) | (0,-150,-0150)
+ (-100,0,-0100) | 
+ (-50,0,-0050)  | 
+ (0,0,0000)     | (0,0,0000)
+ (50,0,0050)    | 
+ (100,0,0100)   | 
+ (150,0,0150)   | (0,150,0150)
+ (200,0,0200)   | 
+ (250,0,0250)   | 
+ (300,0,0300)   | (0,300,0300)
+ (350,0,0350)   | 
+ (400,0,0400)   | 
+ (450,0,0450)   | (0,450,0450)
+ (500,0,0500)   | 
+ (550,0,0550)   | 
+ (600,0,0600)   | (0,600,0600)
+ (650,0,0650)   | 
+ (700,0,0700)   | 
+ (750,0,0750)   | (0,750,0750)
+(21 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
@@ -112,35 +154,53 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
    ->  Append
          ->  Hash Right Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Right Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t2_2.b)
-(20 rows)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+      |       | -225 | -0225
+      |       |  -75 | -0075
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(14 rows)
 
 -- full outer join, with placeholder vars
 EXPLAIN (COSTS OFF)
@@ -148,8 +208,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                             QUERY PLAN                            
 ------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               Filter: (((50) = prt1_p0.a) OR ((75) = prt2_p0.b))
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
@@ -174,7 +242,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               Filter: (((50) = prt1_p4.a) OR ((75) = prt2_p4.b))
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
@@ -211,8 +287,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Hash Left Join
+               Hash Cond: (prt1_p0.a = b)
+               ->  Seq Scan on prt1_p0
+                     Filter: ((a < 450) AND (b = 0))
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
          ->  Hash Left Join
                Hash Cond: (prt1_p1.a = b)
                ->  Seq Scan on prt1_p1
@@ -227,29 +310,42 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                ->  Hash
                      ->  Seq Scan on prt1_p2
                            Filter: ((a < 450) AND (b = 0))
-(17 rows)
+(24 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-(9 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+(14 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
                          QUERY PLAN                         
 ------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = b)
+               Filter: ((prt1_p0.b = 0) OR (a = 0))
+               ->  Seq Scan on prt1_p0
+                     Filter: (a < 450)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = b)
                Filter: ((prt1_p1.b = 0) OR (a = 0))
@@ -274,64 +370,153 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                ->  Hash
                      ->  Result
                            One-Time Filter: false
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt2_p4.b = a)
+               Filter: ((b = 0) OR (prt2_p4.a = 0))
+               ->  Seq Scan on prt2_p4
+                     Filter: (b > 250)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
-(12 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+      |       | 375 | 0375
+      |       | 450 | 0450
+      |       | 525 | 0525
+      |       | 600 | 0600
+      |       | 675 | 0675
+      |       | 750 | 0750
+(20 rows)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Semi Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Semi Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                            Filter: (a = 0)
-         ->  Nested Loop Semi Join
-               Join Filter: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
-               ->  Materialize
-                     ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
-(24 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt1_p3 t2_3
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+                           Filter: (b = 0)
+(37 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
 
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
@@ -342,27 +527,82 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
    ->  Append
          ->  Hash Anti Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Hash Anti Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Hash Anti Join
                Hash Cond: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
-(17 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(27 rows)
 
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
-  sum  |         avg          | sum  |         avg         
--------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+  sum  |         avg          | sum  |        avg         
+-------+----------------------+------+--------------------
+ 95550 | 273.0000000000000000 | 2200 | 6.2857142857142857
 (1 row)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Append
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p0 t1
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+               Index Cond: (a = t1.b)
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.b = t2_1.a)
+         ->  Seq Scan on prt2_p1 t1_1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p1 t2_1
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p2 t1_2
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+               Index Cond: (a = t1_2.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p3 t1_3
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+               Index Cond: (a = t1_3.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p4 t1_4
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+               Index Cond: (a = t1_4.b)
+(27 rows)
+
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+  b   |   c   
+------+-------
+ -225 | -0225
+  -75 | -0075
+   75 | 0075
+  225 | 0225
+  375 | 0375
+  525 | 0525
+  675 | 0675
+(7 rows)
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -374,49 +614,74 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
+                     ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
                            Index Cond: (a = t1.a)
-                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t3
                            Index Cond: (b = t2.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_1
+                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
                            Index Cond: (a = t1_1.a)
-                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3_1
                            Index Cond: (b = t2_1.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_2
+                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
                            Index Cond: (a = t1_2.a)
-                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_2
                            Index Cond: (b = t2_2.a)
-(27 rows)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                           Index Cond: (a = t1_3.a)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_3
+                           Index Cond: (b = t2_3.a)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                           Index Cond: (a = t1_4.a)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t3_4
+                           Index Cond: (b = t2_4.a)
+(43 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   | t2a | t3a | least 
------+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
- 100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
- 250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
- 400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
- 550 | 0 | 0550 |     |     |      
-(12 rows)
+  a   | b |   c   | t2a  | t3a | least 
+------+---+-------+------+-----+-------
+ -250 | 0 | -0250 |      |     |      
+ -200 | 0 | -0200 |      |     |      
+ -150 | 0 | -0150 | -150 |   0 |  -150
+ -100 | 0 | -0100 |      |     |      
+  -50 | 0 | -0050 |      |     |      
+    0 | 0 | 0000  |    0 |   0 |     0
+   50 | 0 | 0050  |      |     |      
+  100 | 0 | 0100  |      |     |      
+  150 | 0 | 0150  |  150 |   0 |   150
+  200 | 0 | 0200  |      |     |      
+  250 | 0 | 0250  |      |     |      
+  300 | 0 | 0300  |  300 |   0 |   300
+  350 | 0 | 0350  |      |     |      
+  400 | 0 | 0400  |      |     |      
+  450 | 0 | 0450  |  450 |   0 |   450
+  500 | 0 | 0500  |      |     |      
+  550 | 0 | 0550  |      |     |      
+  600 | 0 | 0600  |  600 |   0 |   600
+  650 | 0 | 0650  |      |     |      
+  700 | 0 | 0700  |      |     |      
+  750 | 0 | 0750  |  750 |   0 |   750
+(21 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
@@ -430,64 +695,95 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
          Hash Cond: ((t1.c)::text = (t2.c)::text)
          Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Seq Scan on prt1_p1 t1
-               ->  Seq Scan on prt1_p2 t1_1
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
          ->  Hash
                ->  Append
                      ->  Hash Join
                            Hash Cond: (t2.a = t3.b)
-                           ->  Seq Scan on prt1_p1 t2
+                           ->  Seq Scan on prt1_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t3
+                                 ->  Seq Scan on prt2_p0 t3
                      ->  Hash Join
                            Hash Cond: (t2_1.a = t3_1.b)
-                           ->  Seq Scan on prt1_p2 t2_1
+                           ->  Seq Scan on prt1_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t3_1
+                                 ->  Seq Scan on prt2_p1 t3_1
                      ->  Hash Join
                            Hash Cond: (t2_2.a = t3_2.b)
-                           ->  Seq Scan on prt1_p3 t2_2
+                           ->  Seq Scan on prt1_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p3 t3_2
-(26 rows)
+                                 ->  Seq Scan on prt2_p2 t3_2
+                     ->  Hash Join
+                           Hash Cond: (t2_3.a = t3_3.b)
+                           ->  Seq Scan on prt1_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t3_3
+                     ->  Hash Join
+                           Hash Cond: (t2_4.a = t3_4.b)
+                           ->  Seq Scan on prt1_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t3_4
+(38 rows)
 
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, t2.b t2b, t2.c t2c, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
-  a  | t2a | t2c  
------+-----+------
-   0 |   0 | 0000
-  50 |     | 
- 100 |     | 
- 150 | 150 | 0150
- 200 |     | 
- 250 |     | 
- 300 | 300 | 0300
- 350 |     | 
- 400 |     | 
- 450 | 450 | 0450
- 500 |     | 
- 550 |     | 
-(12 rows)
+  a   | t2a  |  t2c  
+------+------+-------
+ -250 |      | 
+ -200 |      | 
+ -150 | -150 | -0150
+ -100 |      | 
+  -50 |      | 
+    0 |    0 | 0000
+   50 |      | 
+  100 |      | 
+  150 |  150 | 0150
+  200 |      | 
+  250 |      | 
+  300 |  300 | 0300
+  350 |      | 
+  400 |      | 
+  450 |  450 | 0450
+  500 |      | 
+  550 |      | 
+  600 |  600 | 0600
+  650 |      | 
+  700 |      | 
+  750 |  750 | 0750
+(21 rows)
 
 --
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
@@ -498,32 +794,49 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
    ->  Append
          ->  Hash Join
                Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
-               ->  Seq Scan on prt2_e_p1 t2
+               ->  Seq Scan on prt2_e_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 t1
+                     ->  Seq Scan on prt1_e_p0 t1
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
-               ->  Seq Scan on prt2_e_p2 t2_1
+               Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+               ->  Seq Scan on prt1_e_p1 t1_1
+                     Filter: (c = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 t1_1
-                           Filter: (c = 0)
+                     ->  Seq Scan on prt2_e_p1 t2_1
          ->  Hash Join
                Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
-               ->  Seq Scan on prt2_e_p3 t2_2
+               ->  Seq Scan on prt2_e_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 t1_2
+                     ->  Seq Scan on prt1_e_p2 t1_2
                            Filter: (c = 0)
-(21 rows)
+         ->  Hash Join
+               Hash Cond: (((t2_3.b + t2_3.a) / 2) = ((t1_3.a + t1_3.b) / 2))
+               ->  Seq Scan on prt2_e_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p3 t1_3
+                           Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (((t2_4.b + t2_4.a) / 2) = ((t1_4.a + t1_4.b) / 2))
+               ->  Seq Scan on prt2_e_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4 t1_4
+                           Filter: (c = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
- 150 | 0 | 150 | 0
- 300 | 0 | 300 | 0
- 450 | 0 | 450 | 0
-(4 rows)
+  a   | c |  b   | c 
+------+---+------+---
+ -250 | 0 | -250 | 0
+ -100 | 0 | -100 | 0
+    0 | 0 |    0 | 0
+    0 | 0 |    0 | 0
+  150 | 0 |  150 | 0
+  300 | 0 |  300 | 0
+  450 | 0 |  450 | 0
+  600 | 0 |  600 | 0
+  750 | 0 |  750 | 0
+(9 rows)
 
 --
 -- N-way join
@@ -536,154 +849,232 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = ((t3.a + t3.b) / 2))
+               Join Filter: (t1.a = t2.b)
                ->  Hash Join
-                     Hash Cond: (t2.b = t1.a)
-                     ->  Seq Scan on prt2_p1 t2
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on prt1_e_p0 t3
                      ->  Hash
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                     Index Cond: (((a + b) / 2) = t2.b)
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
+                     Index Cond: (b = ((t3.a + t3.b) / 2))
          ->  Nested Loop
                Join Filter: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_1.b = t1_1.a)
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
+               ->  Index Scan using iprt1_e_p4_ab2 on prt1_e_p1 t3_1
                      Index Cond: (((a + b) / 2) = t2_1.b)
          ->  Nested Loop
                Join Filter: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_2.b = t1_2.a)
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
+               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_2
                      Index Cond: (((a + b) / 2) = t2_2.b)
-(33 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = ((t3_3.a + t3_3.b) / 2))
+               ->  Nested Loop
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t2_3.b)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t2_4.b)
+               ->  Hash Join
+                     Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                     ->  Seq Scan on prt1_e_p4 t3_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_p4 t1_4
+                                 Filter: (b = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = ((t3_4.a + t3_4.b) / 2))
+(52 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
-(4 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(8 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                          QUERY PLAN                          
---------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Hash Right Join
                Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
-               ->  Seq Scan on prt1_e_p1 t3
+               ->  Seq Scan on prt1_e_p0 t3
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2.b = t1.a)
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_1.a)
-               ->  Seq Scan on prt1_e_p2 t3_1
+               ->  Seq Scan on prt1_e_p1 t3_1
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_1.b = t1_1.a)
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_2.a)
-               ->  Seq Scan on prt1_e_p3 t3_2
+               ->  Seq Scan on prt1_e_p2 t3_2
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_2.b = t1_2.a)
-                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                                        Filter: (b = 0)
-(33 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+               ->  Seq Scan on prt1_e_p4 t3_4
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_4.b = t1_4.a)
+                           ->  Seq Scan on prt2_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_p4 t1_4
+                                       Filter: (b = 0)
+(51 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                            QUERY PLAN                             
--------------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p1 t3
+                           ->  Seq Scan on prt1_e_p0 t3
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p1_b on prt2_p1 t2
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
                      Index Cond: (b = t1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p2 t3_1
+                           ->  Seq Scan on prt1_e_p1 t3_1
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
+               ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
                      Index Cond: (b = t1_1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p3 t3_2
+                           ->  Seq Scan on prt1_e_p2 t3_2
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
                      Index Cond: (b = t1_2.a)
-(30 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_e_p3 t3_3
+                           Filter: (c = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                           Index Cond: (a = ((t3_3.a + t3_3.b) / 2))
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Nested Loop Left Join
+               ->  Hash Right Join
+                     Hash Cond: (t1_4.a = ((t3_4.a + t3_4.b) / 2))
+                     ->  Seq Scan on prt1_p4 t1_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p4 t3_4
+                                 Filter: (c = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = t1_4.a)
+(47 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -692,21 +1083,34 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                                                    QUERY PLAN                                                   
 ----------------------------------------------------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   Sort Key: prt1_p0.a, prt2_p0.b, ((prt1_e_p0.a + prt1_e_p0.b))
    ->  Append
          ->  Hash Full Join
-               Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
-               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               Hash Cond: (prt1_p0.a = ((prt1_e_p0.a + prt1_e_p0.b) / 2))
+               Filter: ((prt1_p0.a = (50)) OR (prt2_p0.b = (75)) OR (((prt1_e_p0.a + prt1_e_p0.b) / 2) = (50)))
                ->  Hash Full Join
-                     Hash Cond: (prt1_p1.a = prt2_p1.b)
-                     ->  Seq Scan on prt1_p1
+                     Hash Cond: (prt1_p0.a = prt2_p0.b)
+                     ->  Seq Scan on prt1_p0
                            Filter: (b = 0)
                      ->  Hash
-                           ->  Seq Scan on prt2_p1
+                           ->  Seq Scan on prt2_p0
                                  Filter: (a = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1
+                     ->  Seq Scan on prt1_e_p0
                            Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = prt1_p1.a)
+               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               ->  Seq Scan on prt1_e_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on prt1_p1
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1
+                                       Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
                Filter: ((prt1_p2.a = (50)) OR (prt2_p2.b = (75)) OR (((prt1_e_p2.a + prt1_e_p2.b) / 2) = (50)))
@@ -733,7 +1137,20 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                ->  Hash
                      ->  Seq Scan on prt1_e_p3
                            Filter: (c = 0)
-(42 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = ((prt1_e_p4.a + prt1_e_p4.b) / 2))
+               Filter: ((prt1_p4.a = (50)) OR (prt2_p4.b = (75)) OR (((prt1_e_p4.a + prt1_e_p4.b) / 2) = (50)))
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = prt2_p4.b)
+                     ->  Seq Scan on prt1_p4
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4
+                                 Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4
+                           Filter: (c = 0)
+(68 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
@@ -751,172 +1168,260 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = t1_3.b)
+               Join Filter: (t1.a = t1_5.b)
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Join
-                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
-                           ->  Seq Scan on prt1_e_p1 t2
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t1_3
+                                 ->  Seq Scan on prt2_p0 t1_5
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
                      Index Cond: (a = ((t2.a + t2.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_1.a = t1_4.b)
+               Join Filter: (t1_1.a = t1_6.b)
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Join
-                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
-                           ->  Seq Scan on prt1_e_p2 t2_1
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_6.b)
+                           ->  Seq Scan on prt1_e_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t1_4
+                                 ->  Seq Scan on prt2_p1 t1_6
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
                      Index Cond: (a = ((t2_1.a + t2_1.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_2.a = t1_5.b)
+               Join Filter: (t1_2.a = t1_7.b)
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
+                           ->  Seq Scan on prt2_p2 t1_7
                                  Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
+                           ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t2_2
+                                 Index Cond: (((a + b) / 2) = t1_7.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t1_8.b)
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Nested Loop
+                           ->  Seq Scan on prt2_p3 t1_8
+                                 Filter: (a = 0)
+                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_3
+                                 Index Cond: (((a + b) / 2) = t1_8.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = ((t2_3.a + t2_3.b) / 2))
+                     Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t1_9.b)
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Join
+                           Hash Cond: (((t2_4.a + t2_4.b) / 2) = t1_9.b)
+                           ->  Seq Scan on prt1_e_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t1_9
+                                       Filter: (a = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = ((t2_4.a + t2_4.b) / 2))
+                     Filter: (b = 0)
+(66 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                               QUERY PLAN                                
--------------------------------------------------------------------------
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
-                           ->  Seq Scan on prt2_p1 t1_3
+                           Hash Cond: (t1_5.b = ((t1_10.a + t1_10.b) / 2))
+                           ->  Seq Scan on prt2_p0 t1_5
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t1_6
+                                 ->  Seq Scan on prt1_e_p0 t1_10
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
-                     Index Cond: (a = t1_3.b)
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t1_5.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
-                           ->  Seq Scan on prt2_p2 t1_4
+                           Hash Cond: (t1_6.b = ((t1_11.a + t1_11.b) / 2))
+                           ->  Seq Scan on prt2_p1 t1_6
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t1_7
+                                 ->  Seq Scan on prt1_e_p1 t1_11
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
-                     Index Cond: (a = t1_4.b)
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
+                     Index Cond: (a = t1_6.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
-                           ->  Seq Scan on prt2_p3 t1_5
+                           Hash Cond: (t1_7.b = ((t1_12.a + t1_12.b) / 2))
+                           ->  Seq Scan on prt2_p2 t1_7
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p3 t1_8
+                                 ->  Seq Scan on prt1_e_p2 t1_12
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t1_5.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t1_7.b)
                      Filter: (b = 0)
-(39 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_8.b = ((t1_13.a + t1_13.b) / 2))
+                           ->  Seq Scan on prt2_p3 t1_8
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p3 t1_13
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t1_8.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_9.b = ((t1_14.a + t1_14.b) / 2))
+                           ->  Seq Scan on prt2_p4 t1_9
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t1_14
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = t1_9.b)
+                     Filter: (b = 0)
+(63 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 -- test merge joins
 SET enable_hashjoin TO off;
 SET enable_nestloop TO off;
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                           QUERY PLAN                           
-----------------------------------------------------------------
+                            QUERY PLAN                            
+------------------------------------------------------------------
  Merge Append
    Sort Key: t1.a
    ->  Merge Semi Join
-         Merge Cond: (t1.a = t1_3.b)
+         Merge Cond: (t1.a = t1_5.b)
          ->  Sort
                Sort Key: t1.a
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+               Merge Cond: (t1_5.b = (((t1_10.a + t1_10.b) / 2)))
                ->  Sort
-                     Sort Key: t1_3.b
-                     ->  Seq Scan on prt2_p1 t1_3
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on prt2_p0 t1_5
                ->  Sort
-                     Sort Key: (((t1_6.a + t1_6.b) / 2))
-                     ->  Seq Scan on prt1_e_p1 t1_6
+                     Sort Key: (((t1_10.a + t1_10.b) / 2))
+                     ->  Seq Scan on prt1_e_p0 t1_10
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_1.a = t1_4.b)
+         Merge Cond: (t1_1.a = t1_6.b)
          ->  Sort
                Sort Key: t1_1.a
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+               Merge Cond: (t1_6.b = (((t1_11.a + t1_11.b) / 2)))
                ->  Sort
-                     Sort Key: t1_4.b
-                     ->  Seq Scan on prt2_p2 t1_4
+                     Sort Key: t1_6.b
+                     ->  Seq Scan on prt2_p1 t1_6
                ->  Sort
-                     Sort Key: (((t1_7.a + t1_7.b) / 2))
-                     ->  Seq Scan on prt1_e_p2 t1_7
+                     Sort Key: (((t1_11.a + t1_11.b) / 2))
+                     ->  Seq Scan on prt1_e_p1 t1_11
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_2.a = t1_5.b)
+         Merge Cond: (t1_2.a = t1_7.b)
          ->  Sort
                Sort Key: t1_2.a
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+               Merge Cond: (t1_7.b = (((t1_12.a + t1_12.b) / 2)))
                ->  Sort
-                     Sort Key: t1_5.b
-                     ->  Seq Scan on prt2_p3 t1_5
+                     Sort Key: t1_7.b
+                     ->  Seq Scan on prt2_p2 t1_7
                ->  Sort
-                     Sort Key: (((t1_8.a + t1_8.b) / 2))
-                     ->  Seq Scan on prt1_e_p3 t1_8
+                     Sort Key: (((t1_12.a + t1_12.b) / 2))
+                     ->  Seq Scan on prt1_e_p2 t1_12
                            Filter: (c = 0)
-(47 rows)
+   ->  Merge Semi Join
+         Merge Cond: (t1_3.a = t1_8.b)
+         ->  Sort
+               Sort Key: t1_3.a
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_8.b = (((t1_13.a + t1_13.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_8.b
+                     ->  Seq Scan on prt2_p3 t1_8
+               ->  Sort
+                     Sort Key: (((t1_13.a + t1_13.b) / 2))
+                     ->  Seq Scan on prt1_e_p3 t1_13
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_4.a = t1_9.b)
+         ->  Sort
+               Sort Key: t1_4.a
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_9.b = (((t1_14.a + t1_14.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_9.b
+                     ->  Seq Scan on prt2_p4 t1_9
+               ->  Sort
+                     Sort Key: (((t1_14.a + t1_14.b) / 2))
+                     ->  Seq Scan on prt1_e_p4 t1_14
+                           Filter: (c = 0)
+(77 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
@@ -933,14 +1438,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
                            ->  Sort
                                  Sort Key: (((t3.a + t3.b) / 2))
-                                 ->  Seq Scan on prt1_e_p1 t3
+                                 ->  Seq Scan on prt1_e_p0 t3
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1.a
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                ->  Sort
                      Sort Key: t2.b
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Merge Left Join
                Merge Cond: (t1_1.a = t2_1.b)
                ->  Sort
@@ -949,14 +1454,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
                            ->  Sort
                                  Sort Key: (((t3_1.a + t3_1.b) / 2))
-                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                 ->  Seq Scan on prt1_e_p1 t3_1
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_1.a
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                ->  Sort
                      Sort Key: t2_1.b
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Merge Left Join
                Merge Cond: (t1_2.a = t2_2.b)
                ->  Sort
@@ -965,32 +1470,74 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
                            ->  Sort
                                  Sort Key: (((t3_2.a + t3_2.b) / 2))
-                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                 ->  Seq Scan on prt1_e_p2 t3_2
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_2.a
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                ->  Sort
                      Sort Key: t2_2.b
-                     ->  Seq Scan on prt2_p3 t2_2
-(51 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Merge Left Join
+               Merge Cond: (t1_3.a = t2_3.b)
+               ->  Sort
+                     Sort Key: t1_3.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_3.a + t3_3.b) / 2)) = t1_3.a)
+                           ->  Sort
+                                 Sort Key: (((t3_3.a + t3_3.b) / 2))
+                                 ->  Seq Scan on prt1_e_p3 t3_3
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_3.a
+                                 ->  Seq Scan on prt1_p3 t1_3
+               ->  Sort
+                     Sort Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Merge Left Join
+               Merge Cond: (t1_4.a = t2_4.b)
+               ->  Sort
+                     Sort Key: t1_4.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_4.a + t3_4.b) / 2)) = t1_4.a)
+                           ->  Sort
+                                 Sort Key: (((t3_4.a + t3_4.b) / 2))
+                                 ->  Seq Scan on prt1_e_p4 t3_4
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_4.a
+                                 ->  Seq Scan on prt1_p4 t1_4
+               ->  Sort
+                     Sort Key: t2_4.b
+                     ->  Seq Scan on prt2_p4 t2_4
+(83 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- MergeAppend on nullable column
 EXPLAIN (COSTS OFF)
@@ -998,8 +1545,18 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Merge Left Join
+               Merge Cond: (prt1_p0.a = b)
+               ->  Sort
+                     Sort Key: prt1_p0.a
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
+               ->  Sort
+                     Sort Key: b
+                     ->  Result
+                           One-Time Filter: false
          ->  Merge Left Join
                Merge Cond: (prt1_p1.a = b)
                ->  Sort
@@ -1020,21 +1577,26 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                      Sort Key: prt2_p2.b
                      ->  Seq Scan on prt2_p2
                            Filter: (b > 250)
-(23 rows)
+(33 rows)
 
 SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  b  
------+-----
-   0 |    
-  50 |    
- 100 |    
- 150 |    
- 200 |    
- 250 |    
- 300 | 300
- 350 |    
- 400 |    
-(9 rows)
+  a   |  b  
+------+-----
+ -250 |    
+ -200 |    
+ -150 |    
+ -100 |    
+  -50 |    
+    0 |    
+   50 |    
+  100 |    
+  150 |    
+  200 |    
+  250 |    
+  300 | 300
+  350 |    
+  400 |    
+(14 rows)
 
 -- merge join when expression with whole-row reference needs to be sorted;
 -- partitionwise join does not apply
@@ -1048,30 +1610,797 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
          Sort Key: t1.a, ((((t1.*)::prt1))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
    ->  Sort
          Sort Key: t2.b, ((((t2.*)::prt2))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt2_p1 t2
-                     ->  Seq Scan on prt2_p2 t2_1
-                     ->  Seq Scan on prt2_p3 t2_2
-(16 rows)
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+(20 rows)
 
 SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.b ORDER BY t1.a;
- a  | b  
-----+----
-  0 |  0
-  6 |  6
- 12 | 12
- 18 | 18
- 24 | 24
-(5 rows)
+  a  |  b  
+-----+-----
+ -24 | -24
+ -18 | -18
+ -12 | -12
+  -6 |  -6
+   0 |   0
+   6 |   6
+  12 |  12
+  18 |  18
+  24 |  24
+(9 rows)
 
 RESET enable_hashjoin;
 RESET enable_nestloop;
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p4 t2_3
+               ->  Seq Scan on prt2_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_4
+                           Filter: (b = 0)
+(22 rows)
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p3 t1_2
+                           Filter: (b = 0)
+(21 rows)
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -150 | -0150 | 0 | -0150
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+(7 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 |   | 
+ -200 | -0200 |   | 
+ -150 | -0150 | 0 | -0150
+ -100 | -0100 |   | 
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+(21 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -200 | 0 | -0200
+ -100 | 0 | -0100
+  -50 | 0 | -0050
+   50 | 0 | 0050
+  100 | 0 | 0100
+  200 | 0 | 0200
+  250 | 0 | 0250
+  350 | 0 | 0350
+  400 | 0 | 0400
+  500 | 0 | 0500
+  550 | 0 | 0550
+  650 | 0 | 0650
+  700 | 0 | 0700
+(14 rows)
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+                             QUERY PLAN                              
+---------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t3.c
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p0_a on prt1_p0 t3
+                           Index Cond: (a = t2.b)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t2.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_1.a = t2_1.b)
+                           ->  Seq Scan on prt1_p1 t3_1
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1 t2_1
+                                       Filter: (a = 0)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p2_a on prt1_p2 t3_2
+                           Index Cond: (a = t2_2.b)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t3_3
+                           Index Cond: (a = t2_3.b)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_4.a = t2_4.b)
+                           ->  Seq Scan on prt1_p4 t3_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t2_4
+                                       Filter: (a = 0)
+(47 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+  a   | a |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.b
+   ->  Hash Right Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p5 t2_5
+                           Filter: (a = 0)
+(24 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: (t1.a = t2.b)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.a, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
+                     Filter: (a = 0)
+(22 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
 --
 -- partitioned by multiple columns
 --
@@ -1141,82 +2470,1552 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 -- test partition matching with N-way join
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                   QUERY PLAN                                   
---------------------------------------------------------------------------------
+                                           QUERY PLAN                                           
+------------------------------------------------------------------------------------------------
  GroupAggregate
    Group Key: t1.c, t2.c, t3.c
    ->  Sort
          Sort Key: t1.c, t3.c
          ->  Append
                ->  Hash Join
-                     Hash Cond: (t1.c = ltrim(t3.c, 'A'::text))
+                     Hash Cond: ((t1.c)::text = ltrim(t3.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1.b = t2.b) AND (t1.c = t2.c))
-                           ->  Seq Scan on plt1_p1 t1
+                           Hash Cond: ((t2.b = t1.b) AND ((t2.c)::text = (t1.c)::text))
+                           ->  Seq Scan on plt2_p4 t2
                            ->  Hash
-                                 ->  Seq Scan on plt2_p1 t2
+                                 ->  Seq Scan on plt1_p4 t1
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p1 t3
+                           ->  Seq Scan on plt1_e_p4 t3
                ->  Hash Join
-                     Hash Cond: (t1_1.c = ltrim(t3_1.c, 'A'::text))
+                     Hash Cond: ((t1_1.c)::text = ltrim(t3_1.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
-                           ->  Seq Scan on plt1_p2 t1_1
+                           Hash Cond: ((t1_1.b = t2_1.b) AND ((t1_1.c)::text = (t2_1.c)::text))
+                           ->  Seq Scan on plt1_p1 t1_1
                            ->  Hash
-                                 ->  Seq Scan on plt2_p2 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p2 t3_1
+                           ->  Seq Scan on plt1_e_p1 t3_1
                ->  Hash Join
-                     Hash Cond: (t1_2.c = ltrim(t3_2.c, 'A'::text))
+                     Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
-                           ->  Seq Scan on plt1_p3 t1_2
+                           Hash Cond: ((t1_2.b = t2_2.b) AND ((t1_2.c)::text = (t2_2.c)::text))
+                           ->  Seq Scan on plt1_p2 t1_2
                            ->  Hash
-                                 ->  Seq Scan on plt2_p3 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p3 t3_2
-(32 rows)
+                           ->  Seq Scan on plt1_e_p2 t3_2
+               ->  Hash Join
+                     Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2_3.b = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text))
+                           ->  Seq Scan on plt2_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p3 t3_3
+(41 rows)
 
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  75.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 123.0000000000000000 | 123.0000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 225.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 273.0000000000000000 | 273.0000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 375.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 423.0000000000000000 | 423.0000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 525.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 573.0000000000000000 | 573.0000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p5 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_3
+                                 ->  Seq Scan on plt2_p3 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Seq Scan on plt2_p2 t2_2
+                                 ->  Seq Scan on plt2_p3 t2_3
+(22 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1241,22 +4040,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
 --------------------------------------------------
  Hash Left Join
    Hash Cond: (t2.b = a)
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t3.a = t2.b)
-               ->  Seq Scan on prt1_p1 t3
-               ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
-         ->  Hash Join
-               Hash Cond: (t3_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t3_1
-               ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
-         ->  Hash Join
-               Hash Cond: (t3_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t3_2
-               ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
+   ->  Hash Join
+         Hash Cond: (t3.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t3
+               ->  Seq Scan on prt1_p1 t3_1
+               ->  Seq Scan on prt1_p2 t3_2
+               ->  Seq Scan on prt1_p3 t3_3
+               ->  Seq Scan on prt1_p4 t3_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
    ->  Hash
          ->  Result
                One-Time Filter: false
@@ -1271,16 +4070,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 --
 -- tests for hash partitioned tables.
@@ -1356,41 +4161,9 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  273.0000000000000000 | 273.0000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
 (6 rows)
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
- Sort
-   Sort Key: t1.a
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
-               ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
-               ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
-                           Filter: (b = 0)
-(21 rows)
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
@@ -1405,26 +4178,24 @@ SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c
    Sort Key: t1.c
    ->  HashAggregate
          Group Key: t1.c, t2.c
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t2.c = t1.c)
-                     ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p1 t1
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on plt1_p4 t1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_1.c = t1_1.c)
-                     ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_p2 t1_1
+                           ->  Seq Scan on plt1_p1 t1_1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_2.c = t1_2.c)
-                     ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Seq Scan on plt1_p2 t1_2
                                  Filter: ((a % 25) = 0)
-(23 rows)
+                           ->  Seq Scan on plt1_p3 t1_3
+                                 Filter: ((a % 25) = 0)
+(21 rows)
 
 --
 -- multiple levels of partitioning
@@ -1826,64 +4597,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
  Hash Join
    Hash Cond: (t1.a = t2.a)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt4_n_p1 t2
                ->  Seq Scan on prt4_n_p2 t2_1
                ->  Seq Scan on prt4_n_p3 t2_2
-(11 rows)
+(13 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a and t1.a = t3.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                        QUERY PLAN                        
+----------------------------------------------------------
  Hash Join
-   Hash Cond: (t2.a = t1.a)
+   Hash Cond: (t3.b = t1.a)
    ->  Append
-         ->  Seq Scan on prt4_n_p1 t2
-         ->  Seq Scan on prt4_n_p2 t2_1
-         ->  Seq Scan on prt4_n_p3 t2_2
+         ->  Seq Scan on prt2_p0 t3
+         ->  Seq Scan on prt2_p1 t3_1
+         ->  Seq Scan on prt2_p2 t3_2
+         ->  Seq Scan on prt2_p3 t3_3
+         ->  Seq Scan on prt2_p4 t3_4
+         ->  Seq Scan on prt2_p5 t3_5
    ->  Hash
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.a = t3_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_2.b)
-                     ->  Seq Scan on prt1_p3 t1_2
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3 t3_2
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt4_n_p1 t2
+                           ->  Seq Scan on prt4_n_p2 t2_1
+                           ->  Seq Scan on prt4_n_p3 t2_2
 (23 rows)
 
 -- partitionwise join can not be applied if there are no equi-join conditions
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (b > t1.a)
-(12 rows)
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+(16 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partitionwise join
@@ -1969,16 +4746,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
-                     ->  Seq Scan on plt1_p1 t3
-                     ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3
+                     ->  Seq Scan on plt1_p1 t3_1
+                     ->  Seq Scan on plt1_p2 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partitionwise join can not be applied for a join between list and range
 -- partitioned table
@@ -1989,14 +4767,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
  Hash Full Join
    Hash Cond: ((t2.c)::text = (t1.c)::text)
    ->  Append
-         ->  Seq Scan on prt1_p1 t2
-         ->  Seq Scan on prt1_p2 t2_1
-         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p0 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p2 t2_2
+         ->  Seq Scan on prt1_p3 t2_3
+         ->  Seq Scan on prt1_p4 t2_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt1_n_p1 t1
                ->  Seq Scan on prt1_n_p2 t1_1
-(10 rows)
+(12 rows)
 
 -- partitionwise join can not be applied if only one of joining table has
 -- default partition
@@ -2012,16 +4792,23 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
    ->  Hash Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index c1c9859651..0e884835fb 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -10,25 +10,39 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -67,11 +81,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -93,20 +115,30 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 
 EXPLAIN (COSTS OFF)
@@ -169,6 +201,128 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -193,28 +347,79 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
 -- test partition matching with N-way join
@@ -222,6 +427,175 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
@@ -267,27 +641,18 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
 ALTER TABLE plt2 DETACH PARTITION plt2_p3;
 ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
 ANALYZE plt2;
-
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.a % 25 = 0 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
 --
 -- multiple levels of partitioning
 --
-- 
2.18.0

#64Rajkumar Raghuwanshi
rajkumar.raghuwanshi@enterprisedb.com
In reply to: amul sul (#63)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Tue, Mar 5, 2019 at 3:45 PM amul sul <sulamul@gmail.com> wrote:

Attached is the rebased atop of the latest master head(35bc0ec7c8).

thanks Amul, patches applied cleanly on PG head.

While testing this I got a server crash with below test case.

CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
INSERT INTO plt1 SELECT i, i % 47, to_char(i % 17, 'FM0000') FROM
generate_series(0, 500) i WHERE i % 17 NOT IN (7, 11, 12, 13, 14, 15,16);
INSERT INTO plt1 SELECT i, i % 47, case when i % 17 = 7 then NULL else
to_char(i % 17, 'FM0000') end FROM generate_series(0, 500) i WHERE i % 17
IN (7,8,9);
ANALYSE plt1;

CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
INSERT INTO plt2 SELECT i, i % 47, to_char(i % 17, 'FM0000') FROM
generate_series(0, 500) i WHERE i % 17 NOT IN (1, 10, 11, 13, 14, 15, 16);
INSERT INTO plt2 SELECT i, i % 47, case when i % 17 = 11 then NULL else
to_char(i % 17, 'FM0000') end FROM generate_series(0, 500) i WHERE i % 17
IN (0,11,12);
ANALYZE plt2;

CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005',
'0006');
CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
INSERT INTO plt1_e SELECT i, i % 47, to_char(i % 17, 'FM0000') FROM
generate_series(0, 500) i WHERE i % 17 NOT IN (1, 7, 10, 11, 12, 13, 14,
15, 16);
ANALYZE plt1_e;

EXPLAIN (COSTS OFF)
SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM
plt1 t1, plt2 t2, plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c
GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
The connection to the server was lost. Attempting reset: Failed.

below is stack trace, looks like some indexes got corrupted, please take a
look.

Core was generated by `postgres: edb postgres [local]
EXPLAIN '.
Program terminated with signal 11, Segmentation fault.
#0 0x0000000000821aee in map_and_merge_partitions (partmaps1=0x2c1c8a8,
partmaps2=0x2c1c8e0, index1=45540240, index2=0, next_index=0x7ffeebd43d3c)
at partbounds.c:4162
4162 if (partmap1->from < 0 && partmap2->from < 0)
Missing separate debuginfos, use: debuginfo-install
keyutils-libs-1.4-5.el6.x86_64 krb5-libs-1.10.3-65.el6.x86_64
libcom_err-1.41.12-24.el6.x86_64 libselinux-2.0.94-7.el6.x86_64
openssl-1.0.1e-57.el6.x86_64 zlib-1.2.3-29.el6.x86_64
(gdb) bt
#0 0x0000000000821aee in map_and_merge_partitions (partmaps1=0x2c1c8a8,
partmaps2=0x2c1c8e0, *index1=45540240*, index2=0,
next_index=0x7ffeebd43d3c) at partbounds.c:4162
#1 0x00000000008226c3 in merge_null_partitions (outer_bi=0x2b6e338,
inner_bi=0x2bf90b0, outer_maps=0x2c1c8a8, inner_maps=0x2c1c8e0,
jointype=JOIN_INNER,
next_index=0x7ffeebd43d3c, null_index=0x7ffeebd43d38,
default_index=0x7ffeebd43d34) at partbounds.c:4610
#2 0x0000000000821726 in partition_list_bounds_merge
(partsupfunc=0x2ba3548, partcollation=0x2ba34e8, outer_rel=0x2b6ce40,
inner_rel=0x2bf8d28, outer_parts=0x7ffeebd43ed8,
inner_parts=0x7ffeebd43ed0, jointype=JOIN_INNER) at partbounds.c:4031
#3 0x000000000081ff5d in partition_bounds_merge (partnatts=1,
partsupfunc=0x2ba3548, partcollation=0x2ba34e8, outer_rel=0x2b6ce40,
inner_rel=0x2bf8d28, jointype=JOIN_INNER,
outer_parts=0x7ffeebd43ed8, inner_parts=0x7ffeebd43ed0) at
partbounds.c:3053
#4 0x00000000007c610f in try_partitionwise_join (root=0x2be2a28,
rel1=0x2b6ce40, rel2=0x2bf8d28, joinrel=0x2c1b0f0,
parent_sjinfo=0x7ffeebd44010,
parent_restrictlist=0x2c1c070) at joinrels.c:1370
#5 0x00000000007c5521 in populate_joinrel_with_paths (root=0x2be2a28,
rel1=0x2b6ce40, rel2=0x2bf8d28, joinrel=0x2c1b0f0, sjinfo=0x7ffeebd44010,
restrictlist=0x2c1c070)
at joinrels.c:914
#6 0x00000000007c4f48 in make_join_rel (root=0x2be2a28, rel1=0x2b6ce40,
rel2=0x2bf8d28) at joinrels.c:748
#7 0x00000000007c4514 in make_rels_by_clause_joins (root=0x2be2a28,
old_rel=0x2b6ce40, other_rels=0x2bae4d8) at joinrels.c:294
#8 0x00000000007c41c8 in join_search_one_level (root=0x2be2a28, level=3)
at joinrels.c:116
#9 0x00000000007abe59 in standard_join_search (root=0x2be2a28,
levels_needed=3, initial_rels=0x2bae500) at allpaths.c:2716
#10 0x00000000007abdca in make_rel_from_joinlist (root=0x2be2a28,
joinlist=0x2bfbae8) at allpaths.c:2647
#11 0x00000000007a86b0 in make_one_rel (root=0x2be2a28, joinlist=0x2bfbae8)
at allpaths.c:227
#12 0x00000000007dada1 in query_planner (root=0x2be2a28, tlist=0x2ba01c8,
qp_callback=0x7e0b25 <standard_qp_callback>, qp_extra=0x7ffeebd44390) at
planmain.c:265
#13 0x00000000007ddf83 in grouping_planner (root=0x2be2a28,
inheritance_update=false, tuple_fraction=0) at planner.c:1929
#14 0x00000000007dc5f5 in subquery_planner (glob=0x2be2990,
parse=0x2c0e8c8, parent_root=0x0, hasRecursion=false, tuple_fraction=0) at
planner.c:997
#15 0x00000000007db1b6 in standard_planner (parse=0x2c0e8c8,
cursorOptions=256, boundParams=0x0) at planner.c:416
#16 0x00000000007daef7 in planner (parse=0x2c0e8c8, cursorOptions=256,
boundParams=0x0) at planner.c:276
#17 0x00000000008e15c5 in pg_plan_query (querytree=0x2c0e8c8,
cursorOptions=256, boundParams=0x0) at postgres.c:878
#18 0x00000000006562cc in ExplainOneQuery (query=0x2c0e8c8,
cursorOptions=256, into=0x0, es=0x2c0e0a0,
queryString=0x2aa24d8 "EXPLAIN (COSTS OFF)\nSELECT avg(t1.a),
avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM\nplt1 t1, plt2 t2,
plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c,
t2.c, t3.c ORDER BY t1"..., params=0x0, queryEnv=0x0) at explain.c:364

Thanks & Regards,
Rajkumar Raghuwanshi
QMG, EnterpriseDB.

Show quoted text

Regards,
Amul Sul

On Mon, Feb 4, 2019 at 11:05 AM amul sul <sulamul@gmail.com> wrote:

There are few whitespaces in 0002 patch that I have fixed in the attached
version.
Rest of the patches are untouched.

Ill continue my review and testing.

Regards,
Amul

On Thu, Jan 31, 2019 at 5:26 PM Etsuro Fujita <
fujita.etsuro@lab.ntt.co.jp> wrote:

(2019/01/22 21:38), Etsuro Fujita wrote:

Will continue to review.

I rebased the patch set against the latest HEAD. Attached is a new
version. I'll move this to the next CF, and continue to review it.

Best regards,
Etsuro Fujita

#65amul sul
sulamul@gmail.com
In reply to: Rajkumar Raghuwanshi (#64)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Thanks Rajkumar,

I am looking into this.

Regards,
Amul

On Thu, Mar 7, 2019 at 11:54 AM Rajkumar Raghuwanshi <
rajkumar.raghuwanshi@enterprisedb.com> wrote:

Show quoted text

On Tue, Mar 5, 2019 at 3:45 PM amul sul <sulamul@gmail.com> wrote:

Attached is the rebased atop of the latest master head(35bc0ec7c8).

thanks Amul, patches applied cleanly on PG head.

While testing this I got a server crash with below test case.

CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN
('0001','0002','0003');
CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN
('0004','0005','0006');
CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
INSERT INTO plt1 SELECT i, i % 47, to_char(i % 17, 'FM0000') FROM
generate_series(0, 500) i WHERE i % 17 NOT IN (7, 11, 12, 13, 14, 15,16);
INSERT INTO plt1 SELECT i, i % 47, case when i % 17 = 7 then NULL else
to_char(i % 17, 'FM0000') end FROM generate_series(0, 500) i WHERE i % 17
IN (7,8,9);
ANALYSE plt1;

CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN
('0004','0005','0006');
CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN
('0007','0008','0009');
CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
INSERT INTO plt2 SELECT i, i % 47, to_char(i % 17, 'FM0000') FROM
generate_series(0, 500) i WHERE i % 17 NOT IN (1, 10, 11, 13, 14, 15, 16);
INSERT INTO plt2 SELECT i, i % 47, case when i % 17 = 11 then NULL else
to_char(i % 17, 'FM0000') end FROM generate_series(0, 500) i WHERE i % 17
IN (0,11,12);
ANALYZE plt2;

CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c,
'A'));
CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005',
'0006');
CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
INSERT INTO plt1_e SELECT i, i % 47, to_char(i % 17, 'FM0000') FROM
generate_series(0, 500) i WHERE i % 17 NOT IN (1, 7, 10, 11, 12, 13, 14,
15, 16);
ANALYZE plt1_e;

EXPLAIN (COSTS OFF)
SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM
plt1 t1, plt2 t2, plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c
GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
The connection to the server was lost. Attempting reset: Failed.

below is stack trace, looks like some indexes got corrupted, please take
a look.

Core was generated by `postgres: edb postgres [local]
EXPLAIN '.
Program terminated with signal 11, Segmentation fault.
#0 0x0000000000821aee in map_and_merge_partitions (partmaps1=0x2c1c8a8,
partmaps2=0x2c1c8e0, index1=45540240, index2=0, next_index=0x7ffeebd43d3c)
at partbounds.c:4162
4162 if (partmap1->from < 0 && partmap2->from < 0)
Missing separate debuginfos, use: debuginfo-install
keyutils-libs-1.4-5.el6.x86_64 krb5-libs-1.10.3-65.el6.x86_64
libcom_err-1.41.12-24.el6.x86_64 libselinux-2.0.94-7.el6.x86_64
openssl-1.0.1e-57.el6.x86_64 zlib-1.2.3-29.el6.x86_64
(gdb) bt
#0 0x0000000000821aee in map_and_merge_partitions (partmaps1=0x2c1c8a8,
partmaps2=0x2c1c8e0, *index1=45540240*, index2=0,
next_index=0x7ffeebd43d3c) at partbounds.c:4162
#1 0x00000000008226c3 in merge_null_partitions (outer_bi=0x2b6e338,
inner_bi=0x2bf90b0, outer_maps=0x2c1c8a8, inner_maps=0x2c1c8e0,
jointype=JOIN_INNER,
next_index=0x7ffeebd43d3c, null_index=0x7ffeebd43d38,
default_index=0x7ffeebd43d34) at partbounds.c:4610
#2 0x0000000000821726 in partition_list_bounds_merge
(partsupfunc=0x2ba3548, partcollation=0x2ba34e8, outer_rel=0x2b6ce40,
inner_rel=0x2bf8d28, outer_parts=0x7ffeebd43ed8,
inner_parts=0x7ffeebd43ed0, jointype=JOIN_INNER) at partbounds.c:4031
#3 0x000000000081ff5d in partition_bounds_merge (partnatts=1,
partsupfunc=0x2ba3548, partcollation=0x2ba34e8, outer_rel=0x2b6ce40,
inner_rel=0x2bf8d28, jointype=JOIN_INNER,
outer_parts=0x7ffeebd43ed8, inner_parts=0x7ffeebd43ed0) at
partbounds.c:3053
#4 0x00000000007c610f in try_partitionwise_join (root=0x2be2a28,
rel1=0x2b6ce40, rel2=0x2bf8d28, joinrel=0x2c1b0f0,
parent_sjinfo=0x7ffeebd44010,
parent_restrictlist=0x2c1c070) at joinrels.c:1370
#5 0x00000000007c5521 in populate_joinrel_with_paths (root=0x2be2a28,
rel1=0x2b6ce40, rel2=0x2bf8d28, joinrel=0x2c1b0f0, sjinfo=0x7ffeebd44010,
restrictlist=0x2c1c070)
at joinrels.c:914
#6 0x00000000007c4f48 in make_join_rel (root=0x2be2a28, rel1=0x2b6ce40,
rel2=0x2bf8d28) at joinrels.c:748
#7 0x00000000007c4514 in make_rels_by_clause_joins (root=0x2be2a28,
old_rel=0x2b6ce40, other_rels=0x2bae4d8) at joinrels.c:294
#8 0x00000000007c41c8 in join_search_one_level (root=0x2be2a28, level=3)
at joinrels.c:116
#9 0x00000000007abe59 in standard_join_search (root=0x2be2a28,
levels_needed=3, initial_rels=0x2bae500) at allpaths.c:2716
#10 0x00000000007abdca in make_rel_from_joinlist (root=0x2be2a28,
joinlist=0x2bfbae8) at allpaths.c:2647
#11 0x00000000007a86b0 in make_one_rel (root=0x2be2a28,
joinlist=0x2bfbae8) at allpaths.c:227
#12 0x00000000007dada1 in query_planner (root=0x2be2a28, tlist=0x2ba01c8,
qp_callback=0x7e0b25 <standard_qp_callback>, qp_extra=0x7ffeebd44390) at
planmain.c:265
#13 0x00000000007ddf83 in grouping_planner (root=0x2be2a28,
inheritance_update=false, tuple_fraction=0) at planner.c:1929
#14 0x00000000007dc5f5 in subquery_planner (glob=0x2be2990,
parse=0x2c0e8c8, parent_root=0x0, hasRecursion=false, tuple_fraction=0) at
planner.c:997
#15 0x00000000007db1b6 in standard_planner (parse=0x2c0e8c8,
cursorOptions=256, boundParams=0x0) at planner.c:416
#16 0x00000000007daef7 in planner (parse=0x2c0e8c8, cursorOptions=256,
boundParams=0x0) at planner.c:276
#17 0x00000000008e15c5 in pg_plan_query (querytree=0x2c0e8c8,
cursorOptions=256, boundParams=0x0) at postgres.c:878
#18 0x00000000006562cc in ExplainOneQuery (query=0x2c0e8c8,
cursorOptions=256, into=0x0, es=0x2c0e0a0,
queryString=0x2aa24d8 "EXPLAIN (COSTS OFF)\nSELECT avg(t1.a),
avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM\nplt1 t1, plt2 t2,
plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c,
t2.c, t3.c ORDER BY t1"..., params=0x0, queryEnv=0x0) at explain.c:364

Thanks & Regards,
Rajkumar Raghuwanshi
QMG, EnterpriseDB.

Regards,
Amul Sul

On Mon, Feb 4, 2019 at 11:05 AM amul sul <sulamul@gmail.com> wrote:

There are few whitespaces in 0002 patch that I have fixed in the
attached version.
Rest of the patches are untouched.

Ill continue my review and testing.

Regards,
Amul

On Thu, Jan 31, 2019 at 5:26 PM Etsuro Fujita <
fujita.etsuro@lab.ntt.co.jp> wrote:

(2019/01/22 21:38), Etsuro Fujita wrote:

Will continue to review.

I rebased the patch set against the latest HEAD. Attached is a new
version. I'll move this to the next CF, and continue to review it.

Best regards,
Etsuro Fujita

#66amul sul
sulamul@gmail.com
In reply to: amul sul (#65)
3 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Thu, Mar 7, 2019 at 1:02 PM amul sul <sulamul@gmail.com> wrote:

Thanks Rajkumar,

I am looking into this.

The crash happens when none of the if-else branch of
handle_missing_partition()
evaluates and returns merged_index unassigned.

Let me explain, in Rajkumar 's test case, the join type is JOIN_INNER. When
only outer rel has null partition, merge_null_partitions() function calls
handle_missing_partition() with missing_side_inner = false and
missing_side_outer = false argument value which fails to set merged_index.

In the attached patch, I tried to fix this case by setting merged_index
explicitly which fixes the reported crash.

Regards,
Amul

Show quoted text

On Thu, Mar 7, 2019 at 11:54 AM Rajkumar Raghuwanshi <
rajkumar.raghuwanshi@enterprisedb.com> wrote:

On Tue, Mar 5, 2019 at 3:45 PM amul sul <sulamul@gmail.com> wrote:

Attached is the rebased atop of the latest master head(35bc0ec7c8).

thanks Amul, patches applied cleanly on PG head.

While testing this I got a server crash with below test case.

CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN
('0001','0002','0003');
CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN
('0004','0005','0006');
CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
INSERT INTO plt1 SELECT i, i % 47, to_char(i % 17, 'FM0000') FROM
generate_series(0, 500) i WHERE i % 17 NOT IN (7, 11, 12, 13, 14, 15,16);
INSERT INTO plt1 SELECT i, i % 47, case when i % 17 = 7 then NULL else
to_char(i % 17, 'FM0000') end FROM generate_series(0, 500) i WHERE i % 17
IN (7,8,9);
ANALYSE plt1;

CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN
('0004','0005','0006');
CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN
('0007','0008','0009');
CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
INSERT INTO plt2 SELECT i, i % 47, to_char(i % 17, 'FM0000') FROM
generate_series(0, 500) i WHERE i % 17 NOT IN (1, 10, 11, 13, 14, 15, 16);
INSERT INTO plt2 SELECT i, i % 47, case when i % 17 = 11 then NULL else
to_char(i % 17, 'FM0000') end FROM generate_series(0, 500) i WHERE i % 17
IN (0,11,12);
ANALYZE plt2;

CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c,
'A'));
CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005',
'0006');
CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
INSERT INTO plt1_e SELECT i, i % 47, to_char(i % 17, 'FM0000') FROM
generate_series(0, 500) i WHERE i % 17 NOT IN (1, 7, 10, 11, 12, 13, 14,
15, 16);
ANALYZE plt1_e;

EXPLAIN (COSTS OFF)
SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM
plt1 t1, plt2 t2, plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c
GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
The connection to the server was lost. Attempting reset: Failed.

below is stack trace, looks like some indexes got corrupted, please take
a look.

Core was generated by `postgres: edb postgres [local]
EXPLAIN '.
Program terminated with signal 11, Segmentation fault.
#0 0x0000000000821aee in map_and_merge_partitions (partmaps1=0x2c1c8a8,
partmaps2=0x2c1c8e0, index1=45540240, index2=0, next_index=0x7ffeebd43d3c)
at partbounds.c:4162
4162 if (partmap1->from < 0 && partmap2->from < 0)
Missing separate debuginfos, use: debuginfo-install
keyutils-libs-1.4-5.el6.x86_64 krb5-libs-1.10.3-65.el6.x86_64
libcom_err-1.41.12-24.el6.x86_64 libselinux-2.0.94-7.el6.x86_64
openssl-1.0.1e-57.el6.x86_64 zlib-1.2.3-29.el6.x86_64
(gdb) bt
#0 0x0000000000821aee in map_and_merge_partitions (partmaps1=0x2c1c8a8,
partmaps2=0x2c1c8e0, *index1=45540240*, index2=0,
next_index=0x7ffeebd43d3c) at partbounds.c:4162
#1 0x00000000008226c3 in merge_null_partitions (outer_bi=0x2b6e338,
inner_bi=0x2bf90b0, outer_maps=0x2c1c8a8, inner_maps=0x2c1c8e0,
jointype=JOIN_INNER,
next_index=0x7ffeebd43d3c, null_index=0x7ffeebd43d38,
default_index=0x7ffeebd43d34) at partbounds.c:4610
#2 0x0000000000821726 in partition_list_bounds_merge
(partsupfunc=0x2ba3548, partcollation=0x2ba34e8, outer_rel=0x2b6ce40,
inner_rel=0x2bf8d28, outer_parts=0x7ffeebd43ed8,
inner_parts=0x7ffeebd43ed0, jointype=JOIN_INNER) at partbounds.c:4031
#3 0x000000000081ff5d in partition_bounds_merge (partnatts=1,
partsupfunc=0x2ba3548, partcollation=0x2ba34e8, outer_rel=0x2b6ce40,
inner_rel=0x2bf8d28, jointype=JOIN_INNER,
outer_parts=0x7ffeebd43ed8, inner_parts=0x7ffeebd43ed0) at
partbounds.c:3053
#4 0x00000000007c610f in try_partitionwise_join (root=0x2be2a28,
rel1=0x2b6ce40, rel2=0x2bf8d28, joinrel=0x2c1b0f0,
parent_sjinfo=0x7ffeebd44010,
parent_restrictlist=0x2c1c070) at joinrels.c:1370
#5 0x00000000007c5521 in populate_joinrel_with_paths (root=0x2be2a28,
rel1=0x2b6ce40, rel2=0x2bf8d28, joinrel=0x2c1b0f0, sjinfo=0x7ffeebd44010,
restrictlist=0x2c1c070)
at joinrels.c:914
#6 0x00000000007c4f48 in make_join_rel (root=0x2be2a28, rel1=0x2b6ce40,
rel2=0x2bf8d28) at joinrels.c:748
#7 0x00000000007c4514 in make_rels_by_clause_joins (root=0x2be2a28,
old_rel=0x2b6ce40, other_rels=0x2bae4d8) at joinrels.c:294
#8 0x00000000007c41c8 in join_search_one_level (root=0x2be2a28, level=3)
at joinrels.c:116
#9 0x00000000007abe59 in standard_join_search (root=0x2be2a28,
levels_needed=3, initial_rels=0x2bae500) at allpaths.c:2716
#10 0x00000000007abdca in make_rel_from_joinlist (root=0x2be2a28,
joinlist=0x2bfbae8) at allpaths.c:2647
#11 0x00000000007a86b0 in make_one_rel (root=0x2be2a28,
joinlist=0x2bfbae8) at allpaths.c:227
#12 0x00000000007dada1 in query_planner (root=0x2be2a28, tlist=0x2ba01c8,
qp_callback=0x7e0b25 <standard_qp_callback>, qp_extra=0x7ffeebd44390) at
planmain.c:265
#13 0x00000000007ddf83 in grouping_planner (root=0x2be2a28,
inheritance_update=false, tuple_fraction=0) at planner.c:1929
#14 0x00000000007dc5f5 in subquery_planner (glob=0x2be2990,
parse=0x2c0e8c8, parent_root=0x0, hasRecursion=false, tuple_fraction=0) at
planner.c:997
#15 0x00000000007db1b6 in standard_planner (parse=0x2c0e8c8,
cursorOptions=256, boundParams=0x0) at planner.c:416
#16 0x00000000007daef7 in planner (parse=0x2c0e8c8, cursorOptions=256,
boundParams=0x0) at planner.c:276
#17 0x00000000008e15c5 in pg_plan_query (querytree=0x2c0e8c8,
cursorOptions=256, boundParams=0x0) at postgres.c:878
#18 0x00000000006562cc in ExplainOneQuery (query=0x2c0e8c8,
cursorOptions=256, into=0x0, es=0x2c0e0a0,
queryString=0x2aa24d8 "EXPLAIN (COSTS OFF)\nSELECT avg(t1.a),
avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM\nplt1 t1, plt2 t2,
plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c,
t2.c, t3.c ORDER BY t1"..., params=0x0, queryEnv=0x0) at explain.c:364

Thanks & Regards,
Rajkumar Raghuwanshi
QMG, EnterpriseDB.

Regards,
Amul Sul

On Mon, Feb 4, 2019 at 11:05 AM amul sul <sulamul@gmail.com> wrote:

There are few whitespaces in 0002 patch that I have fixed in the
attached version.
Rest of the patches are untouched.

Ill continue my review and testing.

Regards,
Amul

On Thu, Jan 31, 2019 at 5:26 PM Etsuro Fujita <
fujita.etsuro@lab.ntt.co.jp> wrote:

(2019/01/22 21:38), Etsuro Fujita wrote:

Will continue to review.

I rebased the patch set against the latest HEAD. Attached is a new
version. I'll move this to the next CF, and continue to review it.

Best regards,
Etsuro Fujita

Attachments:

0002-Partition-wise-join-for-1-1-1-0-0-1-partition-matchi-v20.patchapplication/octet-stream; name=0002-Partition-wise-join-for-1-1-1-0-0-1-partition-matchi-v20.patchDownload
From 497bf4f3a78f3acb36a711253c03d9bf4516fd00 Mon Sep 17 00:00:00 2001
From: Amul Sul <amul.sul@enterprisedb.com>
Date: Mon, 4 Mar 2019 23:49:47 -0500
Subject: [PATCH 2/3] Partition-wise join for 1:1, 1:0, 0:1 partition matching.

Earlier version of partition-wise join implementation allowed
partition-wise join between two relations with exactly same partition
bounds. This commit allows partition-wise join to be applied under
following conditions

1. the partition bounds of joining relations are such that rows from
given partition on one side join can join with rows from maximum one
partition on the other side i.e. bounds of a given partition on one
side match/overlap with those of maximum one partition on the other
side. If the mapping happens to be m:n where m > 1 or n > 1, we have
to gang multiple partition relations together into a single relation.
This means that we have to add simple relations during join
processing, something which is not supported right now.  ALso, in such
a case, different pairs of joining relations can produce different
partition bounds for the same join relation, which again is not
supported right now.

2. For every partition on outer side that can contribute to the result
of an OUTER join, there exists at least one (taken along with item 1,
it means exactly one)  matching partition on the inner side. To
support partition-wise join when the inner matching partition doesn't
exist, we have to add a dummy base relation corresponding to the
non-existent inner partition. We don't have support to add base
relations during join processing.

3. For hash partitioned tables however, we support partition-wise join
only when the bounds exactly match. For hash partitioning it's unusual
to have missing partitions and hence generic partition matching is not
required.

With advanced partition matching, we won't be able to apply
partition-wise join when there are missing matching partitions. So for
a given N-way join, some sub-joins may not be partitioned, while the
overall N-way join is partitioned. We can not try partition-wise join
when one or both of the joining relations are not partitioned in one
of the pairs of joining relation. Skip partition-wise join for that
pair.

An example is A IJ B LJ C where B has an extra partition compared to A
and C. Join B LJ C requires dummy relation to join the extra partition
from B to a missing partition from C, and hence can not use
partition-wise join right now and hence BC is not partitioned.  The
extra partition in B gets eliminated in A IJ B and thus it can use
partition-wise join and is partitioned. Hence (AB)C can use
partition-wise join but not A(BC).

Not every pair of joining relation (including the one presented to
build_joinrel_partition_info()) for the same joinrel can use
partition-wise join or has both the relations partitioned. Hence we
calculate the partition bounds for the join relation in
try_partition_wise_join() instead of doing it here.

The partition bounds produced for the first pair that can use
partition-wise are saved in the RelOptInfo of the joinrel. Partition
bounds produced for subsequent pairs should match that produced by the
first pair.

Support for default partition in list partitioned tables
========================================================

When a list value is present in one of the joining relations and not
the other, and the other relation has default partition, match (join)
the partition containing that list value with the default partition.
If there are multiple matches, we can not proceed with the
partition-wise join similar to the case of matching non-default
partition. If the default partition happens to be on the outer side of
the join, the resulting join partition acts as a default partition as
it will contain all the values from the default partition. If
the partition containing the list value happens to be on the outer
side of the join, the resulting join partition is associated with the
list value, since no other partition key value from the default
partition makes it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join.  Such a
default partition may get joined to a partition from the inner side,
if inner side has a list value that is not present in the outer side.
But if such a join partner does not exist, we won't be able to use
partition-wise join since there is no (dummy) relation on the inner
side to join to.

Support for matching default partition in range partitioned tables
==================================================================

When a range is present in one of the joining relations and not the
other, and the other relation has default partition, match (join) the
partition corresponding range with the default partition. If two
ranges overlap but their ranges don't match exactly, the
non-overlapping portion from one range may join the previous
(non-overlapping portion near the lower bound, if exists), next
(non-overlapping portion near the upper bound, if exists) ranges from
the other relation or default partition from the other relation. This
causes multiple partitions on the other side to be joined with a
single partition on the first side; a case we don't support. Any range
from one side which doesn't overlap with any range on the other side,
may find its join partner in the default partition and the
corresponding range partition will need to be joined with the default
partition.  If the default partition happens to be on the outer side
of the join, the resulting join partition acts as a default partition
as it will contain all the values from the default partition. If the
non-partition corresponding to the range happens to be on the outer
side of the join, the resulting join partition is associated with that
range, since partition key values from the default partition outside
that range won't make it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join.  Such a
default partition may get joined to a non-default partition from the
inner side, if inner side has a range missing in the outer side.  But
if such a join partner does not exist, we won't be able to use
partition-wise join since there is no (dummy) relation on the inner
side to join to.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/optimizer/path/joinrels.c |   95 +-
 src/backend/optimizer/util/relnode.c  |   33 +-
 src/backend/partitioning/partbounds.c | 1682 ++++++++++++++++++++++++-
 src/include/partitioning/partbounds.h |    7 +
 4 files changed, 1771 insertions(+), 46 deletions(-)

diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index dfbbfdac6d..ca4d261bc1 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1320,25 +1320,31 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	PartitionScheme part_scheme;
+	PartitionBoundInfo join_boundinfo;
+	List	   *parts1;
+	List	   *parts2;
+	ListCell   *lc1;
+	ListCell   *lc2;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1351,32 +1357,63 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	 */
 	Assert(joinrel->part_scheme == rel1->part_scheme &&
 		   joinrel->part_scheme == rel2->part_scheme);
+	part_scheme = joinrel->part_scheme;
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * Get the list of matching partitions to be joined along with the
+	 * partition bounds of the join relation. Because of the restrictions
+	 * imposed by partition matching algorithm, not every pair of joining
+	 * relations for this join will be able to use partition-wise join. But all
+	 * those pairs which can use partition-wise join will produce the same
+	 * partition bounds for the join relation.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	join_boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											part_scheme->partsupfunc,
+											part_scheme->partcollation,
+											rel1, rel2,
+											parent_sjinfo->jointype,
+											&parts1, &parts2);
+
+	if (join_boundinfo == NULL)
+		return;
 
-	nparts = joinrel->nparts;
+	if (joinrel->boundinfo == NULL)
+	{
+		Assert(joinrel->nparts == 0 && joinrel->part_rels == NULL);
+		joinrel->boundinfo = join_boundinfo;
+		joinrel->nparts = list_length(parts1);
+		Assert(joinrel->nparts == list_length(parts2));
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) *
+									joinrel->nparts);
+	}
+	else
+	{
+		Assert(partition_bounds_equal(part_scheme->partnatts,
+									  part_scheme->parttyplen,
+									  part_scheme->parttypbyval,
+									  join_boundinfo, joinrel->boundinfo));
+		/*
+		 * Every pair of joining relations should result in the same number
+		 * of child-joins.
+		 */
+		Assert(joinrel->nparts == list_length(parts1));
+		Assert(joinrel->nparts == list_length(parts2));
+		Assert(joinrel->part_rels);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	cnt_parts = 0;
+	forboth(lc1, parts1, lc2, parts2)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
+		int			part1 = lfirst_int(lc1);
+		int			part2 = lfirst_int(lc2);
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1384,6 +1421,10 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		Assert(part1 >= 0 && part2 >= 0);
+		child_rel1 = rel1->part_rels[part1];
+		child_rel2 = rel2->part_rels[part2];
+
 		/*
 		 * If a child table has consider_partitionwise_join=false, it means
 		 * that it's a dummy relation for which we skipped setting up tlist
@@ -1438,12 +1479,24 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 			joinrel->part_rels[cnt_parts] = child_joinrel;
 		}
 
+		/*
+		 * For every pair of joining relations, the set of matching partitions
+		 * would change. However, the base relation partitions constituting
+		 * the given child should remain same for all the joining pairs. Since
+		 * the order in which children are stored in the array of child-joins,
+		 * depends upon partition bounds of the join, which are same for all
+		 * the joining pairs, every joining pair yields the child-joins in the
+		 * same order.
+		 */
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
 
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
+		cnt_parts++;
 	}
+
+	Assert(cnt_parts == joinrel->nparts);
 }
 
 /*
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 4130514952..6354213854 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1594,7 +1594,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1607,24 +1607,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1636,17 +1618,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 637e05e5ca..257d24f6e1 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -23,6 +23,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/pathnodes.h"
 #include "parser/parse_coerce.h"
 #include "partitioning/partbounds.h"
 #include "partitioning/partdesc.h"
@@ -67,6 +68,12 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+typedef struct PartitionMap
+{
+	int from;
+	int to;
+} PartitionMap;
+
 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);
@@ -108,7 +115,58 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 static List *get_range_nulltest(PartitionKey key);
 static bool partition_hbounds_equal(PartitionBoundInfo b1,
 									PartitionBoundInfo b2);
-
+static PartitionBoundInfo partition_range_bounds_merge(
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 List **outer_parts, List **inner_parts,
+							 JoinType jointype, int partnatts,
+							 FmgrInfo *supfuncs, Oid *collations);
+static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype);
+static PartitionBoundInfo partition_hash_bounds_merge(RelOptInfo *outer_rel,
+							RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype);
+static void generate_matching_part_pairs(PartitionMap *outer_maps,
+										 PartitionMap *inner_maps,
+										 int nparts1, int nparts2,
+										 JoinType jointype, int nparts,
+										 List **parts1, List **parts2);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int map_and_merge_partitions(PartitionMap *outer_maps,
+										PartitionMap *inner_maps,
+										int index1, int index2, int *next_index);
+static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *collations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2);
+static bool partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
+						   Oid *collations, PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, int *ub_cmpval,
+						   int *lb_cmpval);
+static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes);
+static bool merge_default_partitions(PartitionBoundInfo outer_bi,
+						 PartitionBoundInfo inner_bi,
+						 PartitionMap *outer_maps,
+						 PartitionMap *inner_maps,
+						 JoinType jointype,
+						 int *next_index, int *default_index);
+static bool merge_null_partitions(PartitionBoundInfo outer_bi,
+								  PartitionBoundInfo inner_bi,
+								  PartitionMap *outer_maps,
+								  PartitionMap *inner_maps,
+								  JoinType jointype,
+								  int *next_index, int *null_index,
+								  int *default_index);
 
 /*
  * get_qual_from_partbound
@@ -2943,3 +3001,1625 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * The function produces the partition bounds for a join between two relations
+ * whose partition bounds are given. The function also returns two lists of
+ * partition indexes one for each of the joining relations. Both the lists
+ * contain the same number of elements. The partition indexes at the same
+ * positions in the lists indicate the pair partitions, one from each side, to
+ * be joined and the position itself corresponds to the index of partition
+ * produced by that child-join in the partitioned join.
+ *
+ * The function returns NULL if we can not find the matching pair of
+ * partitions. This happens if 1. multiple partitions on one side match with
+ * one partition on the other side. 2. a given partition on the outer side
+ * doesn't have a matching partition on the inner side. We can not support the
+ * first case since we don't have a way to represent multiple partitions as a
+ * single relation (RelOptInfo) and then perform join using the ganged
+ * relation. We can not support the second case since the missing inner
+ * partition needs to be represented as an empty relation and we don't have a
+ * way to introduce empty relation during join planning after creating paths
+ * for all the base relations.
+ */
+extern PartitionBoundInfo
+partition_bounds_merge(int partnatts, FmgrInfo *partsupfunc,
+					   Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
+{
+	PartitionBoundInfo 	merged_bounds;
+	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
+						inner_binfo = inner_rel->boundinfo;
+	char				strategy = outer_binfo->strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partsupfunc,
+														partcollation,
+														outer_rel, inner_rel,
+														outer_parts, inner_parts,
+														jointype);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(outer_rel, inner_rel,
+														 outer_parts, inner_parts,
+														 jointype, partnatts,
+														 partsupfunc,
+														 partcollation);
+			break;
+
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = partition_hash_bounds_merge(outer_rel, inner_rel,
+														outer_parts, inner_parts,
+														jointype);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	Assert((*outer_parts == NIL || *inner_parts != NIL) &&
+		   (*inner_parts == NIL || *outer_parts != NIL));
+
+	return merged_bounds;
+}
+
+/*
+ * partition_get_range_bounds
+ *
+ * Given the index of lower bound in datums array, return lower and upper
+ * bounds and the index of the partition with that lower bound.
+ */
+static int
+partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
+						   PartitionRangeBound *lower,
+						   PartitionRangeBound *upper)
+{
+	int			part_index;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The lower bound should correspond to a valid partition. */
+	part_index = bi->indexes[lb_index + 1];
+	Assert(part_index >= 0);
+
+	lower->kind = bi->kind[lb_index];
+	lower->datums = bi->datums[lb_index];
+	lower->lower = true;
+	upper->kind = bi->kind[lb_index + 1];
+	upper->datums = bi->datums[lb_index + 1];
+	upper->lower = false;
+
+	return part_index;
+}
+
+/*
+ * partition_range_get_next_lb_index
+ *
+ * Given the index of lower bound in datums array return the
+ * index of lower bound of the next partition. When the given index corresponds
+ * to the last partition, return number of datums (ndatums).
+ */
+static int
+partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
+{
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The partition index corresponding to the upper bound should be valid. */
+	Assert(bi->indexes[lb_index + 1] >= 0);
+
+	/*
+	 * If there are no bounds left beyond the upper bound, we have reached the
+	 * last partition.
+	 */
+	if (lb_index + 2 < bi->ndatums)
+	{
+		/*
+		 * If the bound next to the upper bound corresponds to no partition,
+		 * that's the next lower bound of the next partition. Otherwise, the
+		 * current upper bound is the lower bound of the next partition.
+		 */
+		if (bi->indexes[lb_index + 2] < 0)
+			return lb_index + 2;
+		else
+			return lb_index + 1;
+	}
+	else
+		return bi->ndatums;
+}
+
+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *partcollations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2)
+{
+	return partition_rbound_cmp(partnatts, partsupfunc, partcollations,
+								bound1->datums, bound1->kind, bound1->lower,
+								bound2);
+}
+
+/*
+ * partition_range_cmp
+ *
+ * Compare the bounds of two range partitions. Set ub_cmpval <, = or > 0, if the
+ * first partition's upper bound is lower than, equal to or higher than the
+ * second partition's upper bound resp. Similarly set lb_cmpval <, =  or > 0,
+ * if the first partition's lower bound is lower than, equal to or higher than
+ * the second partition's lower bound resp.
+ *
+ * Return true, if the ranges overlap, otherwise return false.
+ */
+static bool
+partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
+					PartitionRangeBound *lower_bound1,
+					PartitionRangeBound *upper_bound1,
+					PartitionRangeBound *lower_bound2,
+					PartitionRangeBound *upper_bound2, int *ub_cmpval,
+					int *lb_cmpval)
+{
+	bool		overlap;
+
+	/*
+	 * Compare upper bound of the first partition with the lower bound of the
+	 * second and vice-versa. If lower bound is higher than the upper bound,
+	 * the partitions are not overlapping. All other cases indicate overlapping
+	 * partitions.
+	 */
+	if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+								  lower_bound1, upper_bound2) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = 1;
+		*lb_cmpval = 1;
+	}
+	else if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+									   lower_bound2, upper_bound1) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = -1;
+		*lb_cmpval = -1;
+	}
+	else
+	{
+		overlap = true;
+		*ub_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, upper_bound1,
+											   upper_bound2);
+		*lb_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, lower_bound1,
+											   lower_bound2);
+	}
+
+	return overlap;
+}
+
+/*
+ * partition_range_merge
+ *
+ * Merge the partition bounds of given two partitions such that the join
+ * between the given two partitions fits merged bounds.
+ *
+ * "merged_upper" will be set to one of the given upper bounds and
+ * "merged_lower" will be set to one of the given lower bounds.
+ */
+static void
+partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
+					  Oid *partcollations, JoinType jointype,
+					  PartitionRangeBound *left_lb,
+					  PartitionRangeBound *left_ub,
+					  PartitionRangeBound *right_lb,
+					  PartitionRangeBound *right_ub,
+					  PartitionRangeBound **merged_lb,
+					  PartitionRangeBound **merged_ub)
+{
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = left_ub;
+			*merged_lb = left_lb;
+			break;
+
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) < 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) > 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		case JOIN_FULL:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) > 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) < 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+
+	return;
+}
+
+/*
+ * Add the lower bound of the next range to the list of bounds, if the lower
+ * bound is higher or equal to the previous upper bound. If successful return
+ * true, otherwise false.
+ */
+static bool
+partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
+							  Oid *partcollations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		prev_ub.datums = llast(*merged_datums);
+		prev_ub.kind = llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  next_lb_datums, next_lb_kind, false,
+									  &prev_ub);
+	}
+
+	/*
+	 * The lower bound is lower than the last upper bound, thus does not fit
+	 * the bounds created so far and hence can not be merged with the existing
+	 * bounds.
+	 */
+	if (cmpval < 0)
+		return false;
+
+	/*
+	 * Add bounds of the new merged partition. If the next lower bound is
+	 * higher than the last upper bound, add new range with index
+	 * corresponding to the lower bound as -1. If the merged lower bound
+	 * is same as the last merged upper bound, the last upper bound will be
+	 * reused as the lower bound of the next range.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, next_lb_datums);
+		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	return true;
+}
+
+/*
+ * handle_missing_partition
+ *
+ * If a range appears in one of the joining relations but not the other, a row
+ * in the corresponding partition will not have any join partner in the other
+ * relation, unless the other relation has a default partition. If a given list
+ * value is present in one joining relation but not the other, the default
+ * partition on the other side may contain that value.
+ *
+ * In both these cases, such an extra partition forms a joining pair with the
+ * default partition, if any,  on the other side.
+ *
+ * If the default partition happens to be on the outer side of the join, the
+ * resultant partition will act as the default partition of the join relation.
+ * Otherwise the resultant partition will be associated with the range.
+ *
+ * When the default partition is not present in the other relation, the rows in
+ * the extra partition will be included in the bounds of the join result, if it
+ * appears on the outer side of the join, since all rows from the outer side
+ * are included in the join result.
+ *
+ * This function handles all these cases.
+ *
+ * maps_with_missing and missing_side_default are the partition maps (See
+ * partition_range/list_bounds_merge() for details) and the index of default
+ * partition respectively corresponding the side with missing partition.
+ *
+ * maps_with_extra and extra_part are the partition maps (See
+ * partition_range/list_bounds_merge() for details) and the index of extra
+ * partition respectively corresponding to the side with the extra partition.
+ *
+ * It returns true if the matching succeeds, otherwise returns false.
+ */
+static bool
+handle_missing_partition(PartitionMap *maps_with_missing,
+						 PartitionMap *maps_with_extra,
+						 int missing_side_default,
+						 int extra_part,
+						 bool missing_side_outer,
+						 bool missing_side_inner,
+						 int *next_index, int *default_index,
+						 int *merged_index)
+{
+	bool missing_has_default = (missing_side_default != -1);
+
+	if (missing_has_default)
+	{
+		*merged_index = map_and_merge_partitions(maps_with_missing,
+												 maps_with_extra,
+												 missing_side_default,
+												 extra_part,
+												 next_index);
+		if (*merged_index < 0)
+			return false;
+
+		if (missing_side_outer)
+		{
+			/*
+			 * Default partition on the outer side forms the default
+			 * partition of the join result.
+			 */
+			if (*default_index < 0)
+				*default_index = *merged_index;
+			else if(*default_index != *merged_index)
+			{
+				/*
+				 * Ended up with default partition on the outer side
+				 * being joined with multiple partitions on the inner
+				 * side. We don't support this case.
+				 */
+				return false;
+			}
+
+			/*
+			 * Since the merged partition acts as a default partition, it
+			 * doesn't need a separate index.
+			 */
+			*merged_index = -1;
+		}
+	}
+	else if (missing_side_inner)
+	{
+		/*
+		 * If this partition has already been mapped (say because we
+		 * found an overlapping range earlier), we know where does it
+		 * fit in the join result. Nothing to do in that case. Else
+		 * create a new merged partition.
+		 */
+		PartitionMap *extra_map = &maps_with_extra[extra_part];
+		if (extra_map->to < 0)
+		{
+			extra_map->to = *next_index;
+			*next_index = *next_index + 1;
+			*merged_index = extra_map->to;
+		}
+	}
+	else
+		Assert(*merged_index < 0);
+
+	return true;
+}
+
+static PartitionMap*
+init_partition_map(RelOptInfo *rel)
+{
+	int i, nparts = rel->nparts;
+	PartitionMap *map;
+
+	map = (PartitionMap *) palloc(sizeof(PartitionMap) * nparts);
+
+	for (i = 0; i < nparts; i++)
+	{
+		map[i].from = -1;
+		map[i].to = -1;
+	}
+
+	return map;
+}
+
+/*
+ * Allocate and initialize partition maps. We maintain four maps, two maps
+ * for each joining relation. pmap[i] gives the partition from the other
+ * relation which would join with ith partition of the given relation.
+ * Partition i from the given relation will join with partition pmap[i]
+ * from the other relation to produce partition mmap[i] of the join (merged
+ * partition).
+ *
+ * pmap[i] = -1 indicates that ith partition of a given relation does not
+ * have a matching partition from the other relation.
+ *
+ * mmap[i] = -1 indicates that ith partition of a given relation does not
+ * contribute to the join result. That can happen only when the given
+ * relation is the inner relation and it doesn't have a matching partition
+ * from the outer relation, hence pmap[i] should be -1.
+ *
+ * In case of an outer join, every partition of the outer join will appear
+ * in the join result, and thus has mmap[i] set for all i. But it's not
+ * necessary that every partition on the outer side will have a matching
+ * partition on the inner side. In such a case, we end up with pmap[i] = -1
+ * and mmap[i] != -1.
+ */
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 List **outer_parts, List **inner_parts,
+							 JoinType jointype, int partnatts,
+							 FmgrInfo *partsupfuncs, Oid *partcollations)
+
+{
+	PartitionMap *outer_maps = NULL;
+	PartitionMap *inner_maps = NULL;
+	int			outer_part = 0;
+	int			inner_part = 0;
+	PartitionBoundInfo merged_bounds = NULL;
+	int			outer_lb_index;
+	int			inner_lb_index;
+	int			next_index;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	List	   *merged_kinds = NIL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			inner_default = inner_bi->default_index;
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	outer_maps = init_partition_map(outer_rel);
+	inner_maps = init_partition_map(inner_rel);
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = 0;
+	inner_lb_index = 0;
+	next_index = 0;
+	while (outer_lb_index < outer_bi->ndatums ||
+		   inner_lb_index < inner_bi->ndatums)
+	{
+		PartitionRangeBound outer_lb, outer_ub,
+							inner_lb, inner_ub,
+							*merged_lb = NULL,
+							*merged_ub = NULL;
+
+		int			merged_index = -1;
+		bool		overlap;
+		bool		finished_outer = false;
+		bool		finished_inner = false;
+
+		/* Result of bounds comparison per partition_rbound_cmp(). */
+		int			ub_cmpval;	/* Upper bounds comparison result. */
+		int			lb_cmpval;	/* Lower bounds comparison result. */
+
+		/* Get the range bounds of the next pair of partitions. */
+		if (outer_lb_index < outer_bi->ndatums)
+			outer_part = partition_get_range_bounds(outer_bi, outer_lb_index,
+												&outer_lb, &outer_ub);
+		else
+			finished_outer = true;
+
+		if (inner_lb_index < inner_bi->ndatums)
+			inner_part = partition_get_range_bounds(inner_bi, inner_lb_index,
+												&inner_lb, &inner_ub);
+		else
+			finished_inner = true;
+
+		Assert(!finished_outer || !finished_inner);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (finished_outer)
+		{
+			overlap = false;
+			ub_cmpval = 1;
+			lb_cmpval = 1;
+		}
+		else if (finished_inner)
+		{
+			overlap = false;
+			ub_cmpval = -1;
+			lb_cmpval = -1;
+		}
+		else
+			overlap = partition_range_cmp(partnatts, partsupfuncs, partcollations,
+										  &outer_lb, &outer_ub, &inner_lb,
+										  &inner_ub, &ub_cmpval, &lb_cmpval);
+
+		if (overlap)
+		{
+			/*
+			 * The rows from overlapping portion of ranges on both sides may
+			 * join, hence the corresponding pair of partitions form a joining
+			 * pair. Match them and produce the bounds of the joint partition
+			 * and its index by merging the bounds according to the type of
+			 * join.
+			 */
+			partition_range_merge(partnatts, partsupfuncs, partcollations,
+								  jointype, &outer_lb, &outer_ub, &inner_lb,
+								  &inner_ub, &merged_lb, &merged_ub);
+
+			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+													outer_part, inner_part,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				/* Failed to match the partitions. */
+				return NULL;
+			}
+
+			/*
+			 * If the ranges overlap but don't exactly match, a row from
+			 * non-overlapping portion of the range from one side of join may
+			 * find its join partner in the previous or next overlapping
+			 * partition or default partition on the other side , if such a
+			 * partition exists. All those cases, if true, will cause one
+			 * partition from that side to match at least two partitions on the
+			 * other side; a case that we do not support now. Previous
+			 * partition has been delt with in the previous iteration of this
+			 * loop, next partition will be delt in the next iteration. We will
+			 * deal with the default partition here.
+			 */
+			if ((lb_cmpval < 0 && inner_has_default) ||
+				/* Non-overlapping range on the lower side of outer range. */
+				(lb_cmpval > 0 && outer_has_default) ||
+				/* Non-overlapping range on the lower side of inner range. */
+				(ub_cmpval < 0 && outer_has_default) ||
+				/* Non-overlapping range on the upper side of inner range. */
+				(ub_cmpval > 0 && inner_has_default))
+				/* Non-overlapping range on the upper side of outer range. */
+				return NULL;
+		}
+
+		if (ub_cmpval == 0)
+		{
+			/* Upper bounds of both the ranges match. */
+			Assert(overlap);
+
+			/* Move to the next pair of partitions. */
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the inner side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &outer_lb;
+				merged_ub = &outer_ub;
+
+				/*
+				 * For a FULL join, inner relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the inner relation acts as
+				 * INNER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_inner = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_outer = (jointype == JOIN_FULL);
+				if (!handle_missing_partition(inner_maps,
+											  outer_maps,
+											  inner_default,
+											  outer_part,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the outer side. */
+			Assert(!finished_outer);
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+		}
+		else
+		{
+			Assert(ub_cmpval > 0);
+
+			/* Upper bound of outer range higher than that of the inner. */
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the outer side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &inner_lb;
+				merged_ub = &inner_ub;
+
+				/*
+				 * For a FULL join, outer relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the outer relation acts as
+				 * OUTER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_outer = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_inner = (jointype == JOIN_FULL);
+
+				if (!handle_missing_partition(outer_maps,
+											  inner_maps,
+											  outer_default,
+											  inner_part,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the inner side. */
+			Assert (!finished_inner);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+
+		if (merged_index < 0)
+		{
+			/* We didn't find a new merged partition. */
+			continue;
+		}
+
+		/*
+		 * We have a valid partition index for the next partition of join. The
+		 * partition should have valid range.
+		 */
+		Assert(merged_lb && merged_ub);
+
+		/* Try merging new lower bound with the last upper bound. */
+		if (!partition_range_merge_next_lb(partnatts, partsupfuncs,
+										   partcollations,
+										   merged_lb->datums,
+										   merged_lb->kind, &merged_datums,
+										   &merged_kinds, &merged_indexes))
+			return NULL;
+
+		/* Add upper bound with the merged partition index. */
+		merged_datums = lappend(merged_datums, merged_ub->datums);
+		merged_kinds = lappend(merged_kinds, merged_ub->kind);
+		merged_indexes = lappend_int(merged_indexes, merged_index);
+	}
+
+	if (!merge_default_partitions(outer_bi, inner_bi,
+										  outer_maps, inner_maps,
+										  jointype, &next_index,
+										  &default_index))
+		return NULL;
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_maps, inner_maps,
+								 outer_nparts, inner_nparts,
+								 jointype, next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes,
+													  merged_kinds,
+													  -1, default_index);
+	}
+
+	/* Free any memory we used in this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype)
+{
+	PartitionMap *outer_maps = NULL;
+	PartitionMap *inner_maps = NULL;
+	int			cnto;
+	int			cnti;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			next_index = 0;
+	int			null_index = -1;
+	int			default_index = -1;
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			      *outer_indexes = outer_bi->indexes;
+	int			      *inner_indexes = inner_bi->indexes;
+	int				   outer_default = outer_bi->default_index;
+	int				   inner_default = inner_bi->default_index;
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	outer_maps = init_partition_map(outer_rel);
+	inner_maps = init_partition_map(inner_rel);
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			o_index;
+		int			i_index;
+		int			cmpval;
+		int			merged_index = -1;
+		Datum	   *merged_datum;
+		bool		finished_inner;
+		bool		finished_outer;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+		{
+			finished_outer = true;
+			odatums = NULL;
+			o_index = -1;
+		}
+		else
+		{
+			finished_outer = false;
+			odatums = outer_bi->datums[cnto];
+			o_index = outer_indexes[cnto];
+		}
+
+		if (cnti >= inner_bi->ndatums)
+		{
+			finished_inner = true;
+			idatums = NULL;
+			i_index = -1;
+		}
+		else
+		{
+			finished_inner = false;
+			idatums = inner_bi->datums[cnti];
+			i_index = inner_indexes[cnti];
+		}
+
+		/* If we exhausted both the sides, we won't enter the loop. */
+		Assert(!finished_inner || !finished_outer);
+
+		if (finished_outer)
+			cmpval = 1;
+		else if (finished_inner)
+			cmpval = -1;
+		else
+		{
+			/* Every list datum should map to a valid partition index. */
+			Assert(o_index >= 0 && i_index >= 0 &&
+				   odatums != NULL && idatums != NULL);
+
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0], idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+													o_index, i_index,
+													&next_index);
+
+			if (merged_index < 0)
+				return NULL;
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			/* A datum missing from the inner side. */
+			merged_index = -1;
+			merged_datum = odatums;
+
+			/*
+			 * For a FULL join, inner relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the inner relation acts as
+			 * INNER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_inner = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_outer = (jointype == JOIN_FULL);
+
+			if (!handle_missing_partition(inner_maps,
+										  outer_maps,
+										  inner_default,
+										  o_index,
+										  missing_side_outer,
+										  missing_side_inner,
+										  &next_index,
+										  &default_index,
+										  &merged_index))
+				return NULL;
+
+			/* Move to the next datum on the outer side. */
+			Assert(!finished_outer);
+			cnto++;
+		}
+		else
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			Assert(cmpval > 0);
+
+			/* A datum missing from the outer side. */
+			merged_index = -1;
+			merged_datum = idatums;
+
+			/*
+			 * For a FULL join, outer relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the outer relation acts as
+			 * OUTER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_outer = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_inner = (jointype == JOIN_FULL);
+
+			if (!handle_missing_partition(outer_maps,
+										  inner_maps,
+										  outer_default,
+										  i_index,
+										  missing_side_outer,
+										  missing_side_inner,
+										  &next_index,
+										  &default_index,
+										  &merged_index))
+				return NULL;
+
+			/* Move to the next datum on the right side. */
+			Assert(!finished_inner);
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	if (!merge_null_partitions(outer_bi, inner_bi,
+							   outer_maps, inner_maps,
+							   jointype, &next_index, &null_index,
+							   &default_index))
+		return NULL;
+
+	if (!merge_default_partitions(outer_bi, inner_bi,
+								  outer_maps, inner_maps,
+								  jointype, &next_index,
+								  &default_index))
+		return NULL;
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_maps, inner_maps,
+								 outer_nparts, inner_nparts,
+								 jointype, next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes, NIL,
+													  null_index, default_index);
+	}
+
+	/* Free up all extra memory before returning from this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_bounds_merge()'s arm for hash partitioned tables.
+ *
+ * If the given two hash bounds are same, the function returns the first one
+ * without any change, alongwith the lists of matching partitions. Otherwise it
+ * returns NULL.
+ *
+ * We could try merging the bounds when both the bounds have same greatest
+ * modulii. But there seems to be hardly any requirement for the same.
+ */
+static PartitionBoundInfo
+partition_hash_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype)
+{
+	int			nparts;
+	int			cnt;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * Hash partitioned table does not have explicit NULL accepting partition
+	 * and also does not have a default partition.
+	 */
+	Assert(!partition_bound_has_default(outer_bi) &&
+		   !partition_bound_has_default(inner_bi));
+	Assert(!partition_bound_accepts_nulls(outer_bi) &&
+		   !partition_bound_accepts_nulls(inner_bi));
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+
+	if (outer_nparts != inner_nparts)
+		return NULL;
+	nparts = outer_nparts;
+
+	if (outer_bi->ndatums != inner_bi->ndatums ||
+		!partition_hbounds_equal(outer_bi, inner_bi))
+		return NULL;
+
+	 /*
+	  * Cook up list of matching partitions. Since bounds are exactly same the
+	  * partitions at the same position from both the relations match.
+	  */
+	for (cnt = 0; cnt < nparts; cnt++)
+	{
+		*outer_parts = lappend_int(*outer_parts, cnt);
+		*inner_parts = lappend_int(*inner_parts, cnt);
+	}
+
+	return outer_bi;
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * If the two given partitions (given by index1 and index2 resp.) are
+ * already mapped to each other return the index of corresponding partition in
+ * the merged set of partitions.  If they do not have a merged partition
+ * associated with them, assign a new merged partition index.  If the
+ * partitions are already mapped and their mapped partitions are different from
+ * each other, they can not be merged, so return -1.
+ *
+ * partmaps1[i] gives the mapping of partitions for both relations. It
+ * describes which partition of relation 2 matches ith partition of relation 1,
+ * and which partition in the merged set matches ith partition of relation 1
+ * maps to. Similarly for partmap2.
+ *
+ * index1 and index2 are the indexes of matching partition from respective
+ * relations.
+ *
+ * *next_index is used and incremented when the given partitions require a new
+ * merged partition.
+ */
+
+static int
+map_and_merge_partitions(PartitionMap *partmaps1, PartitionMap *partmaps2,
+						 int index1, int index2, int *next_index)
+{
+	PartitionMap 	*partmap1 = &partmaps1[index1];
+	PartitionMap 	*partmap2 = &partmaps2[index2];
+	int				merged_index;
+
+	/*
+	 * If both the partitions are not mapped to each other, update the
+	 * maps.
+	 */
+	if (partmap1->from < 0 && partmap2->from < 0)
+	{
+		partmap1->from = index2;
+		partmap2->from = index1;
+	}
+
+	/*
+	 * If the given to partitions map to each other, find the corresponding
+	 * merged partition index .
+	 */
+	if (partmap1->from == index2 && partmap2->from == index1)
+	{
+		/*
+		 * If both the partitions are mapped to the same merged partition, get
+		 * the index of merged partition.
+		 */
+		if (partmap1->to == partmap2->to)
+		{
+			merged_index = partmap1->to;
+
+			/*
+			 * If the given two partitions do not have a merged partition
+			 * associated with them, allocate a new merged partition.
+			 */
+			if (merged_index < 0)
+			{
+				merged_index = *next_index;
+				*next_index = *next_index + 1;
+				partmap1->to = merged_index;
+				partmap2->to = merged_index;
+			}
+		}
+
+		/*
+		 * If partition from one relation was mapped to a merged partition but
+		 * not the partition from the other relation, map the same merged
+		 * partition to the partition from other relation, since matching
+		 * partitions map to the same merged partition.
+		 */
+		else if (partmap1->to >= 0 && partmap2->to < 0)
+		{
+			partmap2->to = partmap1->to;
+			merged_index = partmap1->to;
+		}
+		else if (partmap1->to < 0 && partmap2->to >= 0)
+		{
+			partmap1->to = partmap2->to;
+			merged_index = partmap2->to;
+		}
+		else
+		{
+			Assert(partmap1->to != partmap2->to &&
+				   partmap1->to >= 0 && partmap2->to >= 0);
+
+			/*
+			 * Both the partitions map to different merged partitions. This
+			 * means that multiple partitions from one relation matches to one
+			 * partition from the other relation. Partition-wise join does not
+			 * handle this case right now, since it requires ganging multiple
+			 * partitions together (into one RelOptInfo).
+			 */
+			merged_index = -1;
+		}
+	}
+	else
+	{
+		/*
+		 * Multiple partitions from one relation map to one partition from the
+		 * other relation. Partition-wise join does not handle this case right
+		 * now, since it requires ganging multiple partitions together (into
+		 * one RelOptInfo).
+		 */
+		merged_index = -1;
+	}
+
+	return merged_index;
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * partmaps1 map each partition from either side of the join to a merged
+ * partition resp. E.g. partmaps1[i].to gives the merged partition to which ith
+ * partition of first relation maps. Similarly for partmap2. If
+ * partmaps1[i].to == partmaps2[j].to, i and j form the matching pair of
+ * partitions.
+ *
+ * Given these maps this function produces the list pairs of partitions which
+ * when joined produce the merged partitions in the order of merged partition
+ * indexes.
+ *
+ * nparts1 and nparts2 are the number of partitions of the joining relations
+ * resp.
+ *
+ * nparts is the number of merged partitions.
+ *
+ * If successful, the pairs of partitions are returned as two separate lists,
+ * parts1 and parts2 resp., one for each side. Otherwise, those lists will be
+ * set to NIL.
+ */
+static void
+generate_matching_part_pairs(PartitionMap *partmaps1, PartitionMap *partmaps2,
+							 int nparts1, int nparts2,
+							 JoinType jointype, int nparts,
+							 List **matched_parts1, List **matched_parts2)
+{
+	bool		merged = true;
+	int		   *matching1,
+			   *matching2;
+	int 		i;
+	int			max_nparts;
+
+	*matched_parts1 = NIL;
+	*matched_parts2 = NIL;
+
+	matching1 = (int *) palloc(sizeof(int) * nparts),
+	matching2 = (int *) palloc(sizeof(int) * nparts);
+	for (i = 0; i < nparts; i++)
+		matching1[i] = matching2[i] = -1;
+
+	/* Set pairs of matching partitions. */
+	max_nparts = Max(nparts1, nparts2);
+	for (i = 0; i < max_nparts; i++)
+	{
+		if (i < nparts1)
+		{
+			PartitionMap outer_map = partmaps1[i];
+
+			if (outer_map.to >= 0)
+			{
+				Assert(outer_map.to < nparts);
+				matching1[outer_map.to] = i;
+			}
+		}
+
+		if (i < nparts2)
+		{
+			PartitionMap inner_map = partmaps2[i];
+
+			if (inner_map.to >= 0)
+			{
+				Assert(inner_map.to < nparts);
+				matching2[inner_map.to] = i;
+			}
+		}
+	}
+
+	/*
+	 * If we have a partition missing on an inner side, we need to add a dummy
+	 * relation which joins with the outer partition. If the inner relation
+	 * happens to be a base relation, it will require adding a dummy child
+	 * base relation during join processing. Right now, we freeze the base
+	 * relation arrays like PlannerInfo::simple_rte_array after planning for
+	 * base relations. Adding a new (dummy) base relation would require some
+	 * changes to that. So, right now, we do not implement partition-wise join
+	 * in such cases.
+	 */
+	for (i = 0; i < nparts; i++)
+	{
+		int			part1 = matching1[i];
+		int			part2 = matching2[i];
+
+		/* At least one of the partitions should exist. */
+		Assert(part1 >= 0 || part2 >= 0);
+
+		switch (jointype)
+		{
+			case JOIN_INNER:
+			case JOIN_SEMI:
+
+				/*
+				 * An inner or semi join can not return any row when the
+				 * matching partition on either side is missing. We should
+				 * have eliminated all such cases while merging the bounds.
+				 */
+				Assert(part1 >= 0 && part2 >= 0);
+				break;
+
+			case JOIN_LEFT:
+			case JOIN_ANTI:
+				Assert(part1 >= 0);
+				if (part2 < 0)
+					merged = false;
+				break;
+
+			case JOIN_FULL:
+				if (part1 < 0 || part2 < 0)
+					merged = false;
+				break;
+
+			default:
+				elog(ERROR, "unrecognized join type: %d", (int) jointype);
+		}
+
+		if (!merged)
+			break;
+
+		*matched_parts1 = lappend_int(*matched_parts1, part1);
+		*matched_parts2 = lappend_int(*matched_parts2, part2);
+	}
+
+	pfree(matching1);
+	pfree(matching2);
+
+	if (!merged)
+	{
+		list_free(*matched_parts1);
+		list_free(*matched_parts2);
+		*matched_parts1 = NIL;
+		*matched_parts2 = NIL;
+	}
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * Merge default partitions from both sides, if any, and assign the default
+ * partition for the join result, if necessary.
+ *
+ * If both the relations have default partitions, try mapping those to each
+ * other. If the mapping succeeds corresponding merged partition will act as
+ * the default partition of the join result.
+ *
+ * If inner side of the join has default but not the outer side, rows in it
+ * won't appear in the join result. So don't create a default partition. If
+ * outer side of the join has default but not the inner side, rows in it will
+ * appear in the join result, so create a default merged partition.
+ */
+static bool
+merge_default_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+						 PartitionMap *outer_maps, PartitionMap *inner_maps,
+						 JoinType jointype, int *next_index, int *default_index)
+{
+	int				outer_default = outer_bi->default_index;
+	int				inner_default = inner_bi->default_index;
+	bool			outer_has_default = partition_bound_has_default(outer_bi);
+	bool			inner_has_default = partition_bound_has_default(inner_bi);
+	bool			merged = true;
+	PartitionMap 	*outer_default_map = NULL;
+	PartitionMap 	*inner_default_map = NULL;
+
+	if (outer_has_default)
+		outer_default_map = &outer_maps[outer_default];
+
+	if (inner_has_default)
+		inner_default_map = &inner_maps[inner_default];
+
+	if (!outer_has_default && !inner_has_default)
+		Assert(*default_index < 0);
+	else if (outer_default_map != NULL && inner_default_map == NULL)
+	{
+		if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+			jointype == JOIN_ANTI)
+		{
+			if (outer_default_map->to < 0)
+			{
+				outer_default_map->to = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = outer_default_map->to;
+			}
+			else
+				Assert(*default_index == outer_default_map->to);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else if (outer_default_map == NULL && inner_default_map != NULL)
+	{
+		if (jointype == JOIN_FULL)
+		{
+			if (inner_default_map->to < 0)
+			{
+				inner_default_map->to = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = inner_default_map->to;
+			}
+			else
+				Assert(*default_index == inner_default_map->to);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		*default_index = map_and_merge_partitions(outer_maps, inner_maps,
+												  outer_default, inner_default,
+												  next_index);
+
+		if (*default_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
+
+/*
+ * merge_null_partitions
+ *
+ * Merge NULL partitions, i.e. a partition that can hold NULL values for a list
+ * partitioned table, if any. Find the index of merged partition to which the
+ * NULL values would belong in the join result. If one joining relation has a
+ * NULL partition but not the other, try matching it with the default partition
+ * from the other relation since the default partition may have rows with NULL
+ * partition key. We can eliminate a NULL partition when it appears only on the
+ * inner side of the join and the outer side doesn't have a default partition.
+ *
+ * When the equality operator used for join is strict, two NULL values will not
+ * be considered as equal, and thus a NULL partition can be eliminated for an
+ * inner join. But we don't check the strictness operator here.
+ */
+static bool
+merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_maps, PartitionMap *inner_maps,
+					  JoinType jointype, int *next_index,
+					  int *null_index, int *default_index)
+{
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	int			outer_ni = outer_bi->null_index;
+	int			inner_ni = inner_bi->null_index;
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	bool		merged = true;
+
+	if (!outer_has_null && !inner_has_null)
+		Assert(*null_index < 0);
+	else if (outer_has_null && !inner_has_null)
+	{
+		int			merged_index = -1;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, inner relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the inner relation acts as
+		 * INNER relation. For INNER and SEMI join OUTER and INNER
+		 * differentiation is immaterial.
+		 */
+		missing_side_inner = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_outer = (jointype == JOIN_FULL);
+
+		merged = handle_missing_partition(inner_maps,
+										  outer_maps,
+										  inner_default,
+										  outer_ni,
+										  missing_side_outer,
+										  missing_side_inner, next_index,
+										  default_index, &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join to which the outer null partition maps
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = outer_maps[outer_ni].to;
+	}
+	else if (!outer_has_null && inner_has_null)
+	{
+		int			merged_index = -1;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, outer relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the outer relation acts as OUTER
+		 * relation. For INNER and SEMI join OUTER and INNER differentiation is
+		 * immaterial.
+		 */
+		missing_side_outer = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_inner = (jointype == JOIN_FULL);
+		merged = handle_missing_partition(outer_maps,
+										  inner_maps,
+										  outer_default,
+										  inner_ni,
+										  missing_side_outer,
+										  missing_side_inner,
+										  next_index, default_index,
+										  &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join, to which the outer side null partition maps,
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = inner_maps[inner_ni].to;
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		*null_index = map_and_merge_partitions(outer_maps,
+											   inner_maps,
+											   outer_ni,
+											   inner_ni,
+											   next_index);
+		if (*null_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index b1ae39ad63..fb2dc1cbb1 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -16,6 +16,7 @@
 #include "nodes/pg_list.h"
 #include "partitioning/partdefs.h"
 #include "utils/relcache.h"
+struct RelOptInfo;				/* avoid including pathnodes.h here */
 
 
 /*
@@ -107,5 +108,11 @@ extern int partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 							  int nvalues, Datum *values, bool *is_equal);
 extern int partition_hash_bsearch(PartitionBoundInfo boundinfo,
 					   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   struct RelOptInfo *outer_rel,
+					   struct RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
-- 
2.18.0

0001-Hash-partition-bound-equality-refactoring-v20.patchapplication/octet-stream; name=0001-Hash-partition-bound-equality-refactoring-v20.patchDownload
From 8740ccc8ebd92d2d2ae9dfee7e171df15a95faa6 Mon Sep 17 00:00:00 2001
From: Etsuro Fujita <efujita@postgresql.org>
Date: Thu, 31 Jan 2019 19:18:18 +0900
Subject: [PATCH 1/3] Hash partition bound equality refactoring.

Separate the code to check whether two given hash bounds are equal into a
separate function, so that it can be called from multiple places. Right now
it's only caller is partition_bounds_equal() but later we will use it for
merging partition bounds.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/partitioning/partbounds.c | 95 +++++++++++++++++----------
 1 file changed, 61 insertions(+), 34 deletions(-)

diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index e71eb3793b..637e05e5ca 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -106,6 +106,9 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 						 Expr **keyCol,
 						 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static bool partition_hbounds_equal(PartitionBoundInfo b1,
+									PartitionBoundInfo b2);
+
 
 /*
  * get_qual_from_partbound
@@ -654,6 +657,63 @@ create_range_bounds(PartitionBoundSpec **boundspecs, int nparts,
 	return boundinfo;
 }
 
+/*
+ * Are two hash partition bound collections logically equal?
+ *
+ * Hash partition bounds store modulus and remainder in datums array which are
+ * always integers irrespective of the number of partition keys and their data
+ * types. Hence we can compare the hash bound collection without any partition
+ * key specific information. Separating this logic in a function which does not
+ * require partition key specific information allows it be called from places
+ * where the partition key specific information is not completely available.
+ */
+static bool
+partition_hbounds_equal(PartitionBoundInfo b1, PartitionBoundInfo b2)
+{
+	int			greatest_modulus = get_hash_partition_greatest_modulus(b1);
+	int			i;
+
+	Assert(b1->strategy == PARTITION_STRATEGY_HASH &&
+		   b2->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * If two hash partitioned tables have different greatest moduli,
+	 * their partition schemes don't match.  For hash partitioned table,
+	 * the greatest modulus is given by the last datum and number of
+	 * partitions is given by ndatums.
+	 */
+	if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
+		return false;
+
+	/*
+	 * We arrange the partitions in the ascending order of their modulus and
+	 * remainders.  Also every modulus is factor of next larger modulus.
+	 * Therefore we can safely store index of a given partition in indexes
+	 * array at remainder of that partition.  Also entries at (remainder + N *
+	 * modulus) positions in indexes array are all same for (modulus,
+	 * remainder) specification for any partition.  Thus datums array from both
+	 * the given bounds are same, if and only if their indexes array will be
+	 * same.  So, it suffices to compare indexes array.
+	 */
+	for (i = 0; i < greatest_modulus; i++)
+		if (b1->indexes[i] != b2->indexes[i])
+			return false;
+
+#ifdef USE_ASSERT_CHECKING
+
+	/*
+	 * Nonetheless make sure that the bounds are indeed same when the indexes
+	 * match.  Hash partition bound stores modulus and remainder at
+	 * b1->datums[i][0] and b1->datums[i][1] position respectively.
+	 */
+	for (i = 0; i < b1->ndatums; i++)
+		Assert((b1->datums[i][0] == b2->datums[i][0] &&
+				b1->datums[i][1] == b2->datums[i][1]));
+#endif
+
+	return true;
+}
+
 /*
  * Are two partition bound collections logically equal?
  *
@@ -682,41 +742,8 @@ partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
 
 	if (b1->strategy == PARTITION_STRATEGY_HASH)
 	{
-		int			greatest_modulus = get_hash_partition_greatest_modulus(b1);
-
-		/*
-		 * If two hash partitioned tables have different greatest moduli,
-		 * their partition schemes don't match.
-		 */
-		if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
+		if (!partition_hbounds_equal(b1, b2))
 			return false;
-
-		/*
-		 * We arrange the partitions in the ascending order of their moduli
-		 * and remainders.  Also every modulus is factor of next larger
-		 * modulus.  Therefore we can safely store index of a given partition
-		 * in indexes array at remainder of that partition.  Also entries at
-		 * (remainder + N * modulus) positions in indexes array are all same
-		 * for (modulus, remainder) specification for any partition.  Thus
-		 * datums array from both the given bounds are same, if and only if
-		 * their indexes array will be same.  So, it suffices to compare
-		 * indexes array.
-		 */
-		for (i = 0; i < greatest_modulus; i++)
-			if (b1->indexes[i] != b2->indexes[i])
-				return false;
-
-#ifdef USE_ASSERT_CHECKING
-
-		/*
-		 * Nonetheless make sure that the bounds are indeed same when the
-		 * indexes match.  Hash partition bound stores modulus and remainder
-		 * at b1->datums[i][0] and b1->datums[i][1] position respectively.
-		 */
-		for (i = 0; i < b1->ndatums; i++)
-			Assert((b1->datums[i][0] == b2->datums[i][0] &&
-					b1->datums[i][1] == b2->datums[i][1]));
-#endif
 	}
 	else
 	{
-- 
2.18.0

0003-Tests-for-0-1-1-1-and-1-0-partition-matching-v20.patchapplication/octet-stream; name=0003-Tests-for-0-1-1-1-and-1-0-partition-matching-v20.patchDownload
From 6fe0f54b27749fdf6dfa1f0a548977d6899653bb Mon Sep 17 00:00:00 2001
From: Amul Sul <amul.sul@enterprisedb.com>
Date: Tue, 5 Mar 2019 00:40:55 -0500
Subject: [PATCH 3/3] Tests for 0:1, 1:1 and 1:0 partition matching

Rajkumar Raghuvanshi and Ashutosh Bapat.
---
 src/test/regress/expected/partition_join.out | 3955 +++++++++++++++---
 src/test/regress/sql/partition_join.sql      |  427 +-
 2 files changed, 3767 insertions(+), 615 deletions(-)

diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index bbdc373782..7954ba2c91 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -8,59 +8,86 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-(4 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+(7 rows)
 
 -- left outer join, with whole-row reference; partitionwise join does not apply
 EXPLAIN (COSTS OFF)
@@ -72,35 +99,50 @@ SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER
    ->  Hash Right Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(22 rows)
 
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-      t1      |      t2      
---------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
- (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
- (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
- (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
- (550,0,0550) | 
-(12 rows)
+       t1       |       t2       
+----------------+----------------
+ (-250,0,-0250) | 
+ (-200,0,-0200) | 
+ (-150,0,-0150) | (0,-150,-0150)
+ (-100,0,-0100) | 
+ (-50,0,-0050)  | 
+ (0,0,0000)     | (0,0,0000)
+ (50,0,0050)    | 
+ (100,0,0100)   | 
+ (150,0,0150)   | (0,150,0150)
+ (200,0,0200)   | 
+ (250,0,0250)   | 
+ (300,0,0300)   | (0,300,0300)
+ (350,0,0350)   | 
+ (400,0,0400)   | 
+ (450,0,0450)   | (0,450,0450)
+ (500,0,0500)   | 
+ (550,0,0550)   | 
+ (600,0,0600)   | (0,600,0600)
+ (650,0,0650)   | 
+ (700,0,0700)   | 
+ (750,0,0750)   | (0,750,0750)
+(21 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
@@ -112,35 +154,53 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
    ->  Append
          ->  Hash Right Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Right Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t2_2.b)
-(20 rows)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+      |       | -225 | -0225
+      |       |  -75 | -0075
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(14 rows)
 
 -- full outer join, with placeholder vars
 EXPLAIN (COSTS OFF)
@@ -148,8 +208,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                             QUERY PLAN                            
 ------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               Filter: (((50) = prt1_p0.a) OR ((75) = prt2_p0.b))
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
@@ -174,7 +242,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               Filter: (((50) = prt1_p4.a) OR ((75) = prt2_p4.b))
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
@@ -211,8 +287,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Hash Left Join
+               Hash Cond: (prt1_p0.a = b)
+               ->  Seq Scan on prt1_p0
+                     Filter: ((a < 450) AND (b = 0))
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
          ->  Hash Left Join
                Hash Cond: (prt1_p1.a = b)
                ->  Seq Scan on prt1_p1
@@ -227,29 +310,42 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                ->  Hash
                      ->  Seq Scan on prt1_p2
                            Filter: ((a < 450) AND (b = 0))
-(17 rows)
+(24 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-(9 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+(14 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
                          QUERY PLAN                         
 ------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = b)
+               Filter: ((prt1_p0.b = 0) OR (a = 0))
+               ->  Seq Scan on prt1_p0
+                     Filter: (a < 450)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = b)
                Filter: ((prt1_p1.b = 0) OR (a = 0))
@@ -274,64 +370,153 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                ->  Hash
                      ->  Result
                            One-Time Filter: false
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt2_p4.b = a)
+               Filter: ((b = 0) OR (prt2_p4.a = 0))
+               ->  Seq Scan on prt2_p4
+                     Filter: (b > 250)
+               ->  Hash
+                     ->  Result
+                           One-Time Filter: false
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
-(12 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+      |       | 375 | 0375
+      |       | 450 | 0450
+      |       | 525 | 0525
+      |       | 600 | 0600
+      |       | 675 | 0675
+      |       | 750 | 0750
+(20 rows)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Semi Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Semi Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                            Filter: (a = 0)
-         ->  Nested Loop Semi Join
-               Join Filter: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
-               ->  Materialize
-                     ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
-(24 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt1_p3 t2_3
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+                           Filter: (b = 0)
+(37 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
 
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
@@ -342,27 +527,82 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
    ->  Append
          ->  Hash Anti Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Hash Anti Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Hash Anti Join
                Hash Cond: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
-(17 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(27 rows)
 
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
-  sum  |         avg          | sum  |         avg         
--------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+  sum  |         avg          | sum  |        avg         
+-------+----------------------+------+--------------------
+ 95550 | 273.0000000000000000 | 2200 | 6.2857142857142857
 (1 row)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Append
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p0 t1
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+               Index Cond: (a = t1.b)
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.b = t2_1.a)
+         ->  Seq Scan on prt2_p1 t1_1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p1 t2_1
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p2 t1_2
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+               Index Cond: (a = t1_2.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p3 t1_3
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+               Index Cond: (a = t1_3.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p4 t1_4
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+               Index Cond: (a = t1_4.b)
+(27 rows)
+
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+  b   |   c   
+------+-------
+ -225 | -0225
+  -75 | -0075
+   75 | 0075
+  225 | 0225
+  375 | 0375
+  525 | 0525
+  675 | 0675
+(7 rows)
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -374,49 +614,74 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
+                     ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
                            Index Cond: (a = t1.a)
-                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t3
                            Index Cond: (b = t2.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_1
+                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
                            Index Cond: (a = t1_1.a)
-                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3_1
                            Index Cond: (b = t2_1.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_2
+                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
                            Index Cond: (a = t1_2.a)
-                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_2
                            Index Cond: (b = t2_2.a)
-(27 rows)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                           Index Cond: (a = t1_3.a)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_3
+                           Index Cond: (b = t2_3.a)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                           Index Cond: (a = t1_4.a)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t3_4
+                           Index Cond: (b = t2_4.a)
+(43 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   | t2a | t3a | least 
------+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
- 100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
- 250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
- 400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
- 550 | 0 | 0550 |     |     |      
-(12 rows)
+  a   | b |   c   | t2a  | t3a | least 
+------+---+-------+------+-----+-------
+ -250 | 0 | -0250 |      |     |      
+ -200 | 0 | -0200 |      |     |      
+ -150 | 0 | -0150 | -150 |   0 |  -150
+ -100 | 0 | -0100 |      |     |      
+  -50 | 0 | -0050 |      |     |      
+    0 | 0 | 0000  |    0 |   0 |     0
+   50 | 0 | 0050  |      |     |      
+  100 | 0 | 0100  |      |     |      
+  150 | 0 | 0150  |  150 |   0 |   150
+  200 | 0 | 0200  |      |     |      
+  250 | 0 | 0250  |      |     |      
+  300 | 0 | 0300  |  300 |   0 |   300
+  350 | 0 | 0350  |      |     |      
+  400 | 0 | 0400  |      |     |      
+  450 | 0 | 0450  |  450 |   0 |   450
+  500 | 0 | 0500  |      |     |      
+  550 | 0 | 0550  |      |     |      
+  600 | 0 | 0600  |  600 |   0 |   600
+  650 | 0 | 0650  |      |     |      
+  700 | 0 | 0700  |      |     |      
+  750 | 0 | 0750  |  750 |   0 |   750
+(21 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
@@ -430,64 +695,95 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
          Hash Cond: ((t1.c)::text = (t2.c)::text)
          Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Seq Scan on prt1_p1 t1
-               ->  Seq Scan on prt1_p2 t1_1
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
          ->  Hash
                ->  Append
                      ->  Hash Join
                            Hash Cond: (t2.a = t3.b)
-                           ->  Seq Scan on prt1_p1 t2
+                           ->  Seq Scan on prt1_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t3
+                                 ->  Seq Scan on prt2_p0 t3
                      ->  Hash Join
                            Hash Cond: (t2_1.a = t3_1.b)
-                           ->  Seq Scan on prt1_p2 t2_1
+                           ->  Seq Scan on prt1_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t3_1
+                                 ->  Seq Scan on prt2_p1 t3_1
                      ->  Hash Join
                            Hash Cond: (t2_2.a = t3_2.b)
-                           ->  Seq Scan on prt1_p3 t2_2
+                           ->  Seq Scan on prt1_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p3 t3_2
-(26 rows)
+                                 ->  Seq Scan on prt2_p2 t3_2
+                     ->  Hash Join
+                           Hash Cond: (t2_3.a = t3_3.b)
+                           ->  Seq Scan on prt1_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t3_3
+                     ->  Hash Join
+                           Hash Cond: (t2_4.a = t3_4.b)
+                           ->  Seq Scan on prt1_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t3_4
+(38 rows)
 
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, t2.b t2b, t2.c t2c, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
-  a  | t2a | t2c  
------+-----+------
-   0 |   0 | 0000
-  50 |     | 
- 100 |     | 
- 150 | 150 | 0150
- 200 |     | 
- 250 |     | 
- 300 | 300 | 0300
- 350 |     | 
- 400 |     | 
- 450 | 450 | 0450
- 500 |     | 
- 550 |     | 
-(12 rows)
+  a   | t2a  |  t2c  
+------+------+-------
+ -250 |      | 
+ -200 |      | 
+ -150 | -150 | -0150
+ -100 |      | 
+  -50 |      | 
+    0 |    0 | 0000
+   50 |      | 
+  100 |      | 
+  150 |  150 | 0150
+  200 |      | 
+  250 |      | 
+  300 |  300 | 0300
+  350 |      | 
+  400 |      | 
+  450 |  450 | 0450
+  500 |      | 
+  550 |      | 
+  600 |  600 | 0600
+  650 |      | 
+  700 |      | 
+  750 |  750 | 0750
+(21 rows)
 
 --
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
@@ -498,32 +794,49 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
    ->  Append
          ->  Hash Join
                Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
-               ->  Seq Scan on prt2_e_p1 t2
+               ->  Seq Scan on prt2_e_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 t1
+                     ->  Seq Scan on prt1_e_p0 t1
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
-               ->  Seq Scan on prt2_e_p2 t2_1
+               Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+               ->  Seq Scan on prt1_e_p1 t1_1
+                     Filter: (c = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 t1_1
-                           Filter: (c = 0)
+                     ->  Seq Scan on prt2_e_p1 t2_1
          ->  Hash Join
                Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
-               ->  Seq Scan on prt2_e_p3 t2_2
+               ->  Seq Scan on prt2_e_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 t1_2
+                     ->  Seq Scan on prt1_e_p2 t1_2
                            Filter: (c = 0)
-(21 rows)
+         ->  Hash Join
+               Hash Cond: (((t2_3.b + t2_3.a) / 2) = ((t1_3.a + t1_3.b) / 2))
+               ->  Seq Scan on prt2_e_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p3 t1_3
+                           Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (((t2_4.b + t2_4.a) / 2) = ((t1_4.a + t1_4.b) / 2))
+               ->  Seq Scan on prt2_e_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4 t1_4
+                           Filter: (c = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
- 150 | 0 | 150 | 0
- 300 | 0 | 300 | 0
- 450 | 0 | 450 | 0
-(4 rows)
+  a   | c |  b   | c 
+------+---+------+---
+ -250 | 0 | -250 | 0
+ -100 | 0 | -100 | 0
+    0 | 0 |    0 | 0
+    0 | 0 |    0 | 0
+  150 | 0 |  150 | 0
+  300 | 0 |  300 | 0
+  450 | 0 |  450 | 0
+  600 | 0 |  600 | 0
+  750 | 0 |  750 | 0
+(9 rows)
 
 --
 -- N-way join
@@ -536,154 +849,232 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = ((t3.a + t3.b) / 2))
+               Join Filter: (t1.a = t2.b)
                ->  Hash Join
-                     Hash Cond: (t2.b = t1.a)
-                     ->  Seq Scan on prt2_p1 t2
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on prt1_e_p0 t3
                      ->  Hash
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                     Index Cond: (((a + b) / 2) = t2.b)
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
+                     Index Cond: (b = ((t3.a + t3.b) / 2))
          ->  Nested Loop
                Join Filter: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_1.b = t1_1.a)
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
+               ->  Index Scan using iprt1_e_p4_ab2 on prt1_e_p1 t3_1
                      Index Cond: (((a + b) / 2) = t2_1.b)
          ->  Nested Loop
                Join Filter: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_2.b = t1_2.a)
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
+               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_2
                      Index Cond: (((a + b) / 2) = t2_2.b)
-(33 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = ((t3_3.a + t3_3.b) / 2))
+               ->  Nested Loop
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t2_3.b)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t2_4.b)
+               ->  Hash Join
+                     Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                     ->  Seq Scan on prt1_e_p4 t3_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_p4 t1_4
+                                 Filter: (b = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = ((t3_4.a + t3_4.b) / 2))
+(52 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
-(4 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(8 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                          QUERY PLAN                          
---------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Hash Right Join
                Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
-               ->  Seq Scan on prt1_e_p1 t3
+               ->  Seq Scan on prt1_e_p0 t3
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2.b = t1.a)
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_1.a)
-               ->  Seq Scan on prt1_e_p2 t3_1
+               ->  Seq Scan on prt1_e_p1 t3_1
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_1.b = t1_1.a)
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_2.a)
-               ->  Seq Scan on prt1_e_p3 t3_2
+               ->  Seq Scan on prt1_e_p2 t3_2
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_2.b = t1_2.a)
-                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                                        Filter: (b = 0)
-(33 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+               ->  Seq Scan on prt1_e_p4 t3_4
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_4.b = t1_4.a)
+                           ->  Seq Scan on prt2_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_p4 t1_4
+                                       Filter: (b = 0)
+(51 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                            QUERY PLAN                             
--------------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p1 t3
+                           ->  Seq Scan on prt1_e_p0 t3
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p1_b on prt2_p1 t2
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
                      Index Cond: (b = t1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p2 t3_1
+                           ->  Seq Scan on prt1_e_p1 t3_1
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
+               ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
                      Index Cond: (b = t1_1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p3 t3_2
+                           ->  Seq Scan on prt1_e_p2 t3_2
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
                      Index Cond: (b = t1_2.a)
-(30 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_e_p3 t3_3
+                           Filter: (c = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                           Index Cond: (a = ((t3_3.a + t3_3.b) / 2))
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Nested Loop Left Join
+               ->  Hash Right Join
+                     Hash Cond: (t1_4.a = ((t3_4.a + t3_4.b) / 2))
+                     ->  Seq Scan on prt1_p4 t1_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p4 t3_4
+                                 Filter: (c = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = t1_4.a)
+(47 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -692,21 +1083,34 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                                                    QUERY PLAN                                                   
 ----------------------------------------------------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   Sort Key: prt1_p0.a, prt2_p0.b, ((prt1_e_p0.a + prt1_e_p0.b))
    ->  Append
          ->  Hash Full Join
-               Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
-               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               Hash Cond: (prt1_p0.a = ((prt1_e_p0.a + prt1_e_p0.b) / 2))
+               Filter: ((prt1_p0.a = (50)) OR (prt2_p0.b = (75)) OR (((prt1_e_p0.a + prt1_e_p0.b) / 2) = (50)))
                ->  Hash Full Join
-                     Hash Cond: (prt1_p1.a = prt2_p1.b)
-                     ->  Seq Scan on prt1_p1
+                     Hash Cond: (prt1_p0.a = prt2_p0.b)
+                     ->  Seq Scan on prt1_p0
                            Filter: (b = 0)
                      ->  Hash
-                           ->  Seq Scan on prt2_p1
+                           ->  Seq Scan on prt2_p0
                                  Filter: (a = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1
+                     ->  Seq Scan on prt1_e_p0
                            Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = prt1_p1.a)
+               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               ->  Seq Scan on prt1_e_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on prt1_p1
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1
+                                       Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
                Filter: ((prt1_p2.a = (50)) OR (prt2_p2.b = (75)) OR (((prt1_e_p2.a + prt1_e_p2.b) / 2) = (50)))
@@ -733,7 +1137,20 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                ->  Hash
                      ->  Seq Scan on prt1_e_p3
                            Filter: (c = 0)
-(42 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = ((prt1_e_p4.a + prt1_e_p4.b) / 2))
+               Filter: ((prt1_p4.a = (50)) OR (prt2_p4.b = (75)) OR (((prt1_e_p4.a + prt1_e_p4.b) / 2) = (50)))
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = prt2_p4.b)
+                     ->  Seq Scan on prt1_p4
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4
+                                 Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4
+                           Filter: (c = 0)
+(68 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
@@ -751,172 +1168,260 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = t1_3.b)
+               Join Filter: (t1.a = t1_5.b)
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Join
-                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
-                           ->  Seq Scan on prt1_e_p1 t2
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t1_3
+                                 ->  Seq Scan on prt2_p0 t1_5
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
                      Index Cond: (a = ((t2.a + t2.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_1.a = t1_4.b)
+               Join Filter: (t1_1.a = t1_6.b)
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Join
-                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
-                           ->  Seq Scan on prt1_e_p2 t2_1
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_6.b)
+                           ->  Seq Scan on prt1_e_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t1_4
+                                 ->  Seq Scan on prt2_p1 t1_6
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
                      Index Cond: (a = ((t2_1.a + t2_1.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_2.a = t1_5.b)
+               Join Filter: (t1_2.a = t1_7.b)
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
+                           ->  Seq Scan on prt2_p2 t1_7
                                  Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
+                           ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t2_2
+                                 Index Cond: (((a + b) / 2) = t1_7.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t1_8.b)
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Nested Loop
+                           ->  Seq Scan on prt2_p3 t1_8
+                                 Filter: (a = 0)
+                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_3
+                                 Index Cond: (((a + b) / 2) = t1_8.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = ((t2_3.a + t2_3.b) / 2))
+                     Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t1_9.b)
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Join
+                           Hash Cond: (((t2_4.a + t2_4.b) / 2) = t1_9.b)
+                           ->  Seq Scan on prt1_e_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t1_9
+                                       Filter: (a = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = ((t2_4.a + t2_4.b) / 2))
+                     Filter: (b = 0)
+(66 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                               QUERY PLAN                                
--------------------------------------------------------------------------
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
-                           ->  Seq Scan on prt2_p1 t1_3
+                           Hash Cond: (t1_5.b = ((t1_10.a + t1_10.b) / 2))
+                           ->  Seq Scan on prt2_p0 t1_5
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t1_6
+                                 ->  Seq Scan on prt1_e_p0 t1_10
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
-                     Index Cond: (a = t1_3.b)
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t1_5.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
-                           ->  Seq Scan on prt2_p2 t1_4
+                           Hash Cond: (t1_6.b = ((t1_11.a + t1_11.b) / 2))
+                           ->  Seq Scan on prt2_p1 t1_6
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t1_7
+                                 ->  Seq Scan on prt1_e_p1 t1_11
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
-                     Index Cond: (a = t1_4.b)
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
+                     Index Cond: (a = t1_6.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
-                           ->  Seq Scan on prt2_p3 t1_5
+                           Hash Cond: (t1_7.b = ((t1_12.a + t1_12.b) / 2))
+                           ->  Seq Scan on prt2_p2 t1_7
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p3 t1_8
+                                 ->  Seq Scan on prt1_e_p2 t1_12
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t1_5.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t1_7.b)
                      Filter: (b = 0)
-(39 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_8.b = ((t1_13.a + t1_13.b) / 2))
+                           ->  Seq Scan on prt2_p3 t1_8
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p3 t1_13
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t1_8.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_9.b = ((t1_14.a + t1_14.b) / 2))
+                           ->  Seq Scan on prt2_p4 t1_9
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t1_14
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = t1_9.b)
+                     Filter: (b = 0)
+(63 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 -- test merge joins
 SET enable_hashjoin TO off;
 SET enable_nestloop TO off;
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                           QUERY PLAN                           
-----------------------------------------------------------------
+                            QUERY PLAN                            
+------------------------------------------------------------------
  Merge Append
    Sort Key: t1.a
    ->  Merge Semi Join
-         Merge Cond: (t1.a = t1_3.b)
+         Merge Cond: (t1.a = t1_5.b)
          ->  Sort
                Sort Key: t1.a
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+               Merge Cond: (t1_5.b = (((t1_10.a + t1_10.b) / 2)))
                ->  Sort
-                     Sort Key: t1_3.b
-                     ->  Seq Scan on prt2_p1 t1_3
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on prt2_p0 t1_5
                ->  Sort
-                     Sort Key: (((t1_6.a + t1_6.b) / 2))
-                     ->  Seq Scan on prt1_e_p1 t1_6
+                     Sort Key: (((t1_10.a + t1_10.b) / 2))
+                     ->  Seq Scan on prt1_e_p0 t1_10
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_1.a = t1_4.b)
+         Merge Cond: (t1_1.a = t1_6.b)
          ->  Sort
                Sort Key: t1_1.a
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+               Merge Cond: (t1_6.b = (((t1_11.a + t1_11.b) / 2)))
                ->  Sort
-                     Sort Key: t1_4.b
-                     ->  Seq Scan on prt2_p2 t1_4
+                     Sort Key: t1_6.b
+                     ->  Seq Scan on prt2_p1 t1_6
                ->  Sort
-                     Sort Key: (((t1_7.a + t1_7.b) / 2))
-                     ->  Seq Scan on prt1_e_p2 t1_7
+                     Sort Key: (((t1_11.a + t1_11.b) / 2))
+                     ->  Seq Scan on prt1_e_p1 t1_11
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_2.a = t1_5.b)
+         Merge Cond: (t1_2.a = t1_7.b)
          ->  Sort
                Sort Key: t1_2.a
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+               Merge Cond: (t1_7.b = (((t1_12.a + t1_12.b) / 2)))
                ->  Sort
-                     Sort Key: t1_5.b
-                     ->  Seq Scan on prt2_p3 t1_5
+                     Sort Key: t1_7.b
+                     ->  Seq Scan on prt2_p2 t1_7
                ->  Sort
-                     Sort Key: (((t1_8.a + t1_8.b) / 2))
-                     ->  Seq Scan on prt1_e_p3 t1_8
+                     Sort Key: (((t1_12.a + t1_12.b) / 2))
+                     ->  Seq Scan on prt1_e_p2 t1_12
                            Filter: (c = 0)
-(47 rows)
+   ->  Merge Semi Join
+         Merge Cond: (t1_3.a = t1_8.b)
+         ->  Sort
+               Sort Key: t1_3.a
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_8.b = (((t1_13.a + t1_13.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_8.b
+                     ->  Seq Scan on prt2_p3 t1_8
+               ->  Sort
+                     Sort Key: (((t1_13.a + t1_13.b) / 2))
+                     ->  Seq Scan on prt1_e_p3 t1_13
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_4.a = t1_9.b)
+         ->  Sort
+               Sort Key: t1_4.a
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_9.b = (((t1_14.a + t1_14.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_9.b
+                     ->  Seq Scan on prt2_p4 t1_9
+               ->  Sort
+                     Sort Key: (((t1_14.a + t1_14.b) / 2))
+                     ->  Seq Scan on prt1_e_p4 t1_14
+                           Filter: (c = 0)
+(77 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
@@ -933,14 +1438,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
                            ->  Sort
                                  Sort Key: (((t3.a + t3.b) / 2))
-                                 ->  Seq Scan on prt1_e_p1 t3
+                                 ->  Seq Scan on prt1_e_p0 t3
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1.a
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                ->  Sort
                      Sort Key: t2.b
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Merge Left Join
                Merge Cond: (t1_1.a = t2_1.b)
                ->  Sort
@@ -949,14 +1454,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
                            ->  Sort
                                  Sort Key: (((t3_1.a + t3_1.b) / 2))
-                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                 ->  Seq Scan on prt1_e_p1 t3_1
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_1.a
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                ->  Sort
                      Sort Key: t2_1.b
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Merge Left Join
                Merge Cond: (t1_2.a = t2_2.b)
                ->  Sort
@@ -965,32 +1470,74 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
                            ->  Sort
                                  Sort Key: (((t3_2.a + t3_2.b) / 2))
-                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                 ->  Seq Scan on prt1_e_p2 t3_2
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_2.a
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                ->  Sort
                      Sort Key: t2_2.b
-                     ->  Seq Scan on prt2_p3 t2_2
-(51 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Merge Left Join
+               Merge Cond: (t1_3.a = t2_3.b)
+               ->  Sort
+                     Sort Key: t1_3.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_3.a + t3_3.b) / 2)) = t1_3.a)
+                           ->  Sort
+                                 Sort Key: (((t3_3.a + t3_3.b) / 2))
+                                 ->  Seq Scan on prt1_e_p3 t3_3
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_3.a
+                                 ->  Seq Scan on prt1_p3 t1_3
+               ->  Sort
+                     Sort Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Merge Left Join
+               Merge Cond: (t1_4.a = t2_4.b)
+               ->  Sort
+                     Sort Key: t1_4.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_4.a + t3_4.b) / 2)) = t1_4.a)
+                           ->  Sort
+                                 Sort Key: (((t3_4.a + t3_4.b) / 2))
+                                 ->  Seq Scan on prt1_e_p4 t3_4
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_4.a
+                                 ->  Seq Scan on prt1_p4 t1_4
+               ->  Sort
+                     Sort Key: t2_4.b
+                     ->  Seq Scan on prt2_p4 t2_4
+(83 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- MergeAppend on nullable column
 EXPLAIN (COSTS OFF)
@@ -998,8 +1545,18 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, b
+   Sort Key: prt1_p0.a, b
    ->  Append
+         ->  Merge Left Join
+               Merge Cond: (prt1_p0.a = b)
+               ->  Sort
+                     Sort Key: prt1_p0.a
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
+               ->  Sort
+                     Sort Key: b
+                     ->  Result
+                           One-Time Filter: false
          ->  Merge Left Join
                Merge Cond: (prt1_p1.a = b)
                ->  Sort
@@ -1020,21 +1577,26 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                      Sort Key: prt2_p2.b
                      ->  Seq Scan on prt2_p2
                            Filter: (b > 250)
-(23 rows)
+(33 rows)
 
 SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  b  
------+-----
-   0 |    
-  50 |    
- 100 |    
- 150 |    
- 200 |    
- 250 |    
- 300 | 300
- 350 |    
- 400 |    
-(9 rows)
+  a   |  b  
+------+-----
+ -250 |    
+ -200 |    
+ -150 |    
+ -100 |    
+  -50 |    
+    0 |    
+   50 |    
+  100 |    
+  150 |    
+  200 |    
+  250 |    
+  300 | 300
+  350 |    
+  400 |    
+(14 rows)
 
 -- merge join when expression with whole-row reference needs to be sorted;
 -- partitionwise join does not apply
@@ -1048,30 +1610,797 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
          Sort Key: t1.a, ((((t1.*)::prt1))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
    ->  Sort
          Sort Key: t2.b, ((((t2.*)::prt2))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt2_p1 t2
-                     ->  Seq Scan on prt2_p2 t2_1
-                     ->  Seq Scan on prt2_p3 t2_2
-(16 rows)
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+(20 rows)
 
 SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.b ORDER BY t1.a;
- a  | b  
-----+----
-  0 |  0
-  6 |  6
- 12 | 12
- 18 | 18
- 24 | 24
-(5 rows)
+  a  |  b  
+-----+-----
+ -24 | -24
+ -18 | -18
+ -12 | -12
+  -6 |  -6
+   0 |   0
+   6 |   6
+  12 |  12
+  18 |  18
+  24 |  24
+(9 rows)
 
 RESET enable_hashjoin;
 RESET enable_nestloop;
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p4 t2_3
+               ->  Seq Scan on prt2_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_4
+                           Filter: (b = 0)
+(22 rows)
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p3 t1_2
+                           Filter: (b = 0)
+(21 rows)
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -150 | -0150 | 0 | -0150
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+(7 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 |   | 
+ -200 | -0200 |   | 
+ -150 | -0150 | 0 | -0150
+ -100 | -0100 |   | 
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+(21 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -200 | 0 | -0200
+ -100 | 0 | -0100
+  -50 | 0 | -0050
+   50 | 0 | 0050
+  100 | 0 | 0100
+  200 | 0 | 0200
+  250 | 0 | 0250
+  350 | 0 | 0350
+  400 | 0 | 0400
+  500 | 0 | 0500
+  550 | 0 | 0550
+  650 | 0 | 0650
+  700 | 0 | 0700
+(14 rows)
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+                             QUERY PLAN                              
+---------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t3.c
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p0_a on prt1_p0 t3
+                           Index Cond: (a = t2.b)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t2.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_1.a = t2_1.b)
+                           ->  Seq Scan on prt1_p1 t3_1
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1 t2_1
+                                       Filter: (a = 0)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p2_a on prt1_p2 t3_2
+                           Index Cond: (a = t2_2.b)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t3_3
+                           Index Cond: (a = t2_3.b)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_4.a = t2_4.b)
+                           ->  Seq Scan on prt1_p4 t3_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t2_4
+                                       Filter: (a = 0)
+(47 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+  a   | a |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.b
+   ->  Hash Right Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p5 t2_5
+                           Filter: (a = 0)
+(24 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: (t1.a = t2.b)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.a, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
+                     Filter: (a = 0)
+(22 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
 --
 -- partitioned by multiple columns
 --
@@ -1141,82 +2470,1552 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 -- test partition matching with N-way join
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                   QUERY PLAN                                   
---------------------------------------------------------------------------------
+                                           QUERY PLAN                                           
+------------------------------------------------------------------------------------------------
  GroupAggregate
    Group Key: t1.c, t2.c, t3.c
    ->  Sort
          Sort Key: t1.c, t3.c
          ->  Append
                ->  Hash Join
-                     Hash Cond: (t1.c = ltrim(t3.c, 'A'::text))
+                     Hash Cond: ((t1.c)::text = ltrim(t3.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1.b = t2.b) AND (t1.c = t2.c))
-                           ->  Seq Scan on plt1_p1 t1
+                           Hash Cond: ((t2.b = t1.b) AND ((t2.c)::text = (t1.c)::text))
+                           ->  Seq Scan on plt2_p4 t2
                            ->  Hash
-                                 ->  Seq Scan on plt2_p1 t2
+                                 ->  Seq Scan on plt1_p4 t1
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p1 t3
+                           ->  Seq Scan on plt1_e_p4 t3
                ->  Hash Join
-                     Hash Cond: (t1_1.c = ltrim(t3_1.c, 'A'::text))
+                     Hash Cond: ((t1_1.c)::text = ltrim(t3_1.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
-                           ->  Seq Scan on plt1_p2 t1_1
+                           Hash Cond: ((t1_1.b = t2_1.b) AND ((t1_1.c)::text = (t2_1.c)::text))
+                           ->  Seq Scan on plt1_p1 t1_1
                            ->  Hash
-                                 ->  Seq Scan on plt2_p2 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p2 t3_1
+                           ->  Seq Scan on plt1_e_p1 t3_1
                ->  Hash Join
-                     Hash Cond: (t1_2.c = ltrim(t3_2.c, 'A'::text))
+                     Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
-                           ->  Seq Scan on plt1_p3 t1_2
+                           Hash Cond: ((t1_2.b = t2_2.b) AND ((t1_2.c)::text = (t2_2.c)::text))
+                           ->  Seq Scan on plt1_p2 t1_2
                            ->  Hash
-                                 ->  Seq Scan on plt2_p3 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p3 t3_2
-(32 rows)
+                           ->  Seq Scan on plt1_e_p2 t3_2
+               ->  Hash Join
+                     Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2_3.b = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text))
+                           ->  Seq Scan on plt2_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p3 t3_3
+(41 rows)
 
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  75.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 123.0000000000000000 | 123.0000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 225.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 273.0000000000000000 | 273.0000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 375.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 423.0000000000000000 | 423.0000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 525.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 573.0000000000000000 | 573.0000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p5 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_3
+                                 ->  Seq Scan on plt2_p3 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Seq Scan on plt2_p2 t2_2
+                                 ->  Seq Scan on plt2_p3 t2_3
+(22 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1241,22 +4040,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
 --------------------------------------------------
  Hash Left Join
    Hash Cond: (t2.b = a)
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t3.a = t2.b)
-               ->  Seq Scan on prt1_p1 t3
-               ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
-         ->  Hash Join
-               Hash Cond: (t3_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t3_1
-               ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
-         ->  Hash Join
-               Hash Cond: (t3_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t3_2
-               ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
+   ->  Hash Join
+         Hash Cond: (t3.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t3
+               ->  Seq Scan on prt1_p1 t3_1
+               ->  Seq Scan on prt1_p2 t3_2
+               ->  Seq Scan on prt1_p3 t3_3
+               ->  Seq Scan on prt1_p4 t3_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
    ->  Hash
          ->  Result
                One-Time Filter: false
@@ -1271,16 +4070,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 --
 -- tests for hash partitioned tables.
@@ -1356,41 +4161,9 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  273.0000000000000000 | 273.0000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
 (6 rows)
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
- Sort
-   Sort Key: t1.a
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
-               ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
-               ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
-                           Filter: (b = 0)
-(21 rows)
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
@@ -1405,26 +4178,24 @@ SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c
    Sort Key: t1.c
    ->  HashAggregate
          Group Key: t1.c, t2.c
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t2.c = t1.c)
-                     ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p1 t1
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on plt1_p4 t1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_1.c = t1_1.c)
-                     ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_p2 t1_1
+                           ->  Seq Scan on plt1_p1 t1_1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_2.c = t1_2.c)
-                     ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Seq Scan on plt1_p2 t1_2
                                  Filter: ((a % 25) = 0)
-(23 rows)
+                           ->  Seq Scan on plt1_p3 t1_3
+                                 Filter: ((a % 25) = 0)
+(21 rows)
 
 --
 -- multiple levels of partitioning
@@ -1826,64 +4597,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
  Hash Join
    Hash Cond: (t1.a = t2.a)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt4_n_p1 t2
                ->  Seq Scan on prt4_n_p2 t2_1
                ->  Seq Scan on prt4_n_p3 t2_2
-(11 rows)
+(13 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a and t1.a = t3.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                        QUERY PLAN                        
+----------------------------------------------------------
  Hash Join
-   Hash Cond: (t2.a = t1.a)
+   Hash Cond: (t3.b = t1.a)
    ->  Append
-         ->  Seq Scan on prt4_n_p1 t2
-         ->  Seq Scan on prt4_n_p2 t2_1
-         ->  Seq Scan on prt4_n_p3 t2_2
+         ->  Seq Scan on prt2_p0 t3
+         ->  Seq Scan on prt2_p1 t3_1
+         ->  Seq Scan on prt2_p2 t3_2
+         ->  Seq Scan on prt2_p3 t3_3
+         ->  Seq Scan on prt2_p4 t3_4
+         ->  Seq Scan on prt2_p5 t3_5
    ->  Hash
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.a = t3_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_2.b)
-                     ->  Seq Scan on prt1_p3 t1_2
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3 t3_2
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt4_n_p1 t2
+                           ->  Seq Scan on prt4_n_p2 t2_1
+                           ->  Seq Scan on prt4_n_p3 t2_2
 (23 rows)
 
 -- partitionwise join can not be applied if there are no equi-join conditions
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (b > t1.a)
-(12 rows)
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+(16 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partitionwise join
@@ -1969,16 +4746,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
-                     ->  Seq Scan on plt1_p1 t3
-                     ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3
+                     ->  Seq Scan on plt1_p1 t3_1
+                     ->  Seq Scan on plt1_p2 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partitionwise join can not be applied for a join between list and range
 -- partitioned table
@@ -1989,14 +4767,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
  Hash Full Join
    Hash Cond: ((t2.c)::text = (t1.c)::text)
    ->  Append
-         ->  Seq Scan on prt1_p1 t2
-         ->  Seq Scan on prt1_p2 t2_1
-         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p0 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p2 t2_2
+         ->  Seq Scan on prt1_p3 t2_3
+         ->  Seq Scan on prt1_p4 t2_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt1_n_p1 t1
                ->  Seq Scan on prt1_n_p2 t1_1
-(10 rows)
+(12 rows)
 
 -- partitionwise join can not be applied if only one of joining table has
 -- default partition
@@ -2012,16 +4792,23 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
    ->  Hash Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index c1c9859651..0e884835fb 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -10,25 +10,39 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -67,11 +81,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -93,20 +115,30 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 
 EXPLAIN (COSTS OFF)
@@ -169,6 +201,128 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -193,28 +347,79 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
 -- test partition matching with N-way join
@@ -222,6 +427,175 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
@@ -267,27 +641,18 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
 ALTER TABLE plt2 DETACH PARTITION plt2_p3;
 ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
 ANALYZE plt2;
-
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.a % 25 = 0 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
 --
 -- multiple levels of partitioning
 --
-- 
2.18.0

#67Rajkumar Raghuwanshi
rajkumar.raghuwanshi@enterprisedb.com
In reply to: amul sul (#66)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Thu, Mar 7, 2019 at 8:20 PM amul sul <sulamul@gmail.com> wrote:

On Thu, Mar 7, 2019 at 1:02 PM amul sul <sulamul@gmail.com> wrote:

Thanks Rajkumar,

I am looking into this.

The crash happens when none of the if-else branch of
handle_missing_partition()
evaluates and returns merged_index unassigned.

Let me explain, in Rajkumar 's test case, the join type is JOIN_INNER.
When
only outer rel has null partition, merge_null_partitions() function calls
handle_missing_partition() with missing_side_inner = false and
missing_side_outer = false argument value which fails to set merged_index.

In the attached patch, I tried to fix this case by setting merged_index
explicitly which fixes the reported crash.

Thanks Amul, with v20 patches, crash is fixed.

Show quoted text

Regards,
Amul

On Thu, Mar 7, 2019 at 11:54 AM Rajkumar Raghuwanshi <
rajkumar.raghuwanshi@enterprisedb.com> wrote:

On Tue, Mar 5, 2019 at 3:45 PM amul sul <sulamul@gmail.com> wrote:

Attached is the rebased atop of the latest master head(35bc0ec7c8).

thanks Amul, patches applied cleanly on PG head.

While testing this I got a server crash with below test case.

CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN
('0001','0002','0003');
CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN
('0004','0005','0006');
CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN
(NULL,'0008','0009');
CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
INSERT INTO plt1 SELECT i, i % 47, to_char(i % 17, 'FM0000') FROM
generate_series(0, 500) i WHERE i % 17 NOT IN (7, 11, 12, 13, 14, 15,16);
INSERT INTO plt1 SELECT i, i % 47, case when i % 17 = 7 then NULL else
to_char(i % 17, 'FM0000') end FROM generate_series(0, 500) i WHERE i % 17
IN (7,8,9);
ANALYSE plt1;

CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN
('0004','0005','0006');
CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN
('0007','0008','0009');
CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN
('0000',NULL,'0012');
INSERT INTO plt2 SELECT i, i % 47, to_char(i % 17, 'FM0000') FROM
generate_series(0, 500) i WHERE i % 17 NOT IN (1, 10, 11, 13, 14, 15, 16);
INSERT INTO plt2 SELECT i, i % 47, case when i % 17 = 11 then NULL else
to_char(i % 17, 'FM0000') end FROM generate_series(0, 500) i WHERE i % 17
IN (0,11,12);
ANALYZE plt2;

CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c,
'A'));
CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002',
'0003');
CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004',
'0005', '0006');
CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008',
'0009');
CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
INSERT INTO plt1_e SELECT i, i % 47, to_char(i % 17, 'FM0000') FROM
generate_series(0, 500) i WHERE i % 17 NOT IN (1, 7, 10, 11, 12, 13, 14,
15, 16);
ANALYZE plt1_e;

EXPLAIN (COSTS OFF)
SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM
plt1 t1, plt2 t2, plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') =
t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
The connection to the server was lost. Attempting reset: Failed.

below is stack trace, looks like some indexes got corrupted, please
take a look.

Core was generated by `postgres: edb postgres [local]
EXPLAIN '.
Program terminated with signal 11, Segmentation fault.
#0 0x0000000000821aee in map_and_merge_partitions (partmaps1=0x2c1c8a8,
partmaps2=0x2c1c8e0, index1=45540240, index2=0, next_index=0x7ffeebd43d3c)
at partbounds.c:4162
4162 if (partmap1->from < 0 && partmap2->from < 0)
Missing separate debuginfos, use: debuginfo-install
keyutils-libs-1.4-5.el6.x86_64 krb5-libs-1.10.3-65.el6.x86_64
libcom_err-1.41.12-24.el6.x86_64 libselinux-2.0.94-7.el6.x86_64
openssl-1.0.1e-57.el6.x86_64 zlib-1.2.3-29.el6.x86_64
(gdb) bt
#0 0x0000000000821aee in map_and_merge_partitions (partmaps1=0x2c1c8a8,
partmaps2=0x2c1c8e0, *index1=45540240*, index2=0,
next_index=0x7ffeebd43d3c) at partbounds.c:4162
#1 0x00000000008226c3 in merge_null_partitions (outer_bi=0x2b6e338,
inner_bi=0x2bf90b0, outer_maps=0x2c1c8a8, inner_maps=0x2c1c8e0,
jointype=JOIN_INNER,
next_index=0x7ffeebd43d3c, null_index=0x7ffeebd43d38,
default_index=0x7ffeebd43d34) at partbounds.c:4610
#2 0x0000000000821726 in partition_list_bounds_merge
(partsupfunc=0x2ba3548, partcollation=0x2ba34e8, outer_rel=0x2b6ce40,
inner_rel=0x2bf8d28, outer_parts=0x7ffeebd43ed8,
inner_parts=0x7ffeebd43ed0, jointype=JOIN_INNER) at partbounds.c:4031
#3 0x000000000081ff5d in partition_bounds_merge (partnatts=1,
partsupfunc=0x2ba3548, partcollation=0x2ba34e8, outer_rel=0x2b6ce40,
inner_rel=0x2bf8d28, jointype=JOIN_INNER,
outer_parts=0x7ffeebd43ed8, inner_parts=0x7ffeebd43ed0) at
partbounds.c:3053
#4 0x00000000007c610f in try_partitionwise_join (root=0x2be2a28,
rel1=0x2b6ce40, rel2=0x2bf8d28, joinrel=0x2c1b0f0,
parent_sjinfo=0x7ffeebd44010,
parent_restrictlist=0x2c1c070) at joinrels.c:1370
#5 0x00000000007c5521 in populate_joinrel_with_paths (root=0x2be2a28,
rel1=0x2b6ce40, rel2=0x2bf8d28, joinrel=0x2c1b0f0, sjinfo=0x7ffeebd44010,
restrictlist=0x2c1c070)
at joinrels.c:914
#6 0x00000000007c4f48 in make_join_rel (root=0x2be2a28, rel1=0x2b6ce40,
rel2=0x2bf8d28) at joinrels.c:748
#7 0x00000000007c4514 in make_rels_by_clause_joins (root=0x2be2a28,
old_rel=0x2b6ce40, other_rels=0x2bae4d8) at joinrels.c:294
#8 0x00000000007c41c8 in join_search_one_level (root=0x2be2a28,
level=3) at joinrels.c:116
#9 0x00000000007abe59 in standard_join_search (root=0x2be2a28,
levels_needed=3, initial_rels=0x2bae500) at allpaths.c:2716
#10 0x00000000007abdca in make_rel_from_joinlist (root=0x2be2a28,
joinlist=0x2bfbae8) at allpaths.c:2647
#11 0x00000000007a86b0 in make_one_rel (root=0x2be2a28,
joinlist=0x2bfbae8) at allpaths.c:227
#12 0x00000000007dada1 in query_planner (root=0x2be2a28,
tlist=0x2ba01c8, qp_callback=0x7e0b25 <standard_qp_callback>,
qp_extra=0x7ffeebd44390) at planmain.c:265
#13 0x00000000007ddf83 in grouping_planner (root=0x2be2a28,
inheritance_update=false, tuple_fraction=0) at planner.c:1929
#14 0x00000000007dc5f5 in subquery_planner (glob=0x2be2990,
parse=0x2c0e8c8, parent_root=0x0, hasRecursion=false, tuple_fraction=0) at
planner.c:997
#15 0x00000000007db1b6 in standard_planner (parse=0x2c0e8c8,
cursorOptions=256, boundParams=0x0) at planner.c:416
#16 0x00000000007daef7 in planner (parse=0x2c0e8c8, cursorOptions=256,
boundParams=0x0) at planner.c:276
#17 0x00000000008e15c5 in pg_plan_query (querytree=0x2c0e8c8,
cursorOptions=256, boundParams=0x0) at postgres.c:878
#18 0x00000000006562cc in ExplainOneQuery (query=0x2c0e8c8,
cursorOptions=256, into=0x0, es=0x2c0e0a0,
queryString=0x2aa24d8 "EXPLAIN (COSTS OFF)\nSELECT avg(t1.a),
avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM\nplt1 t1, plt2 t2,
plt1_e t3 WHERE t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c,
t2.c, t3.c ORDER BY t1"..., params=0x0, queryEnv=0x0) at explain.c:364

Thanks & Regards,
Rajkumar Raghuwanshi
QMG, EnterpriseDB.

Regards,
Amul Sul

On Mon, Feb 4, 2019 at 11:05 AM amul sul <sulamul@gmail.com> wrote:

There are few whitespaces in 0002 patch that I have fixed in the
attached version.
Rest of the patches are untouched.

Ill continue my review and testing.

Regards,
Amul

On Thu, Jan 31, 2019 at 5:26 PM Etsuro Fujita <
fujita.etsuro@lab.ntt.co.jp> wrote:

(2019/01/22 21:38), Etsuro Fujita wrote:

Will continue to review.

I rebased the patch set against the latest HEAD. Attached is a new
version. I'll move this to the next CF, and continue to review it.

Best regards,
Etsuro Fujita

#68Ashutosh Bapat
ashutosh.bapat.oss@gmail.com
In reply to: amul sul (#66)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Thu, Mar 7, 2019 at 8:20 PM amul sul <sulamul@gmail.com> wrote:

On Thu, Mar 7, 2019 at 1:02 PM amul sul <sulamul@gmail.com> wrote:

Thanks Rajkumar,

I am looking into this.

The crash happens when none of the if-else branch of
handle_missing_partition()
evaluates and returns merged_index unassigned.

Let me explain, in Rajkumar 's test case, the join type is JOIN_INNER.
When
only outer rel has null partition, merge_null_partitions() function calls
handle_missing_partition() with missing_side_inner = false and
missing_side_outer = false

Both missing_side_ variables being false when the NULL partition is missing
on the inner side looks suspicious. I guess from the variable names that
the missing_side_inner should be true in this case.

argument value which fails to set merged_index.

In the attached patch, I tried to fix this case by setting merged_index
explicitly which fixes the reported crash.

I expect handle_missing_partition() to set the merged_index always. In your
patches, I don't see that function in your patches is setting it
explicitly. If we are setting merged_index explicitly somewhere else, other
places may miss that explicit assignment. So it's better to move it inside
this function.

--
Best Wishes,
Ashutosh Bapat

#69amul sul
sulamul@gmail.com
In reply to: Ashutosh Bapat (#68)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Mon, Mar 11, 2019 at 8:29 AM Ashutosh Bapat <ashutosh.bapat.oss@gmail.com>
wrote:

On Thu, Mar 7, 2019 at 8:20 PM amul sul <sulamul@gmail.com> wrote:

On Thu, Mar 7, 2019 at 1:02 PM amul sul <sulamul@gmail.com> wrote:

Thanks Rajkumar,

I am looking into this.

The crash happens when none of the if-else branch of
handle_missing_partition()
evaluates and returns merged_index unassigned.

Let me explain, in Rajkumar 's test case, the join type is JOIN_INNER.
When
only outer rel has null partition, merge_null_partitions() function calls
handle_missing_partition() with missing_side_inner = false and
missing_side_outer = false

Both missing_side_ variables being false when the NULL partition is
missing on the inner side looks suspicious. I guess from the variable names
that the missing_side_inner should be true in this case.

All the places from where this handle_missing_partition() get called
have the following code to decide the value for missing_side_outer/_inner
which
I yet to understand. Do you think this has some flaw?

/*
* For a FULL join, inner relation acts as both OUTER and INNER
* relation. For LEFT and ANTI join the inner relation acts as
* INNER relation. For INNER and SEMI join OUTER and INNER
* differentiation is immaterial.
*/
missing_side_inner = (jointype == JOIN_FULL ||
jointype == JOIN_LEFT ||
jointype == JOIN_ANTI);
missing_side_outer = (jointype == JOIN_FULL);

argument value which fails to set merged_index.

In the attached patch, I tried to fix this case by setting merged_index
explicitly which fixes the reported crash.

I expect handle_missing_partition() to set the merged_index always. In
your patches, I don't see that function in your patches is setting it
explicitly. If we are setting merged_index explicitly somewhere else, other
places may miss that explicit assignment. So it's better to move it inside
this function.

Ok, that can be fixed.

Similarly, I think merge_null_partitions should set null_index instead of
asserting when null partitions missing from both the side, make sense?

Regards,
Amul

#70Ashutosh Bapat
ashutosh.bapat.oss@gmail.com
In reply to: amul sul (#69)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Mon, Mar 11, 2019 at 10:40 AM amul sul <sulamul@gmail.com> wrote:

All the places from where this handle_missing_partition() get called
have the following code to decide the value for missing_side_outer/_inner
which
I yet to understand. Do you think this has some flaw?

/*
* For a FULL join, inner relation acts as both OUTER and INNER
* relation. For LEFT and ANTI join the inner relation acts as
* INNER relation. For INNER and SEMI join OUTER and INNER
* differentiation is immaterial.
*/
missing_side_inner = (jointype == JOIN_FULL ||
jointype == JOIN_LEFT ||
jointype == JOIN_ANTI);
missing_side_outer = (jointype == JOIN_FULL);

I was wrong, sorry. The comment says it all.

argument value which fails to set merged_index.

In the attached patch, I tried to fix this case by setting merged_index
explicitly which fixes the reported crash.

I expect handle_missing_partition() to set the merged_index always. In
your patches, I don't see that function in your patches is setting it
explicitly. If we are setting merged_index explicitly somewhere else, other
places may miss that explicit assignment. So it's better to move it inside
this function.

Ok, that can be fixed.

Similarly, I think merge_null_partitions should set null_index instead of
asserting when null partitions missing from both the side, make sense?

I think not. null_index, once set shouldn't change and hence does not
change with each pair of partitions being matched. So, it makes sense to
make sure that null_index remains invalid if none of the tables have null
partition.

--
Best Wishes,
Ashutosh Bapat

#71amul sul
sulamul@gmail.com
In reply to: Ashutosh Bapat (#70)
3 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Attached version is rebase atop of the latest master head(fdc7efcc30), also
incorporates the Ashutosh's suggestion, thanks.

Regards,
Amul

On Mon, Mar 11, 2019 at 10:14 PM Ashutosh Bapat <
ashutosh.bapat.oss@gmail.com> wrote:

Show quoted text

On Mon, Mar 11, 2019 at 10:40 AM amul sul <sulamul@gmail.com> wrote:

All the places from where this handle_missing_partition() get called
have the following code to decide the value for missing_side_outer/_inner
which
I yet to understand. Do you think this has some flaw?

/*
* For a FULL join, inner relation acts as both OUTER and INNER
* relation. For LEFT and ANTI join the inner relation acts as
* INNER relation. For INNER and SEMI join OUTER and INNER
* differentiation is immaterial.
*/
missing_side_inner = (jointype == JOIN_FULL ||
jointype == JOIN_LEFT ||
jointype == JOIN_ANTI);
missing_side_outer = (jointype == JOIN_FULL);

I was wrong, sorry. The comment says it all.

argument value which fails to set merged_index.

In the attached patch, I tried to fix this case by setting merged_index
explicitly which fixes the reported crash.

I expect handle_missing_partition() to set the merged_index always. In
your patches, I don't see that function in your patches is setting it
explicitly. If we are setting merged_index explicitly somewhere else, other
places may miss that explicit assignment. So it's better to move it inside
this function.

Ok, that can be fixed.

Similarly, I think merge_null_partitions should set null_index instead of
asserting when null partitions missing from both the side, make sense?

I think not. null_index, once set shouldn't change and hence does not
change with each pair of partitions being matched. So, it makes sense to
make sure that null_index remains invalid if none of the tables have null
partition.

--
Best Wishes,
Ashutosh Bapat

Attachments:

0002-Partition-wise-join-for-1-1-1-0-0-1-partition-matchi-v21.patchapplication/octet-stream; name=0002-Partition-wise-join-for-1-1-1-0-0-1-partition-matchi-v21.patchDownload
From 496cb36ab294044c4d545af7a53e31709e9d9a6a Mon Sep 17 00:00:00 2001
From: Amul Sul <amul.sul@enterprisedb.com>
Date: Thu, 18 Apr 2019 04:54:29 -0400
Subject: [PATCH 2/3] Partition-wise join for 1:1, 1:0, 0:1 partition matching.

Earlier version of partition-wise join implementation allowed
partition-wise join between two relations with exactly same partition
bounds. This commit allows partition-wise join to be applied under
following conditions

1. the partition bounds of joining relations are such that rows from
given partition on one side join can join with rows from maximum one
partition on the other side i.e. bounds of a given partition on one
side match/overlap with those of maximum one partition on the other
side. If the mapping happens to be m:n where m > 1 or n > 1, we have
to gang multiple partition relations together into a single relation.
This means that we have to add simple relations during join
processing, something which is not supported right now.  ALso, in such
a case, different pairs of joining relations can produce different
partition bounds for the same join relation, which again is not
supported right now.

2. For every partition on outer side that can contribute to the result
of an OUTER join, there exists at least one (taken along with item 1,
it means exactly one)  matching partition on the inner side. To
support partition-wise join when the inner matching partition doesn't
exist, we have to add a dummy base relation corresponding to the
non-existent inner partition. We don't have support to add base
relations during join processing.

3. For hash partitioned tables however, we support partition-wise join
only when the bounds exactly match. For hash partitioning it's unusual
to have missing partitions and hence generic partition matching is not
required.

With advanced partition matching, we won't be able to apply
partition-wise join when there are missing matching partitions. So for
a given N-way join, some sub-joins may not be partitioned, while the
overall N-way join is partitioned. We can not try partition-wise join
when one or both of the joining relations are not partitioned in one
of the pairs of joining relation. Skip partition-wise join for that
pair.

An example is A IJ B LJ C where B has an extra partition compared to A
and C. Join B LJ C requires dummy relation to join the extra partition
from B to a missing partition from C, and hence can not use
partition-wise join right now and hence BC is not partitioned.  The
extra partition in B gets eliminated in A IJ B and thus it can use
partition-wise join and is partitioned. Hence (AB)C can use
partition-wise join but not A(BC).

Not every pair of joining relation (including the one presented to
build_joinrel_partition_info()) for the same joinrel can use
partition-wise join or has both the relations partitioned. Hence we
calculate the partition bounds for the join relation in
try_partition_wise_join() instead of doing it here.

The partition bounds produced for the first pair that can use
partition-wise are saved in the RelOptInfo of the joinrel. Partition
bounds produced for subsequent pairs should match that produced by the
first pair.

Support for default partition in list partitioned tables
========================================================

When a list value is present in one of the joining relations and not
the other, and the other relation has default partition, match (join)
the partition containing that list value with the default partition.
If there are multiple matches, we can not proceed with the
partition-wise join similar to the case of matching non-default
partition. If the default partition happens to be on the outer side of
the join, the resulting join partition acts as a default partition as
it will contain all the values from the default partition. If
the partition containing the list value happens to be on the outer
side of the join, the resulting join partition is associated with the
list value, since no other partition key value from the default
partition makes it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join.  Such a
default partition may get joined to a partition from the inner side,
if inner side has a list value that is not present in the outer side.
But if such a join partner does not exist, we won't be able to use
partition-wise join since there is no (dummy) relation on the inner
side to join to.

Support for matching default partition in range partitioned tables
==================================================================

When a range is present in one of the joining relations and not the
other, and the other relation has default partition, match (join) the
partition corresponding range with the default partition. If two
ranges overlap but their ranges don't match exactly, the
non-overlapping portion from one range may join the previous
(non-overlapping portion near the lower bound, if exists), next
(non-overlapping portion near the upper bound, if exists) ranges from
the other relation or default partition from the other relation. This
causes multiple partitions on the other side to be joined with a
single partition on the first side; a case we don't support. Any range
from one side which doesn't overlap with any range on the other side,
may find its join partner in the default partition and the
corresponding range partition will need to be joined with the default
partition.  If the default partition happens to be on the outer side
of the join, the resulting join partition acts as a default partition
as it will contain all the values from the default partition. If the
non-partition corresponding to the range happens to be on the outer
side of the join, the resulting join partition is associated with that
range, since partition key values from the default partition outside
that range won't make it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join.  Such a
default partition may get joined to a non-default partition from the
inner side, if inner side has a range missing in the outer side.  But
if such a join partner does not exist, we won't be able to use
partition-wise join since there is no (dummy) relation on the inner
side to join to.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/optimizer/path/joinrels.c |  103 +-
 src/backend/optimizer/util/relnode.c  |   33 +-
 src/backend/partitioning/partbounds.c | 1682 ++++++++++++++++++++++++-
 src/include/partitioning/partbounds.h |    7 +
 4 files changed, 1775 insertions(+), 50 deletions(-)

diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 46623c33c6..5c433f495a 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1339,25 +1339,31 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	PartitionScheme part_scheme;
+	PartitionBoundInfo join_boundinfo;
+	List	   *parts1;
+	List	   *parts2;
+	ListCell   *lc1;
+	ListCell   *lc2;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1370,36 +1376,65 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	 */
 	Assert(joinrel->part_scheme == rel1->part_scheme &&
 		   joinrel->part_scheme == rel2->part_scheme);
+	part_scheme = joinrel->part_scheme;
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * Get the list of matching partitions to be joined along with the
+	 * partition bounds of the join relation. Because of the restrictions
+	 * imposed by partition matching algorithm, not every pair of joining
+	 * relations for this join will be able to use partition-wise join. But all
+	 * those pairs which can use partition-wise join will produce the same
+	 * partition bounds for the join relation.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	join_boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											part_scheme->partsupfunc,
+											part_scheme->partcollation,
+											rel1, rel2,
+											parent_sjinfo->jointype,
+											&parts1, &parts2);
+
+	if (join_boundinfo == NULL)
+		return;
 
-	nparts = joinrel->nparts;
+	if (joinrel->boundinfo == NULL)
+	{
+		Assert(joinrel->nparts == 0 && joinrel->part_rels == NULL);
+		joinrel->boundinfo = join_boundinfo;
+		joinrel->nparts = list_length(parts1);
+		Assert(joinrel->nparts == list_length(parts2));
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) *
+									joinrel->nparts);
+	}
+	else
+	{
+		Assert(partition_bounds_equal(part_scheme->partnatts,
+									  part_scheme->parttyplen,
+									  part_scheme->parttypbyval,
+									  join_boundinfo, joinrel->boundinfo));
+		/*
+		 * Every pair of joining relations should result in the same number
+		 * of child-joins.
+		 */
+		Assert(joinrel->nparts == list_length(parts1));
+		Assert(joinrel->nparts == list_length(parts2));
+		Assert(joinrel->part_rels);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	cnt_parts = -1;
+	forboth(lc1, parts1, lc2, parts2)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
-		bool		rel1_empty = (child_rel1 == NULL ||
-								  IS_DUMMY_REL(child_rel1));
-		bool		rel2_empty = (child_rel2 == NULL ||
-								  IS_DUMMY_REL(child_rel2));
+		int			part1 = lfirst_int(lc1);
+		int			part2 = lfirst_int(lc2);
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		bool		rel1_empty;
+		bool		rel2_empty;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1407,6 +1442,13 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		Assert(part1 >= 0 && part2 >= 0);
+		child_rel1 = rel1->part_rels[part1];
+		child_rel2 = rel2->part_rels[part2];
+		rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1));
+		rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2));
+		cnt_parts++;
+
 		/*
 		 * Check for cases where we can prove that this segment of the join
 		 * returns no rows, due to one or both inputs being empty (including
@@ -1506,12 +1548,23 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 			joinrel->part_rels[cnt_parts] = child_joinrel;
 		}
 
+		/*
+		 * For every pair of joining relations, the set of matching partitions
+		 * would change. However, the base relation partitions constituting
+		 * the given child should remain same for all the joining pairs. Since
+		 * the order in which children are stored in the array of child-joins,
+		 * depends upon partition bounds of the join, which are same for all
+		 * the joining pairs, every joining pair yields the child-joins in the
+		 * same order.
+		 */
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
 
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
 	}
+
+	Assert(cnt_parts == (joinrel->nparts - 1));
 }
 
 /*
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 272e2eb10a..af33511aab 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1623,7 +1623,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1636,24 +1636,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1665,17 +1647,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 35c03f611a..2086db720e 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -25,6 +25,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/pathnodes.h"
 #include "parser/parse_coerce.h"
 #include "partitioning/partbounds.h"
 #include "partitioning/partdesc.h"
@@ -69,6 +70,12 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+typedef struct PartitionMap
+{
+	int from;
+	int to;
+} PartitionMap;
+
 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);
@@ -110,7 +117,58 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 static List *get_range_nulltest(PartitionKey key);
 static bool partition_hbounds_equal(PartitionBoundInfo b1,
 									PartitionBoundInfo b2);
-
+static PartitionBoundInfo partition_range_bounds_merge(
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 List **outer_parts, List **inner_parts,
+							 JoinType jointype, int partnatts,
+							 FmgrInfo *supfuncs, Oid *collations);
+static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype);
+static PartitionBoundInfo partition_hash_bounds_merge(RelOptInfo *outer_rel,
+							RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype);
+static void generate_matching_part_pairs(PartitionMap *outer_maps,
+										 PartitionMap *inner_maps,
+										 int nparts1, int nparts2,
+										 JoinType jointype, int nparts,
+										 List **parts1, List **parts2);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int map_and_merge_partitions(PartitionMap *outer_maps,
+										PartitionMap *inner_maps,
+										int index1, int index2, int *next_index);
+static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *collations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2);
+static bool partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
+						   Oid *collations, PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, int *ub_cmpval,
+						   int *lb_cmpval);
+static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes);
+static bool merge_default_partitions(PartitionBoundInfo outer_bi,
+						 PartitionBoundInfo inner_bi,
+						 PartitionMap *outer_maps,
+						 PartitionMap *inner_maps,
+						 JoinType jointype,
+						 int *next_index, int *default_index);
+static bool merge_null_partitions(PartitionBoundInfo outer_bi,
+								  PartitionBoundInfo inner_bi,
+								  PartitionMap *outer_maps,
+								  PartitionMap *inner_maps,
+								  JoinType jointype,
+								  int *next_index, int *null_index,
+								  int *default_index);
 
 /*
  * get_qual_from_partbound
@@ -3007,3 +3065,1625 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * The function produces the partition bounds for a join between two relations
+ * whose partition bounds are given. The function also returns two lists of
+ * partition indexes one for each of the joining relations. Both the lists
+ * contain the same number of elements. The partition indexes at the same
+ * positions in the lists indicate the pair partitions, one from each side, to
+ * be joined and the position itself corresponds to the index of partition
+ * produced by that child-join in the partitioned join.
+ *
+ * The function returns NULL if we can not find the matching pair of
+ * partitions. This happens if 1. multiple partitions on one side match with
+ * one partition on the other side. 2. a given partition on the outer side
+ * doesn't have a matching partition on the inner side. We can not support the
+ * first case since we don't have a way to represent multiple partitions as a
+ * single relation (RelOptInfo) and then perform join using the ganged
+ * relation. We can not support the second case since the missing inner
+ * partition needs to be represented as an empty relation and we don't have a
+ * way to introduce empty relation during join planning after creating paths
+ * for all the base relations.
+ */
+extern PartitionBoundInfo
+partition_bounds_merge(int partnatts, FmgrInfo *partsupfunc,
+					   Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
+{
+	PartitionBoundInfo 	merged_bounds;
+	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
+						inner_binfo = inner_rel->boundinfo;
+	char				strategy = outer_binfo->strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partsupfunc,
+														partcollation,
+														outer_rel, inner_rel,
+														outer_parts, inner_parts,
+														jointype);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(outer_rel, inner_rel,
+														 outer_parts, inner_parts,
+														 jointype, partnatts,
+														 partsupfunc,
+														 partcollation);
+			break;
+
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = partition_hash_bounds_merge(outer_rel, inner_rel,
+														outer_parts, inner_parts,
+														jointype);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	Assert((*outer_parts == NIL || *inner_parts != NIL) &&
+		   (*inner_parts == NIL || *outer_parts != NIL));
+
+	return merged_bounds;
+}
+
+/*
+ * partition_get_range_bounds
+ *
+ * Given the index of lower bound in datums array, return lower and upper
+ * bounds and the index of the partition with that lower bound.
+ */
+static int
+partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
+						   PartitionRangeBound *lower,
+						   PartitionRangeBound *upper)
+{
+	int			part_index;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The lower bound should correspond to a valid partition. */
+	part_index = bi->indexes[lb_index + 1];
+	Assert(part_index >= 0);
+
+	lower->kind = bi->kind[lb_index];
+	lower->datums = bi->datums[lb_index];
+	lower->lower = true;
+	upper->kind = bi->kind[lb_index + 1];
+	upper->datums = bi->datums[lb_index + 1];
+	upper->lower = false;
+
+	return part_index;
+}
+
+/*
+ * partition_range_get_next_lb_index
+ *
+ * Given the index of lower bound in datums array return the
+ * index of lower bound of the next partition. When the given index corresponds
+ * to the last partition, return number of datums (ndatums).
+ */
+static int
+partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
+{
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The partition index corresponding to the upper bound should be valid. */
+	Assert(bi->indexes[lb_index + 1] >= 0);
+
+	/*
+	 * If there are no bounds left beyond the upper bound, we have reached the
+	 * last partition.
+	 */
+	if (lb_index + 2 < bi->ndatums)
+	{
+		/*
+		 * If the bound next to the upper bound corresponds to no partition,
+		 * that's the next lower bound of the next partition. Otherwise, the
+		 * current upper bound is the lower bound of the next partition.
+		 */
+		if (bi->indexes[lb_index + 2] < 0)
+			return lb_index + 2;
+		else
+			return lb_index + 1;
+	}
+	else
+		return bi->ndatums;
+}
+
+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *partcollations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2)
+{
+	return partition_rbound_cmp(partnatts, partsupfunc, partcollations,
+								bound1->datums, bound1->kind, bound1->lower,
+								bound2);
+}
+
+/*
+ * partition_range_cmp
+ *
+ * Compare the bounds of two range partitions. Set ub_cmpval <, = or > 0, if the
+ * first partition's upper bound is lower than, equal to or higher than the
+ * second partition's upper bound resp. Similarly set lb_cmpval <, =  or > 0,
+ * if the first partition's lower bound is lower than, equal to or higher than
+ * the second partition's lower bound resp.
+ *
+ * Return true, if the ranges overlap, otherwise return false.
+ */
+static bool
+partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
+					PartitionRangeBound *lower_bound1,
+					PartitionRangeBound *upper_bound1,
+					PartitionRangeBound *lower_bound2,
+					PartitionRangeBound *upper_bound2, int *ub_cmpval,
+					int *lb_cmpval)
+{
+	bool		overlap;
+
+	/*
+	 * Compare upper bound of the first partition with the lower bound of the
+	 * second and vice-versa. If lower bound is higher than the upper bound,
+	 * the partitions are not overlapping. All other cases indicate overlapping
+	 * partitions.
+	 */
+	if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+								  lower_bound1, upper_bound2) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = 1;
+		*lb_cmpval = 1;
+	}
+	else if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+									   lower_bound2, upper_bound1) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = -1;
+		*lb_cmpval = -1;
+	}
+	else
+	{
+		overlap = true;
+		*ub_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, upper_bound1,
+											   upper_bound2);
+		*lb_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, lower_bound1,
+											   lower_bound2);
+	}
+
+	return overlap;
+}
+
+/*
+ * partition_range_merge
+ *
+ * Merge the partition bounds of given two partitions such that the join
+ * between the given two partitions fits merged bounds.
+ *
+ * "merged_upper" will be set to one of the given upper bounds and
+ * "merged_lower" will be set to one of the given lower bounds.
+ */
+static void
+partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
+					  Oid *partcollations, JoinType jointype,
+					  PartitionRangeBound *left_lb,
+					  PartitionRangeBound *left_ub,
+					  PartitionRangeBound *right_lb,
+					  PartitionRangeBound *right_ub,
+					  PartitionRangeBound **merged_lb,
+					  PartitionRangeBound **merged_ub)
+{
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = left_ub;
+			*merged_lb = left_lb;
+			break;
+
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) < 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) > 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		case JOIN_FULL:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) > 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) < 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+
+	return;
+}
+
+/*
+ * Add the lower bound of the next range to the list of bounds, if the lower
+ * bound is higher or equal to the previous upper bound. If successful return
+ * true, otherwise false.
+ */
+static bool
+partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
+							  Oid *partcollations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		prev_ub.datums = llast(*merged_datums);
+		prev_ub.kind = llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  next_lb_datums, next_lb_kind, false,
+									  &prev_ub);
+	}
+
+	/*
+	 * The lower bound is lower than the last upper bound, thus does not fit
+	 * the bounds created so far and hence can not be merged with the existing
+	 * bounds.
+	 */
+	if (cmpval < 0)
+		return false;
+
+	/*
+	 * Add bounds of the new merged partition. If the next lower bound is
+	 * higher than the last upper bound, add new range with index
+	 * corresponding to the lower bound as -1. If the merged lower bound
+	 * is same as the last merged upper bound, the last upper bound will be
+	 * reused as the lower bound of the next range.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, next_lb_datums);
+		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	return true;
+}
+
+/*
+ * handle_missing_partition
+ *
+ * If a range appears in one of the joining relations but not the other, a row
+ * in the corresponding partition will not have any join partner in the other
+ * relation, unless the other relation has a default partition. If a given list
+ * value is present in one joining relation but not the other, the default
+ * partition on the other side may contain that value.
+ *
+ * In both these cases, such an extra partition forms a joining pair with the
+ * default partition, if any,  on the other side.
+ *
+ * If the default partition happens to be on the outer side of the join, the
+ * resultant partition will act as the default partition of the join relation.
+ * Otherwise the resultant partition will be associated with the range.
+ *
+ * When the default partition is not present in the other relation, the rows in
+ * the extra partition will be included in the bounds of the join result, if it
+ * appears on the outer side of the join, since all rows from the outer side
+ * are included in the join result.
+ *
+ * This function handles all these cases.
+ *
+ * maps_with_missing and missing_side_default are the partition maps (See
+ * partition_range/list_bounds_merge() for details) and the index of default
+ * partition respectively corresponding the side with missing partition.
+ *
+ * maps_with_extra and extra_part are the partition maps (See
+ * partition_range/list_bounds_merge() for details) and the index of extra
+ * partition respectively corresponding to the side with the extra partition.
+ *
+ * It returns true if the matching succeeds, otherwise returns false.
+ */
+static bool
+handle_missing_partition(PartitionMap *maps_with_missing,
+						 PartitionMap *maps_with_extra,
+						 int missing_side_default,
+						 int extra_part,
+						 bool missing_side_outer,
+						 bool missing_side_inner,
+						 int *next_index, int *default_index,
+						 int *merged_index)
+{
+	bool missing_has_default = (missing_side_default != -1);
+
+	if (missing_has_default)
+	{
+		*merged_index = map_and_merge_partitions(maps_with_missing,
+												 maps_with_extra,
+												 missing_side_default,
+												 extra_part,
+												 next_index);
+		if (*merged_index < 0)
+			return false;
+
+		if (missing_side_outer)
+		{
+			/*
+			 * Default partition on the outer side forms the default
+			 * partition of the join result.
+			 */
+			if (*default_index < 0)
+				*default_index = *merged_index;
+			else if(*default_index != *merged_index)
+			{
+				/*
+				 * Ended up with default partition on the outer side
+				 * being joined with multiple partitions on the inner
+				 * side. We don't support this case.
+				 */
+				return false;
+			}
+
+			/*
+			 * Since the merged partition acts as a default partition, it
+			 * doesn't need a separate index.
+			 */
+			*merged_index = -1;
+		}
+	}
+	else if (missing_side_inner)
+	{
+		/*
+		 * If this partition has already been mapped (say because we
+		 * found an overlapping range earlier), we know where does it
+		 * fit in the join result. Nothing to do in that case. Else
+		 * create a new merged partition.
+		 */
+		PartitionMap *extra_map = &maps_with_extra[extra_part];
+		if (extra_map->to < 0)
+		{
+			extra_map->to = *next_index;
+			*next_index = *next_index + 1;
+			*merged_index = extra_map->to;
+		}
+	}
+	else
+		*merged_index = -1;
+
+	return true;
+}
+
+static PartitionMap*
+init_partition_map(RelOptInfo *rel)
+{
+	int i, nparts = rel->nparts;
+	PartitionMap *map;
+
+	map = (PartitionMap *) palloc(sizeof(PartitionMap) * nparts);
+
+	for (i = 0; i < nparts; i++)
+	{
+		map[i].from = -1;
+		map[i].to = -1;
+	}
+
+	return map;
+}
+
+/*
+ * Allocate and initialize partition maps. We maintain four maps, two maps
+ * for each joining relation. pmap[i] gives the partition from the other
+ * relation which would join with ith partition of the given relation.
+ * Partition i from the given relation will join with partition pmap[i]
+ * from the other relation to produce partition mmap[i] of the join (merged
+ * partition).
+ *
+ * pmap[i] = -1 indicates that ith partition of a given relation does not
+ * have a matching partition from the other relation.
+ *
+ * mmap[i] = -1 indicates that ith partition of a given relation does not
+ * contribute to the join result. That can happen only when the given
+ * relation is the inner relation and it doesn't have a matching partition
+ * from the outer relation, hence pmap[i] should be -1.
+ *
+ * In case of an outer join, every partition of the outer join will appear
+ * in the join result, and thus has mmap[i] set for all i. But it's not
+ * necessary that every partition on the outer side will have a matching
+ * partition on the inner side. In such a case, we end up with pmap[i] = -1
+ * and mmap[i] != -1.
+ */
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 List **outer_parts, List **inner_parts,
+							 JoinType jointype, int partnatts,
+							 FmgrInfo *partsupfuncs, Oid *partcollations)
+
+{
+	PartitionMap *outer_maps = NULL;
+	PartitionMap *inner_maps = NULL;
+	int			outer_part = 0;
+	int			inner_part = 0;
+	PartitionBoundInfo merged_bounds = NULL;
+	int			outer_lb_index;
+	int			inner_lb_index;
+	int			next_index;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	List	   *merged_kinds = NIL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			inner_default = inner_bi->default_index;
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	outer_maps = init_partition_map(outer_rel);
+	inner_maps = init_partition_map(inner_rel);
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = 0;
+	inner_lb_index = 0;
+	next_index = 0;
+	while (outer_lb_index < outer_bi->ndatums ||
+		   inner_lb_index < inner_bi->ndatums)
+	{
+		PartitionRangeBound outer_lb, outer_ub,
+							inner_lb, inner_ub,
+							*merged_lb = NULL,
+							*merged_ub = NULL;
+
+		int			merged_index = -1;
+		bool		overlap;
+		bool		finished_outer = false;
+		bool		finished_inner = false;
+
+		/* Result of bounds comparison per partition_rbound_cmp(). */
+		int			ub_cmpval;	/* Upper bounds comparison result. */
+		int			lb_cmpval;	/* Lower bounds comparison result. */
+
+		/* Get the range bounds of the next pair of partitions. */
+		if (outer_lb_index < outer_bi->ndatums)
+			outer_part = partition_get_range_bounds(outer_bi, outer_lb_index,
+												&outer_lb, &outer_ub);
+		else
+			finished_outer = true;
+
+		if (inner_lb_index < inner_bi->ndatums)
+			inner_part = partition_get_range_bounds(inner_bi, inner_lb_index,
+												&inner_lb, &inner_ub);
+		else
+			finished_inner = true;
+
+		Assert(!finished_outer || !finished_inner);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (finished_outer)
+		{
+			overlap = false;
+			ub_cmpval = 1;
+			lb_cmpval = 1;
+		}
+		else if (finished_inner)
+		{
+			overlap = false;
+			ub_cmpval = -1;
+			lb_cmpval = -1;
+		}
+		else
+			overlap = partition_range_cmp(partnatts, partsupfuncs, partcollations,
+										  &outer_lb, &outer_ub, &inner_lb,
+										  &inner_ub, &ub_cmpval, &lb_cmpval);
+
+		if (overlap)
+		{
+			/*
+			 * The rows from overlapping portion of ranges on both sides may
+			 * join, hence the corresponding pair of partitions form a joining
+			 * pair. Match them and produce the bounds of the joint partition
+			 * and its index by merging the bounds according to the type of
+			 * join.
+			 */
+			partition_range_merge(partnatts, partsupfuncs, partcollations,
+								  jointype, &outer_lb, &outer_ub, &inner_lb,
+								  &inner_ub, &merged_lb, &merged_ub);
+
+			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+													outer_part, inner_part,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				/* Failed to match the partitions. */
+				return NULL;
+			}
+
+			/*
+			 * If the ranges overlap but don't exactly match, a row from
+			 * non-overlapping portion of the range from one side of join may
+			 * find its join partner in the previous or next overlapping
+			 * partition or default partition on the other side , if such a
+			 * partition exists. All those cases, if true, will cause one
+			 * partition from that side to match at least two partitions on the
+			 * other side; a case that we do not support now. Previous
+			 * partition has been delt with in the previous iteration of this
+			 * loop, next partition will be delt in the next iteration. We will
+			 * deal with the default partition here.
+			 */
+			if ((lb_cmpval < 0 && inner_has_default) ||
+				/* Non-overlapping range on the lower side of outer range. */
+				(lb_cmpval > 0 && outer_has_default) ||
+				/* Non-overlapping range on the lower side of inner range. */
+				(ub_cmpval < 0 && outer_has_default) ||
+				/* Non-overlapping range on the upper side of inner range. */
+				(ub_cmpval > 0 && inner_has_default))
+				/* Non-overlapping range on the upper side of outer range. */
+				return NULL;
+		}
+
+		if (ub_cmpval == 0)
+		{
+			/* Upper bounds of both the ranges match. */
+			Assert(overlap);
+
+			/* Move to the next pair of partitions. */
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the inner side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &outer_lb;
+				merged_ub = &outer_ub;
+
+				/*
+				 * For a FULL join, inner relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the inner relation acts as
+				 * INNER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_inner = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_outer = (jointype == JOIN_FULL);
+				if (!handle_missing_partition(inner_maps,
+											  outer_maps,
+											  inner_default,
+											  outer_part,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the outer side. */
+			Assert(!finished_outer);
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+		}
+		else
+		{
+			Assert(ub_cmpval > 0);
+
+			/* Upper bound of outer range higher than that of the inner. */
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the outer side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &inner_lb;
+				merged_ub = &inner_ub;
+
+				/*
+				 * For a FULL join, outer relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the outer relation acts as
+				 * OUTER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_outer = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_inner = (jointype == JOIN_FULL);
+
+				if (!handle_missing_partition(outer_maps,
+											  inner_maps,
+											  outer_default,
+											  inner_part,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the inner side. */
+			Assert (!finished_inner);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+
+		if (merged_index < 0)
+		{
+			/* We didn't find a new merged partition. */
+			continue;
+		}
+
+		/*
+		 * We have a valid partition index for the next partition of join. The
+		 * partition should have valid range.
+		 */
+		Assert(merged_lb && merged_ub);
+
+		/* Try merging new lower bound with the last upper bound. */
+		if (!partition_range_merge_next_lb(partnatts, partsupfuncs,
+										   partcollations,
+										   merged_lb->datums,
+										   merged_lb->kind, &merged_datums,
+										   &merged_kinds, &merged_indexes))
+			return NULL;
+
+		/* Add upper bound with the merged partition index. */
+		merged_datums = lappend(merged_datums, merged_ub->datums);
+		merged_kinds = lappend(merged_kinds, merged_ub->kind);
+		merged_indexes = lappend_int(merged_indexes, merged_index);
+	}
+
+	if (!merge_default_partitions(outer_bi, inner_bi,
+										  outer_maps, inner_maps,
+										  jointype, &next_index,
+										  &default_index))
+		return NULL;
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_maps, inner_maps,
+								 outer_nparts, inner_nparts,
+								 jointype, next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes,
+													  merged_kinds,
+													  -1, default_index);
+	}
+
+	/* Free any memory we used in this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype)
+{
+	PartitionMap *outer_maps = NULL;
+	PartitionMap *inner_maps = NULL;
+	int			cnto;
+	int			cnti;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			next_index = 0;
+	int			null_index = -1;
+	int			default_index = -1;
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			      *outer_indexes = outer_bi->indexes;
+	int			      *inner_indexes = inner_bi->indexes;
+	int				   outer_default = outer_bi->default_index;
+	int				   inner_default = inner_bi->default_index;
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	outer_maps = init_partition_map(outer_rel);
+	inner_maps = init_partition_map(inner_rel);
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			o_index;
+		int			i_index;
+		int			cmpval;
+		int			merged_index = -1;
+		Datum	   *merged_datum;
+		bool		finished_inner;
+		bool		finished_outer;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+		{
+			finished_outer = true;
+			odatums = NULL;
+			o_index = -1;
+		}
+		else
+		{
+			finished_outer = false;
+			odatums = outer_bi->datums[cnto];
+			o_index = outer_indexes[cnto];
+		}
+
+		if (cnti >= inner_bi->ndatums)
+		{
+			finished_inner = true;
+			idatums = NULL;
+			i_index = -1;
+		}
+		else
+		{
+			finished_inner = false;
+			idatums = inner_bi->datums[cnti];
+			i_index = inner_indexes[cnti];
+		}
+
+		/* If we exhausted both the sides, we won't enter the loop. */
+		Assert(!finished_inner || !finished_outer);
+
+		if (finished_outer)
+			cmpval = 1;
+		else if (finished_inner)
+			cmpval = -1;
+		else
+		{
+			/* Every list datum should map to a valid partition index. */
+			Assert(o_index >= 0 && i_index >= 0 &&
+				   odatums != NULL && idatums != NULL);
+
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0], idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+													o_index, i_index,
+													&next_index);
+
+			if (merged_index < 0)
+				return NULL;
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			/* A datum missing from the inner side. */
+			merged_index = -1;
+			merged_datum = odatums;
+
+			/*
+			 * For a FULL join, inner relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the inner relation acts as
+			 * INNER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_inner = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_outer = (jointype == JOIN_FULL);
+
+			if (!handle_missing_partition(inner_maps,
+										  outer_maps,
+										  inner_default,
+										  o_index,
+										  missing_side_outer,
+										  missing_side_inner,
+										  &next_index,
+										  &default_index,
+										  &merged_index))
+				return NULL;
+
+			/* Move to the next datum on the outer side. */
+			Assert(!finished_outer);
+			cnto++;
+		}
+		else
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			Assert(cmpval > 0);
+
+			/* A datum missing from the outer side. */
+			merged_index = -1;
+			merged_datum = idatums;
+
+			/*
+			 * For a FULL join, outer relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the outer relation acts as
+			 * OUTER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_outer = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_inner = (jointype == JOIN_FULL);
+
+			if (!handle_missing_partition(outer_maps,
+										  inner_maps,
+										  outer_default,
+										  i_index,
+										  missing_side_outer,
+										  missing_side_inner,
+										  &next_index,
+										  &default_index,
+										  &merged_index))
+				return NULL;
+
+			/* Move to the next datum on the right side. */
+			Assert(!finished_inner);
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	if (!merge_null_partitions(outer_bi, inner_bi,
+							   outer_maps, inner_maps,
+							   jointype, &next_index, &null_index,
+							   &default_index))
+		return NULL;
+
+	if (!merge_default_partitions(outer_bi, inner_bi,
+								  outer_maps, inner_maps,
+								  jointype, &next_index,
+								  &default_index))
+		return NULL;
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_maps, inner_maps,
+								 outer_nparts, inner_nparts,
+								 jointype, next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes, NIL,
+													  null_index, default_index);
+	}
+
+	/* Free up all extra memory before returning from this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_bounds_merge()'s arm for hash partitioned tables.
+ *
+ * If the given two hash bounds are same, the function returns the first one
+ * without any change, alongwith the lists of matching partitions. Otherwise it
+ * returns NULL.
+ *
+ * We could try merging the bounds when both the bounds have same greatest
+ * modulii. But there seems to be hardly any requirement for the same.
+ */
+static PartitionBoundInfo
+partition_hash_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype)
+{
+	int			nparts;
+	int			cnt;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * Hash partitioned table does not have explicit NULL accepting partition
+	 * and also does not have a default partition.
+	 */
+	Assert(!partition_bound_has_default(outer_bi) &&
+		   !partition_bound_has_default(inner_bi));
+	Assert(!partition_bound_accepts_nulls(outer_bi) &&
+		   !partition_bound_accepts_nulls(inner_bi));
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+
+	if (outer_nparts != inner_nparts)
+		return NULL;
+	nparts = outer_nparts;
+
+	if (outer_bi->ndatums != inner_bi->ndatums ||
+		!partition_hbounds_equal(outer_bi, inner_bi))
+		return NULL;
+
+	 /*
+	  * Cook up list of matching partitions. Since bounds are exactly same the
+	  * partitions at the same position from both the relations match.
+	  */
+	for (cnt = 0; cnt < nparts; cnt++)
+	{
+		*outer_parts = lappend_int(*outer_parts, cnt);
+		*inner_parts = lappend_int(*inner_parts, cnt);
+	}
+
+	return outer_bi;
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * If the two given partitions (given by index1 and index2 resp.) are
+ * already mapped to each other return the index of corresponding partition in
+ * the merged set of partitions.  If they do not have a merged partition
+ * associated with them, assign a new merged partition index.  If the
+ * partitions are already mapped and their mapped partitions are different from
+ * each other, they can not be merged, so return -1.
+ *
+ * partmaps1[i] gives the mapping of partitions for both relations. It
+ * describes which partition of relation 2 matches ith partition of relation 1,
+ * and which partition in the merged set matches ith partition of relation 1
+ * maps to. Similarly for partmap2.
+ *
+ * index1 and index2 are the indexes of matching partition from respective
+ * relations.
+ *
+ * *next_index is used and incremented when the given partitions require a new
+ * merged partition.
+ */
+
+static int
+map_and_merge_partitions(PartitionMap *partmaps1, PartitionMap *partmaps2,
+						 int index1, int index2, int *next_index)
+{
+	PartitionMap 	*partmap1 = &partmaps1[index1];
+	PartitionMap 	*partmap2 = &partmaps2[index2];
+	int				merged_index;
+
+	/*
+	 * If both the partitions are not mapped to each other, update the
+	 * maps.
+	 */
+	if (partmap1->from < 0 && partmap2->from < 0)
+	{
+		partmap1->from = index2;
+		partmap2->from = index1;
+	}
+
+	/*
+	 * If the given to partitions map to each other, find the corresponding
+	 * merged partition index .
+	 */
+	if (partmap1->from == index2 && partmap2->from == index1)
+	{
+		/*
+		 * If both the partitions are mapped to the same merged partition, get
+		 * the index of merged partition.
+		 */
+		if (partmap1->to == partmap2->to)
+		{
+			merged_index = partmap1->to;
+
+			/*
+			 * If the given two partitions do not have a merged partition
+			 * associated with them, allocate a new merged partition.
+			 */
+			if (merged_index < 0)
+			{
+				merged_index = *next_index;
+				*next_index = *next_index + 1;
+				partmap1->to = merged_index;
+				partmap2->to = merged_index;
+			}
+		}
+
+		/*
+		 * If partition from one relation was mapped to a merged partition but
+		 * not the partition from the other relation, map the same merged
+		 * partition to the partition from other relation, since matching
+		 * partitions map to the same merged partition.
+		 */
+		else if (partmap1->to >= 0 && partmap2->to < 0)
+		{
+			partmap2->to = partmap1->to;
+			merged_index = partmap1->to;
+		}
+		else if (partmap1->to < 0 && partmap2->to >= 0)
+		{
+			partmap1->to = partmap2->to;
+			merged_index = partmap2->to;
+		}
+		else
+		{
+			Assert(partmap1->to != partmap2->to &&
+				   partmap1->to >= 0 && partmap2->to >= 0);
+
+			/*
+			 * Both the partitions map to different merged partitions. This
+			 * means that multiple partitions from one relation matches to one
+			 * partition from the other relation. Partition-wise join does not
+			 * handle this case right now, since it requires ganging multiple
+			 * partitions together (into one RelOptInfo).
+			 */
+			merged_index = -1;
+		}
+	}
+	else
+	{
+		/*
+		 * Multiple partitions from one relation map to one partition from the
+		 * other relation. Partition-wise join does not handle this case right
+		 * now, since it requires ganging multiple partitions together (into
+		 * one RelOptInfo).
+		 */
+		merged_index = -1;
+	}
+
+	return merged_index;
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * partmaps1 map each partition from either side of the join to a merged
+ * partition resp. E.g. partmaps1[i].to gives the merged partition to which ith
+ * partition of first relation maps. Similarly for partmap2. If
+ * partmaps1[i].to == partmaps2[j].to, i and j form the matching pair of
+ * partitions.
+ *
+ * Given these maps this function produces the list pairs of partitions which
+ * when joined produce the merged partitions in the order of merged partition
+ * indexes.
+ *
+ * nparts1 and nparts2 are the number of partitions of the joining relations
+ * resp.
+ *
+ * nparts is the number of merged partitions.
+ *
+ * If successful, the pairs of partitions are returned as two separate lists,
+ * parts1 and parts2 resp., one for each side. Otherwise, those lists will be
+ * set to NIL.
+ */
+static void
+generate_matching_part_pairs(PartitionMap *partmaps1, PartitionMap *partmaps2,
+							 int nparts1, int nparts2,
+							 JoinType jointype, int nparts,
+							 List **matched_parts1, List **matched_parts2)
+{
+	bool		merged = true;
+	int		   *matching1,
+			   *matching2;
+	int 		i;
+	int			max_nparts;
+
+	*matched_parts1 = NIL;
+	*matched_parts2 = NIL;
+
+	matching1 = (int *) palloc(sizeof(int) * nparts),
+	matching2 = (int *) palloc(sizeof(int) * nparts);
+	for (i = 0; i < nparts; i++)
+		matching1[i] = matching2[i] = -1;
+
+	/* Set pairs of matching partitions. */
+	max_nparts = Max(nparts1, nparts2);
+	for (i = 0; i < max_nparts; i++)
+	{
+		if (i < nparts1)
+		{
+			PartitionMap outer_map = partmaps1[i];
+
+			if (outer_map.to >= 0)
+			{
+				Assert(outer_map.to < nparts);
+				matching1[outer_map.to] = i;
+			}
+		}
+
+		if (i < nparts2)
+		{
+			PartitionMap inner_map = partmaps2[i];
+
+			if (inner_map.to >= 0)
+			{
+				Assert(inner_map.to < nparts);
+				matching2[inner_map.to] = i;
+			}
+		}
+	}
+
+	/*
+	 * If we have a partition missing on an inner side, we need to add a dummy
+	 * relation which joins with the outer partition. If the inner relation
+	 * happens to be a base relation, it will require adding a dummy child
+	 * base relation during join processing. Right now, we freeze the base
+	 * relation arrays like PlannerInfo::simple_rte_array after planning for
+	 * base relations. Adding a new (dummy) base relation would require some
+	 * changes to that. So, right now, we do not implement partition-wise join
+	 * in such cases.
+	 */
+	for (i = 0; i < nparts; i++)
+	{
+		int			part1 = matching1[i];
+		int			part2 = matching2[i];
+
+		/* At least one of the partitions should exist. */
+		Assert(part1 >= 0 || part2 >= 0);
+
+		switch (jointype)
+		{
+			case JOIN_INNER:
+			case JOIN_SEMI:
+
+				/*
+				 * An inner or semi join can not return any row when the
+				 * matching partition on either side is missing. We should
+				 * have eliminated all such cases while merging the bounds.
+				 */
+				Assert(part1 >= 0 && part2 >= 0);
+				break;
+
+			case JOIN_LEFT:
+			case JOIN_ANTI:
+				Assert(part1 >= 0);
+				if (part2 < 0)
+					merged = false;
+				break;
+
+			case JOIN_FULL:
+				if (part1 < 0 || part2 < 0)
+					merged = false;
+				break;
+
+			default:
+				elog(ERROR, "unrecognized join type: %d", (int) jointype);
+		}
+
+		if (!merged)
+			break;
+
+		*matched_parts1 = lappend_int(*matched_parts1, part1);
+		*matched_parts2 = lappend_int(*matched_parts2, part2);
+	}
+
+	pfree(matching1);
+	pfree(matching2);
+
+	if (!merged)
+	{
+		list_free(*matched_parts1);
+		list_free(*matched_parts2);
+		*matched_parts1 = NIL;
+		*matched_parts2 = NIL;
+	}
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * Merge default partitions from both sides, if any, and assign the default
+ * partition for the join result, if necessary.
+ *
+ * If both the relations have default partitions, try mapping those to each
+ * other. If the mapping succeeds corresponding merged partition will act as
+ * the default partition of the join result.
+ *
+ * If inner side of the join has default but not the outer side, rows in it
+ * won't appear in the join result. So don't create a default partition. If
+ * outer side of the join has default but not the inner side, rows in it will
+ * appear in the join result, so create a default merged partition.
+ */
+static bool
+merge_default_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+						 PartitionMap *outer_maps, PartitionMap *inner_maps,
+						 JoinType jointype, int *next_index, int *default_index)
+{
+	int				outer_default = outer_bi->default_index;
+	int				inner_default = inner_bi->default_index;
+	bool			outer_has_default = partition_bound_has_default(outer_bi);
+	bool			inner_has_default = partition_bound_has_default(inner_bi);
+	bool			merged = true;
+	PartitionMap 	*outer_default_map = NULL;
+	PartitionMap 	*inner_default_map = NULL;
+
+	if (outer_has_default)
+		outer_default_map = &outer_maps[outer_default];
+
+	if (inner_has_default)
+		inner_default_map = &inner_maps[inner_default];
+
+	if (!outer_has_default && !inner_has_default)
+		Assert(*default_index < 0);
+	else if (outer_default_map != NULL && inner_default_map == NULL)
+	{
+		if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+			jointype == JOIN_ANTI)
+		{
+			if (outer_default_map->to < 0)
+			{
+				outer_default_map->to = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = outer_default_map->to;
+			}
+			else
+				Assert(*default_index == outer_default_map->to);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else if (outer_default_map == NULL && inner_default_map != NULL)
+	{
+		if (jointype == JOIN_FULL)
+		{
+			if (inner_default_map->to < 0)
+			{
+				inner_default_map->to = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = inner_default_map->to;
+			}
+			else
+				Assert(*default_index == inner_default_map->to);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		*default_index = map_and_merge_partitions(outer_maps, inner_maps,
+												  outer_default, inner_default,
+												  next_index);
+
+		if (*default_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
+
+/*
+ * merge_null_partitions
+ *
+ * Merge NULL partitions, i.e. a partition that can hold NULL values for a list
+ * partitioned table, if any. Find the index of merged partition to which the
+ * NULL values would belong in the join result. If one joining relation has a
+ * NULL partition but not the other, try matching it with the default partition
+ * from the other relation since the default partition may have rows with NULL
+ * partition key. We can eliminate a NULL partition when it appears only on the
+ * inner side of the join and the outer side doesn't have a default partition.
+ *
+ * When the equality operator used for join is strict, two NULL values will not
+ * be considered as equal, and thus a NULL partition can be eliminated for an
+ * inner join. But we don't check the strictness operator here.
+ */
+static bool
+merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_maps, PartitionMap *inner_maps,
+					  JoinType jointype, int *next_index,
+					  int *null_index, int *default_index)
+{
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	int			outer_ni = outer_bi->null_index;
+	int			inner_ni = inner_bi->null_index;
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	bool		merged = true;
+
+	if (!outer_has_null && !inner_has_null)
+		Assert(*null_index < 0);
+	else if (outer_has_null && !inner_has_null)
+	{
+		int			merged_index = -1;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, inner relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the inner relation acts as
+		 * INNER relation. For INNER and SEMI join OUTER and INNER
+		 * differentiation is immaterial.
+		 */
+		missing_side_inner = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_outer = (jointype == JOIN_FULL);
+
+		merged = handle_missing_partition(inner_maps,
+										  outer_maps,
+										  inner_default,
+										  outer_ni,
+										  missing_side_outer,
+										  missing_side_inner, next_index,
+										  default_index, &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join to which the outer null partition maps
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = outer_maps[outer_ni].to;
+	}
+	else if (!outer_has_null && inner_has_null)
+	{
+		int			merged_index = -1;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, outer relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the outer relation acts as OUTER
+		 * relation. For INNER and SEMI join OUTER and INNER differentiation is
+		 * immaterial.
+		 */
+		missing_side_outer = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_inner = (jointype == JOIN_FULL);
+		merged = handle_missing_partition(outer_maps,
+										  inner_maps,
+										  outer_default,
+										  inner_ni,
+										  missing_side_outer,
+										  missing_side_inner,
+										  next_index, default_index,
+										  &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join, to which the outer side null partition maps,
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = inner_maps[inner_ni].to;
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		*null_index = map_and_merge_partitions(outer_maps,
+											   inner_maps,
+											   outer_ni,
+											   inner_ni,
+											   next_index);
+		if (*null_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index c954965e4e..3823ad4cab 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -16,6 +16,7 @@
 #include "nodes/pg_list.h"
 #include "partitioning/partdefs.h"
 #include "utils/relcache.h"
+struct RelOptInfo;				/* avoid including pathnodes.h here */
 
 
 /*
@@ -109,5 +110,11 @@ extern int partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 							  int nvalues, Datum *values, bool *is_equal);
 extern int partition_hash_bsearch(PartitionBoundInfo boundinfo,
 					   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   struct RelOptInfo *outer_rel,
+					   struct RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
-- 
2.18.0

0001-Hash-partition-bound-equality-refactoring-v21.patchapplication/octet-stream; name=0001-Hash-partition-bound-equality-refactoring-v21.patchDownload
From df87e688e0a9ac1f0b4820a3e5b920a08fe3196b Mon Sep 17 00:00:00 2001
From: Etsuro Fujita <efujita@postgresql.org>
Date: Thu, 31 Jan 2019 19:18:18 +0900
Subject: [PATCH 1/3] Hash partition bound equality refactoring.

Separate the code to check whether two given hash bounds are equal into a
separate function, so that it can be called from multiple places. Right now
it's only caller is partition_bounds_equal() but later we will use it for
merging partition bounds.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/partitioning/partbounds.c | 95 +++++++++++++++++----------
 1 file changed, 61 insertions(+), 34 deletions(-)

diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 0e33a370c6..35c03f611a 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -108,6 +108,9 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 						 Expr **keyCol,
 						 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static bool partition_hbounds_equal(PartitionBoundInfo b1,
+									PartitionBoundInfo b2);
+
 
 /*
  * get_qual_from_partbound
@@ -656,6 +659,63 @@ create_range_bounds(PartitionBoundSpec **boundspecs, int nparts,
 	return boundinfo;
 }
 
+/*
+ * Are two hash partition bound collections logically equal?
+ *
+ * Hash partition bounds store modulus and remainder in datums array which are
+ * always integers irrespective of the number of partition keys and their data
+ * types. Hence we can compare the hash bound collection without any partition
+ * key specific information. Separating this logic in a function which does not
+ * require partition key specific information allows it be called from places
+ * where the partition key specific information is not completely available.
+ */
+static bool
+partition_hbounds_equal(PartitionBoundInfo b1, PartitionBoundInfo b2)
+{
+	int			greatest_modulus = get_hash_partition_greatest_modulus(b1);
+	int			i;
+
+	Assert(b1->strategy == PARTITION_STRATEGY_HASH &&
+		   b2->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * If two hash partitioned tables have different greatest moduli,
+	 * their partition schemes don't match.  For hash partitioned table,
+	 * the greatest modulus is given by the last datum and number of
+	 * partitions is given by ndatums.
+	 */
+	if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
+		return false;
+
+	/*
+	 * We arrange the partitions in the ascending order of their modulus and
+	 * remainders.  Also every modulus is factor of next larger modulus.
+	 * Therefore we can safely store index of a given partition in indexes
+	 * array at remainder of that partition.  Also entries at (remainder + N *
+	 * modulus) positions in indexes array are all same for (modulus,
+	 * remainder) specification for any partition.  Thus datums array from both
+	 * the given bounds are same, if and only if their indexes array will be
+	 * same.  So, it suffices to compare indexes array.
+	 */
+	for (i = 0; i < greatest_modulus; i++)
+		if (b1->indexes[i] != b2->indexes[i])
+			return false;
+
+#ifdef USE_ASSERT_CHECKING
+
+	/*
+	 * Nonetheless make sure that the bounds are indeed same when the indexes
+	 * match.  Hash partition bound stores modulus and remainder at
+	 * b1->datums[i][0] and b1->datums[i][1] position respectively.
+	 */
+	for (i = 0; i < b1->ndatums; i++)
+		Assert((b1->datums[i][0] == b2->datums[i][0] &&
+				b1->datums[i][1] == b2->datums[i][1]));
+#endif
+
+	return true;
+}
+
 /*
  * Are two partition bound collections logically equal?
  *
@@ -684,41 +744,8 @@ partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
 
 	if (b1->strategy == PARTITION_STRATEGY_HASH)
 	{
-		int			greatest_modulus = get_hash_partition_greatest_modulus(b1);
-
-		/*
-		 * If two hash partitioned tables have different greatest moduli,
-		 * their partition schemes don't match.
-		 */
-		if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
+		if (!partition_hbounds_equal(b1, b2))
 			return false;
-
-		/*
-		 * We arrange the partitions in the ascending order of their moduli
-		 * and remainders.  Also every modulus is factor of next larger
-		 * modulus.  Therefore we can safely store index of a given partition
-		 * in indexes array at remainder of that partition.  Also entries at
-		 * (remainder + N * modulus) positions in indexes array are all same
-		 * for (modulus, remainder) specification for any partition.  Thus
-		 * datums array from both the given bounds are same, if and only if
-		 * their indexes array will be same.  So, it suffices to compare
-		 * indexes array.
-		 */
-		for (i = 0; i < greatest_modulus; i++)
-			if (b1->indexes[i] != b2->indexes[i])
-				return false;
-
-#ifdef USE_ASSERT_CHECKING
-
-		/*
-		 * Nonetheless make sure that the bounds are indeed same when the
-		 * indexes match.  Hash partition bound stores modulus and remainder
-		 * at b1->datums[i][0] and b1->datums[i][1] position respectively.
-		 */
-		for (i = 0; i < b1->ndatums; i++)
-			Assert((b1->datums[i][0] == b2->datums[i][0] &&
-					b1->datums[i][1] == b2->datums[i][1]));
-#endif
 	}
 	else
 	{
-- 
2.18.0

0003-Tests-for-0-1-1-1-and-1-0-partition-matching-v21.patchapplication/octet-stream; name=0003-Tests-for-0-1-1-1-and-1-0-partition-matching-v21.patchDownload
From 512ae6f3ec4788d7453f2df56f8b8fd437d928ee Mon Sep 17 00:00:00 2001
From: Amul Sul <amul.sul@enterprisedb.com>
Date: Wed, 24 Apr 2019 05:28:25 -0400
Subject: [PATCH 3/3] Tests for 0:1, 1:1 and 1:0 partition matching

Rajkumar Raghuvanshi and Ashutosh Bapat.
---
 src/test/regress/expected/partition_join.out | 3944 +++++++++++++++---
 src/test/regress/sql/partition_join.sql      |  427 +-
 2 files changed, 3751 insertions(+), 620 deletions(-)

diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 1296edcdae..d26be918aa 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -8,59 +8,86 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-(4 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+(7 rows)
 
 -- left outer join, with whole-row reference; partitionwise join does not apply
 EXPLAIN (COSTS OFF)
@@ -72,35 +99,50 @@ SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER
    ->  Hash Right Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(22 rows)
 
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-      t1      |      t2      
---------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
- (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
- (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
- (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
- (550,0,0550) | 
-(12 rows)
+       t1       |       t2       
+----------------+----------------
+ (-250,0,-0250) | 
+ (-200,0,-0200) | 
+ (-150,0,-0150) | (0,-150,-0150)
+ (-100,0,-0100) | 
+ (-50,0,-0050)  | 
+ (0,0,0000)     | (0,0,0000)
+ (50,0,0050)    | 
+ (100,0,0100)   | 
+ (150,0,0150)   | (0,150,0150)
+ (200,0,0200)   | 
+ (250,0,0250)   | 
+ (300,0,0300)   | (0,300,0300)
+ (350,0,0350)   | 
+ (400,0,0400)   | 
+ (450,0,0450)   | (0,450,0450)
+ (500,0,0500)   | 
+ (550,0,0550)   | 
+ (600,0,0600)   | (0,600,0600)
+ (650,0,0650)   | 
+ (700,0,0700)   | 
+ (750,0,0750)   | (0,750,0750)
+(21 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
@@ -112,35 +154,53 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
    ->  Append
          ->  Hash Right Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Right Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t2_2.b)
-(20 rows)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+      |       | -225 | -0225
+      |       |  -75 | -0075
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(14 rows)
 
 -- full outer join, with placeholder vars
 EXPLAIN (COSTS OFF)
@@ -148,8 +208,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                             QUERY PLAN                            
 ------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               Filter: (((50) = prt1_p0.a) OR ((75) = prt2_p0.b))
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
@@ -174,7 +242,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               Filter: (((50) = prt1_p4.a) OR ((75) = prt2_p4.b))
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
@@ -211,35 +287,44 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Hash Right Join
-         Hash Cond: (prt2_p2.b = prt1_p1.a)
+         Hash Cond: (prt2_p2.b = prt1_p0.a)
          ->  Append
                ->  Seq Scan on prt2_p2
                      Filter: (b > 250)
                ->  Seq Scan on prt2_p3
                      Filter: (b > 250)
+               ->  Seq Scan on prt2_p4
+                     Filter: (b > 250)
          ->  Hash
                ->  Append
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p1
                            Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p2
                            Filter: ((a < 450) AND (b = 0))
-(15 rows)
+(19 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-(9 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+(14 rows)
 
 -- Currently we can't do partitioned join if nullable-side partitions are pruned
 EXPLAIN (COSTS OFF)
@@ -247,11 +332,13 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                       QUERY PLAN                      
 ------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Hash Full Join
-         Hash Cond: (prt1_p1.a = prt2_p2.b)
-         Filter: ((prt1_p1.b = 0) OR (prt2_p2.a = 0))
+         Hash Cond: (prt1_p0.a = prt2_p2.b)
+         Filter: ((prt1_p0.b = 0) OR (prt2_p2.a = 0))
          ->  Append
+               ->  Seq Scan on prt1_p0
+                     Filter: (a < 450)
                ->  Seq Scan on prt1_p1
                      Filter: (a < 450)
                ->  Seq Scan on prt1_p2
@@ -262,64 +349,147 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                            Filter: (b > 250)
                      ->  Seq Scan on prt2_p3
                            Filter: (b > 250)
-(16 rows)
+                     ->  Seq Scan on prt2_p4
+                           Filter: (b > 250)
+(20 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
-(12 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+      |       | 375 | 0375
+      |       | 450 | 0450
+      |       | 525 | 0525
+      |       | 600 | 0600
+      |       | 675 | 0675
+      |       | 750 | 0750
+(20 rows)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Semi Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Semi Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                            Filter: (a = 0)
-         ->  Nested Loop Semi Join
-               Join Filter: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
-               ->  Materialize
-                     ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
-(24 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt1_p3 t2_3
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+                           Filter: (b = 0)
+(37 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
 
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
@@ -330,27 +500,82 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
    ->  Append
          ->  Hash Anti Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Hash Anti Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Hash Anti Join
                Hash Cond: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
-(17 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(27 rows)
 
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
-  sum  |         avg          | sum  |         avg         
--------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+  sum  |         avg          | sum  |        avg         
+-------+----------------------+------+--------------------
+ 95550 | 273.0000000000000000 | 2200 | 6.2857142857142857
 (1 row)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Append
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p0 t1
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+               Index Cond: (a = t1.b)
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.b = t2_1.a)
+         ->  Seq Scan on prt2_p1 t1_1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p1 t2_1
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p2 t1_2
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+               Index Cond: (a = t1_2.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p3 t1_3
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+               Index Cond: (a = t1_3.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p4 t1_4
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+               Index Cond: (a = t1_4.b)
+(27 rows)
+
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+  b   |   c   
+------+-------
+ -225 | -0225
+  -75 | -0075
+   75 | 0075
+  225 | 0225
+  375 | 0375
+  525 | 0525
+  675 | 0675
+(7 rows)
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -362,49 +587,74 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
+                     ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
                            Index Cond: (a = t1.a)
-                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t3
                            Index Cond: (b = t2.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_1
+                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
                            Index Cond: (a = t1_1.a)
-                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3_1
                            Index Cond: (b = t2_1.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_2
+                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
                            Index Cond: (a = t1_2.a)
-                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_2
                            Index Cond: (b = t2_2.a)
-(27 rows)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                           Index Cond: (a = t1_3.a)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_3
+                           Index Cond: (b = t2_3.a)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                           Index Cond: (a = t1_4.a)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t3_4
+                           Index Cond: (b = t2_4.a)
+(43 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   | t2a | t3a | least 
------+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
- 100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
- 250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
- 400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
- 550 | 0 | 0550 |     |     |      
-(12 rows)
+  a   | b |   c   | t2a  | t3a | least 
+------+---+-------+------+-----+-------
+ -250 | 0 | -0250 |      |     |      
+ -200 | 0 | -0200 |      |     |      
+ -150 | 0 | -0150 | -150 |   0 |  -150
+ -100 | 0 | -0100 |      |     |      
+  -50 | 0 | -0050 |      |     |      
+    0 | 0 | 0000  |    0 |   0 |     0
+   50 | 0 | 0050  |      |     |      
+  100 | 0 | 0100  |      |     |      
+  150 | 0 | 0150  |  150 |   0 |   150
+  200 | 0 | 0200  |      |     |      
+  250 | 0 | 0250  |      |     |      
+  300 | 0 | 0300  |  300 |   0 |   300
+  350 | 0 | 0350  |      |     |      
+  400 | 0 | 0400  |      |     |      
+  450 | 0 | 0450  |  450 |   0 |   450
+  500 | 0 | 0500  |      |     |      
+  550 | 0 | 0550  |      |     |      
+  600 | 0 | 0600  |  600 |   0 |   600
+  650 | 0 | 0650  |      |     |      
+  700 | 0 | 0700  |      |     |      
+  750 | 0 | 0750  |  750 |   0 |   750
+(21 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
@@ -418,64 +668,95 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
          Hash Cond: ((t1.c)::text = (t2.c)::text)
          Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Seq Scan on prt1_p1 t1
-               ->  Seq Scan on prt1_p2 t1_1
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
          ->  Hash
                ->  Append
                      ->  Hash Join
                            Hash Cond: (t2.a = t3.b)
-                           ->  Seq Scan on prt1_p1 t2
+                           ->  Seq Scan on prt1_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t3
+                                 ->  Seq Scan on prt2_p0 t3
                      ->  Hash Join
                            Hash Cond: (t2_1.a = t3_1.b)
-                           ->  Seq Scan on prt1_p2 t2_1
+                           ->  Seq Scan on prt1_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t3_1
+                                 ->  Seq Scan on prt2_p1 t3_1
                      ->  Hash Join
                            Hash Cond: (t2_2.a = t3_2.b)
-                           ->  Seq Scan on prt1_p3 t2_2
+                           ->  Seq Scan on prt1_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p3 t3_2
-(26 rows)
+                                 ->  Seq Scan on prt2_p2 t3_2
+                     ->  Hash Join
+                           Hash Cond: (t2_3.a = t3_3.b)
+                           ->  Seq Scan on prt1_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t3_3
+                     ->  Hash Join
+                           Hash Cond: (t2_4.a = t3_4.b)
+                           ->  Seq Scan on prt1_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t3_4
+(38 rows)
 
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, t2.b t2b, t2.c t2c, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
-  a  | t2a | t2c  
------+-----+------
-   0 |   0 | 0000
-  50 |     | 
- 100 |     | 
- 150 | 150 | 0150
- 200 |     | 
- 250 |     | 
- 300 | 300 | 0300
- 350 |     | 
- 400 |     | 
- 450 | 450 | 0450
- 500 |     | 
- 550 |     | 
-(12 rows)
+  a   | t2a  |  t2c  
+------+------+-------
+ -250 |      | 
+ -200 |      | 
+ -150 | -150 | -0150
+ -100 |      | 
+  -50 |      | 
+    0 |    0 | 0000
+   50 |      | 
+  100 |      | 
+  150 |  150 | 0150
+  200 |      | 
+  250 |      | 
+  300 |  300 | 0300
+  350 |      | 
+  400 |      | 
+  450 |  450 | 0450
+  500 |      | 
+  550 |      | 
+  600 |  600 | 0600
+  650 |      | 
+  700 |      | 
+  750 |  750 | 0750
+(21 rows)
 
 --
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
@@ -486,32 +767,49 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
    ->  Append
          ->  Hash Join
                Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
-               ->  Seq Scan on prt2_e_p1 t2
+               ->  Seq Scan on prt2_e_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 t1
+                     ->  Seq Scan on prt1_e_p0 t1
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
-               ->  Seq Scan on prt2_e_p2 t2_1
+               Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+               ->  Seq Scan on prt1_e_p1 t1_1
+                     Filter: (c = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 t1_1
-                           Filter: (c = 0)
+                     ->  Seq Scan on prt2_e_p1 t2_1
          ->  Hash Join
                Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
-               ->  Seq Scan on prt2_e_p3 t2_2
+               ->  Seq Scan on prt2_e_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 t1_2
+                     ->  Seq Scan on prt1_e_p2 t1_2
                            Filter: (c = 0)
-(21 rows)
+         ->  Hash Join
+               Hash Cond: (((t2_3.b + t2_3.a) / 2) = ((t1_3.a + t1_3.b) / 2))
+               ->  Seq Scan on prt2_e_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p3 t1_3
+                           Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (((t2_4.b + t2_4.a) / 2) = ((t1_4.a + t1_4.b) / 2))
+               ->  Seq Scan on prt2_e_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4 t1_4
+                           Filter: (c = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
- 150 | 0 | 150 | 0
- 300 | 0 | 300 | 0
- 450 | 0 | 450 | 0
-(4 rows)
+  a   | c |  b   | c 
+------+---+------+---
+ -250 | 0 | -250 | 0
+ -100 | 0 | -100 | 0
+    0 | 0 |    0 | 0
+    0 | 0 |    0 | 0
+  150 | 0 |  150 | 0
+  300 | 0 |  300 | 0
+  450 | 0 |  450 | 0
+  600 | 0 |  600 | 0
+  750 | 0 |  750 | 0
+(9 rows)
 
 --
 -- N-way join
@@ -524,154 +822,232 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = ((t3.a + t3.b) / 2))
+               Join Filter: (t1.a = t2.b)
                ->  Hash Join
-                     Hash Cond: (t2.b = t1.a)
-                     ->  Seq Scan on prt2_p1 t2
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on prt1_e_p0 t3
                      ->  Hash
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                     Index Cond: (((a + b) / 2) = t2.b)
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
+                     Index Cond: (b = ((t3.a + t3.b) / 2))
          ->  Nested Loop
                Join Filter: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_1.b = t1_1.a)
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
+               ->  Index Scan using iprt1_e_p4_ab2 on prt1_e_p1 t3_1
                      Index Cond: (((a + b) / 2) = t2_1.b)
          ->  Nested Loop
                Join Filter: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_2.b = t1_2.a)
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
+               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_2
                      Index Cond: (((a + b) / 2) = t2_2.b)
-(33 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = ((t3_3.a + t3_3.b) / 2))
+               ->  Nested Loop
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t2_3.b)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t2_4.b)
+               ->  Hash Join
+                     Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                     ->  Seq Scan on prt1_e_p4 t3_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_p4 t1_4
+                                 Filter: (b = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = ((t3_4.a + t3_4.b) / 2))
+(52 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
-(4 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(8 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                          QUERY PLAN                          
---------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Hash Right Join
                Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
-               ->  Seq Scan on prt1_e_p1 t3
+               ->  Seq Scan on prt1_e_p0 t3
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2.b = t1.a)
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_1.a)
-               ->  Seq Scan on prt1_e_p2 t3_1
+               ->  Seq Scan on prt1_e_p1 t3_1
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_1.b = t1_1.a)
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_2.a)
-               ->  Seq Scan on prt1_e_p3 t3_2
+               ->  Seq Scan on prt1_e_p2 t3_2
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_2.b = t1_2.a)
-                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                                        Filter: (b = 0)
-(33 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+               ->  Seq Scan on prt1_e_p4 t3_4
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_4.b = t1_4.a)
+                           ->  Seq Scan on prt2_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_p4 t1_4
+                                       Filter: (b = 0)
+(51 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                            QUERY PLAN                             
--------------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p1 t3
+                           ->  Seq Scan on prt1_e_p0 t3
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p1_b on prt2_p1 t2
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
                      Index Cond: (b = t1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p2 t3_1
+                           ->  Seq Scan on prt1_e_p1 t3_1
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
+               ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
                      Index Cond: (b = t1_1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p3 t3_2
+                           ->  Seq Scan on prt1_e_p2 t3_2
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
                      Index Cond: (b = t1_2.a)
-(30 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_e_p3 t3_3
+                           Filter: (c = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                           Index Cond: (a = ((t3_3.a + t3_3.b) / 2))
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Nested Loop Left Join
+               ->  Hash Right Join
+                     Hash Cond: (t1_4.a = ((t3_4.a + t3_4.b) / 2))
+                     ->  Seq Scan on prt1_p4 t1_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p4 t3_4
+                                 Filter: (c = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = t1_4.a)
+(47 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -680,21 +1056,34 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                                                    QUERY PLAN                                                   
 ----------------------------------------------------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   Sort Key: prt1_p0.a, prt2_p0.b, ((prt1_e_p0.a + prt1_e_p0.b))
    ->  Append
          ->  Hash Full Join
-               Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
-               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               Hash Cond: (prt1_p0.a = ((prt1_e_p0.a + prt1_e_p0.b) / 2))
+               Filter: ((prt1_p0.a = (50)) OR (prt2_p0.b = (75)) OR (((prt1_e_p0.a + prt1_e_p0.b) / 2) = (50)))
                ->  Hash Full Join
-                     Hash Cond: (prt1_p1.a = prt2_p1.b)
-                     ->  Seq Scan on prt1_p1
+                     Hash Cond: (prt1_p0.a = prt2_p0.b)
+                     ->  Seq Scan on prt1_p0
                            Filter: (b = 0)
                      ->  Hash
-                           ->  Seq Scan on prt2_p1
+                           ->  Seq Scan on prt2_p0
                                  Filter: (a = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1
+                     ->  Seq Scan on prt1_e_p0
                            Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = prt1_p1.a)
+               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               ->  Seq Scan on prt1_e_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on prt1_p1
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1
+                                       Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
                Filter: ((prt1_p2.a = (50)) OR (prt2_p2.b = (75)) OR (((prt1_e_p2.a + prt1_e_p2.b) / 2) = (50)))
@@ -721,7 +1110,20 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                ->  Hash
                      ->  Seq Scan on prt1_e_p3
                            Filter: (c = 0)
-(42 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = ((prt1_e_p4.a + prt1_e_p4.b) / 2))
+               Filter: ((prt1_p4.a = (50)) OR (prt2_p4.b = (75)) OR (((prt1_e_p4.a + prt1_e_p4.b) / 2) = (50)))
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = prt2_p4.b)
+                     ->  Seq Scan on prt1_p4
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4
+                                 Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4
+                           Filter: (c = 0)
+(68 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
@@ -739,172 +1141,260 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = t1_3.b)
+               Join Filter: (t1.a = t1_5.b)
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Join
-                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
-                           ->  Seq Scan on prt1_e_p1 t2
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t1_3
+                                 ->  Seq Scan on prt2_p0 t1_5
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
                      Index Cond: (a = ((t2.a + t2.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_1.a = t1_4.b)
+               Join Filter: (t1_1.a = t1_6.b)
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Join
-                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
-                           ->  Seq Scan on prt1_e_p2 t2_1
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_6.b)
+                           ->  Seq Scan on prt1_e_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t1_4
+                                 ->  Seq Scan on prt2_p1 t1_6
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
                      Index Cond: (a = ((t2_1.a + t2_1.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_2.a = t1_5.b)
+               Join Filter: (t1_2.a = t1_7.b)
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
+                           ->  Seq Scan on prt2_p2 t1_7
                                  Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
+                           ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t2_2
+                                 Index Cond: (((a + b) / 2) = t1_7.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t1_8.b)
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Nested Loop
+                           ->  Seq Scan on prt2_p3 t1_8
+                                 Filter: (a = 0)
+                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_3
+                                 Index Cond: (((a + b) / 2) = t1_8.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = ((t2_3.a + t2_3.b) / 2))
+                     Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t1_9.b)
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Join
+                           Hash Cond: (((t2_4.a + t2_4.b) / 2) = t1_9.b)
+                           ->  Seq Scan on prt1_e_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t1_9
+                                       Filter: (a = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = ((t2_4.a + t2_4.b) / 2))
+                     Filter: (b = 0)
+(66 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                               QUERY PLAN                                
--------------------------------------------------------------------------
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
-                           ->  Seq Scan on prt2_p1 t1_3
+                           Hash Cond: (t1_5.b = ((t1_10.a + t1_10.b) / 2))
+                           ->  Seq Scan on prt2_p0 t1_5
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t1_6
+                                 ->  Seq Scan on prt1_e_p0 t1_10
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
-                     Index Cond: (a = t1_3.b)
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t1_5.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
-                           ->  Seq Scan on prt2_p2 t1_4
+                           Hash Cond: (t1_6.b = ((t1_11.a + t1_11.b) / 2))
+                           ->  Seq Scan on prt2_p1 t1_6
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t1_7
+                                 ->  Seq Scan on prt1_e_p1 t1_11
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
-                     Index Cond: (a = t1_4.b)
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
+                     Index Cond: (a = t1_6.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
-                           ->  Seq Scan on prt2_p3 t1_5
+                           Hash Cond: (t1_7.b = ((t1_12.a + t1_12.b) / 2))
+                           ->  Seq Scan on prt2_p2 t1_7
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p3 t1_8
+                                 ->  Seq Scan on prt1_e_p2 t1_12
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t1_5.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t1_7.b)
                      Filter: (b = 0)
-(39 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_8.b = ((t1_13.a + t1_13.b) / 2))
+                           ->  Seq Scan on prt2_p3 t1_8
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p3 t1_13
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t1_8.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_9.b = ((t1_14.a + t1_14.b) / 2))
+                           ->  Seq Scan on prt2_p4 t1_9
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t1_14
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = t1_9.b)
+                     Filter: (b = 0)
+(63 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 -- test merge joins
 SET enable_hashjoin TO off;
 SET enable_nestloop TO off;
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                           QUERY PLAN                           
-----------------------------------------------------------------
+                            QUERY PLAN                            
+------------------------------------------------------------------
  Merge Append
    Sort Key: t1.a
    ->  Merge Semi Join
-         Merge Cond: (t1.a = t1_3.b)
+         Merge Cond: (t1.a = t1_5.b)
          ->  Sort
                Sort Key: t1.a
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+               Merge Cond: (t1_5.b = (((t1_10.a + t1_10.b) / 2)))
                ->  Sort
-                     Sort Key: t1_3.b
-                     ->  Seq Scan on prt2_p1 t1_3
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on prt2_p0 t1_5
                ->  Sort
-                     Sort Key: (((t1_6.a + t1_6.b) / 2))
-                     ->  Seq Scan on prt1_e_p1 t1_6
+                     Sort Key: (((t1_10.a + t1_10.b) / 2))
+                     ->  Seq Scan on prt1_e_p0 t1_10
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_1.a = t1_4.b)
+         Merge Cond: (t1_1.a = t1_6.b)
          ->  Sort
                Sort Key: t1_1.a
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+               Merge Cond: (t1_6.b = (((t1_11.a + t1_11.b) / 2)))
                ->  Sort
-                     Sort Key: t1_4.b
-                     ->  Seq Scan on prt2_p2 t1_4
+                     Sort Key: t1_6.b
+                     ->  Seq Scan on prt2_p1 t1_6
                ->  Sort
-                     Sort Key: (((t1_7.a + t1_7.b) / 2))
-                     ->  Seq Scan on prt1_e_p2 t1_7
+                     Sort Key: (((t1_11.a + t1_11.b) / 2))
+                     ->  Seq Scan on prt1_e_p1 t1_11
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_2.a = t1_5.b)
+         Merge Cond: (t1_2.a = t1_7.b)
          ->  Sort
                Sort Key: t1_2.a
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+               Merge Cond: (t1_7.b = (((t1_12.a + t1_12.b) / 2)))
                ->  Sort
-                     Sort Key: t1_5.b
-                     ->  Seq Scan on prt2_p3 t1_5
+                     Sort Key: t1_7.b
+                     ->  Seq Scan on prt2_p2 t1_7
                ->  Sort
-                     Sort Key: (((t1_8.a + t1_8.b) / 2))
-                     ->  Seq Scan on prt1_e_p3 t1_8
+                     Sort Key: (((t1_12.a + t1_12.b) / 2))
+                     ->  Seq Scan on prt1_e_p2 t1_12
                            Filter: (c = 0)
-(47 rows)
+   ->  Merge Semi Join
+         Merge Cond: (t1_3.a = t1_8.b)
+         ->  Sort
+               Sort Key: t1_3.a
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_8.b = (((t1_13.a + t1_13.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_8.b
+                     ->  Seq Scan on prt2_p3 t1_8
+               ->  Sort
+                     Sort Key: (((t1_13.a + t1_13.b) / 2))
+                     ->  Seq Scan on prt1_e_p3 t1_13
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_4.a = t1_9.b)
+         ->  Sort
+               Sort Key: t1_4.a
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_9.b = (((t1_14.a + t1_14.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_9.b
+                     ->  Seq Scan on prt2_p4 t1_9
+               ->  Sort
+                     Sort Key: (((t1_14.a + t1_14.b) / 2))
+                     ->  Seq Scan on prt1_e_p4 t1_14
+                           Filter: (c = 0)
+(77 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
@@ -921,14 +1411,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
                            ->  Sort
                                  Sort Key: (((t3.a + t3.b) / 2))
-                                 ->  Seq Scan on prt1_e_p1 t3
+                                 ->  Seq Scan on prt1_e_p0 t3
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1.a
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                ->  Sort
                      Sort Key: t2.b
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Merge Left Join
                Merge Cond: (t1_1.a = t2_1.b)
                ->  Sort
@@ -937,14 +1427,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
                            ->  Sort
                                  Sort Key: (((t3_1.a + t3_1.b) / 2))
-                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                 ->  Seq Scan on prt1_e_p1 t3_1
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_1.a
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                ->  Sort
                      Sort Key: t2_1.b
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Merge Left Join
                Merge Cond: (t1_2.a = t2_2.b)
                ->  Sort
@@ -953,32 +1443,74 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
                            ->  Sort
                                  Sort Key: (((t3_2.a + t3_2.b) / 2))
-                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                 ->  Seq Scan on prt1_e_p2 t3_2
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_2.a
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                ->  Sort
                      Sort Key: t2_2.b
-                     ->  Seq Scan on prt2_p3 t2_2
-(51 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Merge Left Join
+               Merge Cond: (t1_3.a = t2_3.b)
+               ->  Sort
+                     Sort Key: t1_3.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_3.a + t3_3.b) / 2)) = t1_3.a)
+                           ->  Sort
+                                 Sort Key: (((t3_3.a + t3_3.b) / 2))
+                                 ->  Seq Scan on prt1_e_p3 t3_3
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_3.a
+                                 ->  Seq Scan on prt1_p3 t1_3
+               ->  Sort
+                     Sort Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Merge Left Join
+               Merge Cond: (t1_4.a = t2_4.b)
+               ->  Sort
+                     Sort Key: t1_4.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_4.a + t3_4.b) / 2)) = t1_4.a)
+                           ->  Sort
+                                 Sort Key: (((t3_4.a + t3_4.b) / 2))
+                                 ->  Seq Scan on prt1_e_p4 t3_4
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_4.a
+                                 ->  Seq Scan on prt1_p4 t1_4
+               ->  Sort
+                     Sort Key: t2_4.b
+                     ->  Seq Scan on prt2_p4 t2_4
+(83 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- MergeAppend on nullable column
 -- This should generate a partitionwise join, but currently fails to
@@ -987,12 +1519,14 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Merge Left Join
-         Merge Cond: (prt1_p1.a = prt2_p2.b)
+         Merge Cond: (prt1_p0.a = prt2_p2.b)
          ->  Sort
-               Sort Key: prt1_p1.a
+               Sort Key: prt1_p0.a
                ->  Append
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p1
                            Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p2
@@ -1004,21 +1538,28 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                            Filter: (b > 250)
                      ->  Seq Scan on prt2_p3
                            Filter: (b > 250)
-(18 rows)
+                     ->  Seq Scan on prt2_p4
+                           Filter: (b > 250)
+(22 rows)
 
 SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  b  
------+-----
-   0 |    
-  50 |    
- 100 |    
- 150 |    
- 200 |    
- 250 |    
- 300 | 300
- 350 |    
- 400 |    
-(9 rows)
+  a   |  b  
+------+-----
+ -250 |    
+ -200 |    
+ -150 |    
+ -100 |    
+  -50 |    
+    0 |    
+   50 |    
+  100 |    
+  150 |    
+  200 |    
+  250 |    
+  300 | 300
+  350 |    
+  400 |    
+(14 rows)
 
 -- merge join when expression with whole-row reference needs to be sorted;
 -- partitionwise join does not apply
@@ -1032,30 +1573,797 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
          Sort Key: t1.a, ((((t1.*)::prt1))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
    ->  Sort
          Sort Key: t2.b, ((((t2.*)::prt2))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt2_p1 t2
-                     ->  Seq Scan on prt2_p2 t2_1
-                     ->  Seq Scan on prt2_p3 t2_2
-(16 rows)
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+(20 rows)
 
 SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.b ORDER BY t1.a;
- a  | b  
-----+----
-  0 |  0
-  6 |  6
- 12 | 12
- 18 | 18
- 24 | 24
-(5 rows)
+  a  |  b  
+-----+-----
+ -24 | -24
+ -18 | -18
+ -12 | -12
+  -6 |  -6
+   0 |   0
+   6 |   6
+  12 |  12
+  18 |  18
+  24 |  24
+(9 rows)
 
 RESET enable_hashjoin;
 RESET enable_nestloop;
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p4 t2_3
+               ->  Seq Scan on prt2_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_4
+                           Filter: (b = 0)
+(22 rows)
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p3 t1_2
+                           Filter: (b = 0)
+(21 rows)
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -150 | -0150 | 0 | -0150
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+(7 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 |   | 
+ -200 | -0200 |   | 
+ -150 | -0150 | 0 | -0150
+ -100 | -0100 |   | 
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+(21 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -200 | 0 | -0200
+ -100 | 0 | -0100
+  -50 | 0 | -0050
+   50 | 0 | 0050
+  100 | 0 | 0100
+  200 | 0 | 0200
+  250 | 0 | 0250
+  350 | 0 | 0350
+  400 | 0 | 0400
+  500 | 0 | 0500
+  550 | 0 | 0550
+  650 | 0 | 0650
+  700 | 0 | 0700
+(14 rows)
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+                             QUERY PLAN                              
+---------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t3.c
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p0_a on prt1_p0 t3
+                           Index Cond: (a = t2.b)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t2.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_1.a = t2_1.b)
+                           ->  Seq Scan on prt1_p1 t3_1
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1 t2_1
+                                       Filter: (a = 0)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p2_a on prt1_p2 t3_2
+                           Index Cond: (a = t2_2.b)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t3_3
+                           Index Cond: (a = t2_3.b)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_4.a = t2_4.b)
+                           ->  Seq Scan on prt1_p4 t3_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t2_4
+                                       Filter: (a = 0)
+(47 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+  a   | a |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.b
+   ->  Hash Right Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p5 t2_5
+                           Filter: (a = 0)
+(24 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: (t1.a = t2.b)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.a, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
+                     Filter: (a = 0)
+(22 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
 --
 -- partitioned by multiple columns
 --
@@ -1125,82 +2433,1552 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 -- test partition matching with N-way join
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                   QUERY PLAN                                   
---------------------------------------------------------------------------------
+                                           QUERY PLAN                                           
+------------------------------------------------------------------------------------------------
  GroupAggregate
    Group Key: t1.c, t2.c, t3.c
    ->  Sort
          Sort Key: t1.c, t3.c
          ->  Append
                ->  Hash Join
-                     Hash Cond: (t1.c = ltrim(t3.c, 'A'::text))
+                     Hash Cond: ((t1.c)::text = ltrim(t3.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1.b = t2.b) AND (t1.c = t2.c))
-                           ->  Seq Scan on plt1_p1 t1
+                           Hash Cond: ((t2.b = t1.b) AND ((t2.c)::text = (t1.c)::text))
+                           ->  Seq Scan on plt2_p4 t2
                            ->  Hash
-                                 ->  Seq Scan on plt2_p1 t2
+                                 ->  Seq Scan on plt1_p4 t1
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p1 t3
+                           ->  Seq Scan on plt1_e_p4 t3
                ->  Hash Join
-                     Hash Cond: (t1_1.c = ltrim(t3_1.c, 'A'::text))
+                     Hash Cond: ((t1_1.c)::text = ltrim(t3_1.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
-                           ->  Seq Scan on plt1_p2 t1_1
+                           Hash Cond: ((t1_1.b = t2_1.b) AND ((t1_1.c)::text = (t2_1.c)::text))
+                           ->  Seq Scan on plt1_p1 t1_1
                            ->  Hash
-                                 ->  Seq Scan on plt2_p2 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p2 t3_1
+                           ->  Seq Scan on plt1_e_p1 t3_1
                ->  Hash Join
-                     Hash Cond: (t1_2.c = ltrim(t3_2.c, 'A'::text))
+                     Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
-                           ->  Seq Scan on plt1_p3 t1_2
+                           Hash Cond: ((t1_2.b = t2_2.b) AND ((t1_2.c)::text = (t2_2.c)::text))
+                           ->  Seq Scan on plt1_p2 t1_2
                            ->  Hash
-                                 ->  Seq Scan on plt2_p3 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p3 t3_2
-(32 rows)
+                           ->  Seq Scan on plt1_e_p2 t3_2
+               ->  Hash Join
+                     Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2_3.b = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text))
+                           ->  Seq Scan on plt2_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p3 t3_3
+(41 rows)
 
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  75.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 123.0000000000000000 | 123.0000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 225.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 273.0000000000000000 | 273.0000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 375.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 423.0000000000000000 | 423.0000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 525.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 573.0000000000000000 | 573.0000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p5 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_3
+                                 ->  Seq Scan on plt2_p3 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Seq Scan on plt2_p2 t2_2
+                                 ->  Seq Scan on plt2_p3 t2_3
+(22 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1225,22 +4003,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
 --------------------------------------------------
  Hash Left Join
    Hash Cond: (t2.b = a)
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t3.a = t2.b)
-               ->  Seq Scan on prt1_p1 t3
-               ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
-         ->  Hash Join
-               Hash Cond: (t3_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t3_1
-               ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
-         ->  Hash Join
-               Hash Cond: (t3_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t3_2
-               ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
+   ->  Hash Join
+         Hash Cond: (t3.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t3
+               ->  Seq Scan on prt1_p1 t3_1
+               ->  Seq Scan on prt1_p2 t3_2
+               ->  Seq Scan on prt1_p3 t3_3
+               ->  Seq Scan on prt1_p4 t3_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
    ->  Hash
          ->  Result
                One-Time Filter: false
@@ -1255,16 +4033,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 --
 -- tests for hash partitioned tables.
@@ -1340,41 +4124,9 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  273.0000000000000000 | 273.0000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
 (6 rows)
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
- Sort
-   Sort Key: t1.a
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
-               ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
-               ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
-                           Filter: (b = 0)
-(21 rows)
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
@@ -1389,26 +4141,24 @@ SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c
    Sort Key: t1.c
    ->  HashAggregate
          Group Key: t1.c, t2.c
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t2.c = t1.c)
-                     ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p1 t1
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on plt1_p4 t1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_1.c = t1_1.c)
-                     ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_p2 t1_1
+                           ->  Seq Scan on plt1_p1 t1_1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_2.c = t1_2.c)
-                     ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Seq Scan on plt1_p2 t1_2
                                  Filter: ((a % 25) = 0)
-(23 rows)
+                           ->  Seq Scan on plt1_p3 t1_3
+                                 Filter: ((a % 25) = 0)
+(21 rows)
 
 --
 -- multiple levels of partitioning
@@ -1804,64 +4554,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
  Hash Join
    Hash Cond: (t1.a = t2.a)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt4_n_p1 t2
                ->  Seq Scan on prt4_n_p2 t2_1
                ->  Seq Scan on prt4_n_p3 t2_2
-(11 rows)
+(13 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a and t1.a = t3.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                        QUERY PLAN                        
+----------------------------------------------------------
  Hash Join
-   Hash Cond: (t2.a = t1.a)
+   Hash Cond: (t3.b = t1.a)
    ->  Append
-         ->  Seq Scan on prt4_n_p1 t2
-         ->  Seq Scan on prt4_n_p2 t2_1
-         ->  Seq Scan on prt4_n_p3 t2_2
+         ->  Seq Scan on prt2_p0 t3
+         ->  Seq Scan on prt2_p1 t3_1
+         ->  Seq Scan on prt2_p2 t3_2
+         ->  Seq Scan on prt2_p3 t3_3
+         ->  Seq Scan on prt2_p4 t3_4
+         ->  Seq Scan on prt2_p5 t3_5
    ->  Hash
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.a = t3_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_2.b)
-                     ->  Seq Scan on prt1_p3 t1_2
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3 t3_2
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt4_n_p1 t2
+                           ->  Seq Scan on prt4_n_p2 t2_1
+                           ->  Seq Scan on prt4_n_p3 t2_2
 (23 rows)
 
 -- partitionwise join can not be applied if there are no equi-join conditions
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (b > t1.a)
-(12 rows)
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+(16 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partitionwise join
@@ -1947,16 +4703,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
-                     ->  Seq Scan on plt1_p1 t3
-                     ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3
+                     ->  Seq Scan on plt1_p1 t3_1
+                     ->  Seq Scan on plt1_p2 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partitionwise join can not be applied for a join between list and range
 -- partitioned table
@@ -1967,14 +4724,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
  Hash Full Join
    Hash Cond: ((t2.c)::text = (t1.c)::text)
    ->  Append
-         ->  Seq Scan on prt1_p1 t2
-         ->  Seq Scan on prt1_p2 t2_1
-         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p0 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p2 t2_2
+         ->  Seq Scan on prt1_p3 t2_3
+         ->  Seq Scan on prt1_p4 t2_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt1_n_p1 t1
                ->  Seq Scan on prt1_n_p2 t1_1
-(10 rows)
+(12 rows)
 
 -- partitionwise join can not be applied if only one of joining table has
 -- default partition
@@ -1990,16 +4749,23 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
    ->  Hash Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index db9a6b4a96..7597b069fa 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -10,25 +10,39 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -69,11 +83,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -95,20 +117,30 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 
 EXPLAIN (COSTS OFF)
@@ -172,6 +204,128 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -196,28 +350,79 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
 -- test partition matching with N-way join
@@ -225,6 +430,175 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
@@ -270,27 +644,18 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
 ALTER TABLE plt2 DETACH PARTITION plt2_p3;
 ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
 ANALYZE plt2;
-
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.a % 25 = 0 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
 --
 -- multiple levels of partitioning
 --
-- 
2.18.0

#72Rajkumar Raghuwanshi
rajkumar.raghuwanshi@enterprisedb.com
In reply to: amul sul (#71)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Wed, Apr 24, 2019 at 4:56 PM amul sul <sulamul@gmail.com> wrote:

Attached version is rebase atop of the latest master head(fdc7efcc30), also
incorporates the Ashutosh's suggestion, thanks.

Thanks for rebase patch, patches applied cleanly on PG head.
I did some crash testing with extra test case [0006 patch] [1]/messages/by-id/CAFjFpReKuV_4LRRfdy80BqX8oZfwbo+HWLQNv3CsJ5iGPSyTfA@mail.gmail.com </messages/by-id/CA+q6zcU3X4=BfqnWXAUPBFtKK7vy0HO7-+mAW6KB2Zy_EPtC_Q@mail.gmail.com&gt; and found no
more issue.

Thanks & Regards,
Rajkumar Raghuwanshi
QMG, EnterpriseDB.

[1]: /messages/by-id/CAFjFpReKuV_4LRRfdy80BqX8oZfwbo+HWLQNv3CsJ5iGPSyTfA@mail.gmail.com </messages/by-id/CA+q6zcU3X4=BfqnWXAUPBFtKK7vy0HO7-+mAW6KB2Zy_EPtC_Q@mail.gmail.com&gt;
/messages/by-id/CAFjFpReKuV_4LRRfdy80BqX8oZfwbo+HWLQNv3CsJ5iGPSyTfA@mail.gmail.com
</messages/by-id/CA+q6zcU3X4=BfqnWXAUPBFtKK7vy0HO7-+mAW6KB2Zy_EPtC_Q@mail.gmail.com&gt;

Show quoted text

Regards,
Amul

On Mon, Mar 11, 2019 at 10:14 PM Ashutosh Bapat <
ashutosh.bapat.oss@gmail.com> wrote:

On Mon, Mar 11, 2019 at 10:40 AM amul sul <sulamul@gmail.com> wrote:

All the places from where this handle_missing_partition() get called
have the following code to decide the value for
missing_side_outer/_inner which
I yet to understand. Do you think this has some flaw?

/*
* For a FULL join, inner relation acts as both OUTER and INNER
* relation. For LEFT and ANTI join the inner relation acts as
* INNER relation. For INNER and SEMI join OUTER and INNER
* differentiation is immaterial.
*/
missing_side_inner = (jointype == JOIN_FULL ||
jointype == JOIN_LEFT ||
jointype == JOIN_ANTI);
missing_side_outer = (jointype == JOIN_FULL);

I was wrong, sorry. The comment says it all.

argument value which fails to set merged_index.

In the attached patch, I tried to fix this case by setting
merged_index
explicitly which fixes the reported crash.

I expect handle_missing_partition() to set the merged_index always. In
your patches, I don't see that function in your patches is setting it
explicitly. If we are setting merged_index explicitly somewhere else, other
places may miss that explicit assignment. So it's better to move it inside
this function.

Ok, that can be fixed.

Similarly, I think merge_null_partitions should set null_index instead
of
asserting when null partitions missing from both the side, make sense?

I think not. null_index, once set shouldn't change and hence does not
change with each pair of partitions being matched. So, it makes sense to
make sure that null_index remains invalid if none of the tables have null
partition.

--
Best Wishes,
Ashutosh Bapat

#73Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: amul sul (#71)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Hi Amul, Ashutosh,

On 2019/04/24 20:26, amul sul wrote:

Attached version is rebase atop of the latest master head(fdc7efcc30), also
incorporates the Ashutosh's suggestion, thanks.

Reading the commit message of 0002 and after testing to confirm, I
understand that the patch doesn't handle OUTER joins where the nullable
side is missing some partitions. The reason given is that join planning
would have to add base relations corresponding to missing partitions on
the nullable side, which we can't do. While working on partition pruning
refactoring recently (committed as 428b260f87e), we faced a similar
situation in that pruned partitions are like missing partitions, because
they're not added to the PlannerInfo anymore, whereas before that commit,
they'd be added and marked dummy afterwards. Earlier versions of my patch
had code to add dummy base relations for such pruned partitions, because
partitionwise join expected pairs of matched partitions to be valid base
relations, because that's how things were when partitionwise joins feature
was committed. Join path generated in this case would have a
constant-false Result path (an empty relation) for the nullable side. Tom
strongly objected to that idea saying that such join paths are kind of
silly [1]/messages/by-id/25035.1553905052@sss.pgh.pa.us, even outside the context of partitionwise join. He suggested
that we abandon partitionwise join in such cases, because having to build
a dummy base relation for pruned partitions only to generate silly-looking
paths would be an ugly kludge. I guess the same argument applies to the
case where the nullable side is missing some partitions, so the right way
to support partitionwise join case in that case wouldn't be to figure out
how joinrels.c could add dummy base relations.

He did mention that cases where the nullable side is provably empty can be
handled by simply returning the path of the non-nullable side with
suitable projection path added on top to emit NULLs for the columns of the
nullable-side. If we teach populate_joinrel_with_paths() and underlings
about that, then we can allow partitionwise join even in the case where
the nullable side has some partitions missing.

Thanks,
Amit

[1]: /messages/by-id/25035.1553905052@sss.pgh.pa.us

#74Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Amit Langote (#73)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On 2019/05/14 13:23, Amit Langote wrote:

Tom
strongly objected to that idea saying that such join paths are kind of
silly [1], even outside the context of partitionwise join. He suggested
that we abandon partitionwise join in such cases, because having to build
a dummy base relation for pruned partitions only to generate silly-looking
paths would be an ugly kludge.

I forgot to mention that he even committed a patch to disable
partitionwise joins in such cases, which was also applied to v11 branch.

https://git.postgresql.org/gitweb/?p=postgresql.git;a=commitdiff;h=d70c147fa217c4bae32ac1afb86ab42d98b36fdf

Note that there were also other reasons for committing, beside what I
described in my previous email.

Thanks,
Amit

#75Ashutosh Bapat
ashutosh.bapat.oss@gmail.com
In reply to: Amit Langote (#74)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Tue, May 14, 2019 at 10:00 AM Amit Langote <Langote_Amit_f8@lab.ntt.co.jp>
wrote:

On 2019/05/14 13:23, Amit Langote wrote:

Tom
strongly objected to that idea saying that such join paths are kind of
silly [1], even outside the context of partitionwise join. He suggested
that we abandon partitionwise join in such cases, because having to build
a dummy base relation for pruned partitions only to generate

silly-looking

paths would be an ugly kludge.

I forgot to mention that he even committed a patch to disable
partitionwise joins in such cases, which was also applied to v11 branch.

https://git.postgresql.org/gitweb/?p=postgresql.git;a=commitdiff;h=d70c147fa217c4bae32ac1afb86ab42d98b36fdf

Note that there were also other reasons for committing, beside what I
described in my previous email.

I haven't seen the actual commit, but we could use these patches to enable
partition-wise join when partitions are pruned. For that the partition
descriptor of the pruned partition table should be arranged as if those
partitions are missing in the table itself. However, we will still need
code to handle the cases when the partitions are missing on the nullable
side. Tom mentioned the idea of using just projection to produce join
tuples with rows on the outer side appended with null columns from the
nullable side. If we can implement that, we can remove the restrictions in
this patch.

--
Best Wishes,
Ashutosh Bapat

#76Robert Haas
robertmhaas@gmail.com
In reply to: Amit Langote (#73)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Tue, May 14, 2019 at 12:24 AM Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

He did mention that cases where the nullable side is provably empty can be
handled by simply returning the path of the non-nullable side with
suitable projection path added on top to emit NULLs for the columns of the
nullable-side. If we teach populate_joinrel_with_paths() and underlings
about that, then we can allow partitionwise join even in the case where
the nullable side has some partitions missing.

Yes, I think that would be a good approach to pursue.

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

#77Thomas Munro
thomas.munro@gmail.com
In reply to: Robert Haas (#76)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Sat, May 18, 2019 at 12:20 AM Robert Haas <robertmhaas@gmail.com> wrote:

On Tue, May 14, 2019 at 12:24 AM Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

He did mention that cases where the nullable side is provably empty can be
handled by simply returning the path of the non-nullable side with
suitable projection path added on top to emit NULLs for the columns of the
nullable-side. If we teach populate_joinrel_with_paths() and underlings
about that, then we can allow partitionwise join even in the case where
the nullable side has some partitions missing.

Yes, I think that would be a good approach to pursue.

Hi Ashutosh, Amul, Fujita-san,

Could we please have a fresh rebase for the new Commitfest?

Thanks,

--
Thomas Munro
https://enterprisedb.com

#78Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Thomas Munro (#77)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Mon, Jul 1, 2019 at 6:50 PM Thomas Munro <thomas.munro@gmail.com> wrote:

On Sat, May 18, 2019 at 12:20 AM Robert Haas <robertmhaas@gmail.com> wrote:

On Tue, May 14, 2019 at 12:24 AM Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

He did mention that cases where the nullable side is provably empty can be
handled by simply returning the path of the non-nullable side with
suitable projection path added on top to emit NULLs for the columns of the
nullable-side. If we teach populate_joinrel_with_paths() and underlings
about that, then we can allow partitionwise join even in the case where
the nullable side has some partitions missing.

Yes, I think that would be a good approach to pursue.

Hi Ashutosh, Amul, Fujita-san,

Could we please have a fresh rebase for the new Commitfest?

Will do unless Ashutosh, Amul, or anyone wants to.

Thanks!

Best regards,
Etsuro Fujita

#79amul sul
sulamul@gmail.com
In reply to: Etsuro Fujita (#78)
3 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Attached version is rebase atop of the latest master head(c74d49d41c),
thanks.

Regards,
Amul

On Mon, Jul 1, 2019 at 4:46 PM Etsuro Fujita <etsuro.fujita@gmail.com>
wrote:

Show quoted text

On Mon, Jul 1, 2019 at 6:50 PM Thomas Munro <thomas.munro@gmail.com>
wrote:

On Sat, May 18, 2019 at 12:20 AM Robert Haas <robertmhaas@gmail.com>

wrote:

On Tue, May 14, 2019 at 12:24 AM Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

He did mention that cases where the nullable side is provably empty

can be

handled by simply returning the path of the non-nullable side with
suitable projection path added on top to emit NULLs for the columns

of the

nullable-side. If we teach populate_joinrel_with_paths() and

underlings

about that, then we can allow partitionwise join even in the case

where

the nullable side has some partitions missing.

Yes, I think that would be a good approach to pursue.

Hi Ashutosh, Amul, Fujita-san,

Could we please have a fresh rebase for the new Commitfest?

Will do unless Ashutosh, Amul, or anyone wants to.

Thanks!

Best regards,
Etsuro Fujita

Attachments:

0001-Hash-partition-bound-equality-refactoring-v22.patchapplication/octet-stream; name=0001-Hash-partition-bound-equality-refactoring-v22.patchDownload
From 68cc5cb996393094dfd2c9cf64a9d0f0b079169b Mon Sep 17 00:00:00 2001
From: Amul Sul <amul.sul@enterprisedb.com>
Date: Mon, 1 Jul 2019 07:06:47 -0400
Subject: [PATCH 1/3] Hash partition bound equality refactoring.

Separate the code to check whether two given hash bounds are equal into a
separate function, so that it can be called from multiple places. Right now
it's only caller is partition_bounds_equal() but later we will use it for
merging partition bounds.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/partitioning/partbounds.c | 95 +++++++++++++++++----------
 1 file changed, 61 insertions(+), 34 deletions(-)

diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index f3c9236ad5..aa863a30e3 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -108,6 +108,9 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 									 Expr **keyCol,
 									 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static bool partition_hbounds_equal(PartitionBoundInfo b1,
+									PartitionBoundInfo b2);
+
 
 /*
  * get_qual_from_partbound
@@ -656,6 +659,63 @@ create_range_bounds(PartitionBoundSpec **boundspecs, int nparts,
 	return boundinfo;
 }
 
+/*
+ * Are two hash partition bound collections logically equal?
+ *
+ * Hash partition bounds store modulus and remainder in datums array which are
+ * always integers irrespective of the number of partition keys and their data
+ * types. Hence we can compare the hash bound collection without any partition
+ * key specific information. Separating this logic in a function which does not
+ * require partition key specific information allows it be called from places
+ * where the partition key specific information is not completely available.
+ */
+static bool
+partition_hbounds_equal(PartitionBoundInfo b1, PartitionBoundInfo b2)
+{
+	int			greatest_modulus = get_hash_partition_greatest_modulus(b1);
+	int			i;
+
+	Assert(b1->strategy == PARTITION_STRATEGY_HASH &&
+		   b2->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * If two hash partitioned tables have different greatest moduli,
+	 * their partition schemes don't match.  For hash partitioned table,
+	 * the greatest modulus is given by the last datum and number of
+	 * partitions is given by ndatums.
+	 */
+	if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
+		return false;
+
+	/*
+	 * We arrange the partitions in the ascending order of their modulus and
+	 * remainders.  Also every modulus is factor of next larger modulus.
+	 * Therefore we can safely store index of a given partition in indexes
+	 * array at remainder of that partition.  Also entries at (remainder + N *
+	 * modulus) positions in indexes array are all same for (modulus,
+	 * remainder) specification for any partition.  Thus datums array from both
+	 * the given bounds are same, if and only if their indexes array will be
+	 * same.  So, it suffices to compare indexes array.
+	 */
+	for (i = 0; i < greatest_modulus; i++)
+		if (b1->indexes[i] != b2->indexes[i])
+			return false;
+
+#ifdef USE_ASSERT_CHECKING
+
+	/*
+	 * Nonetheless make sure that the bounds are indeed same when the indexes
+	 * match.  Hash partition bound stores modulus and remainder at
+	 * b1->datums[i][0] and b1->datums[i][1] position respectively.
+	 */
+	for (i = 0; i < b1->ndatums; i++)
+		Assert((b1->datums[i][0] == b2->datums[i][0] &&
+				b1->datums[i][1] == b2->datums[i][1]));
+#endif
+
+	return true;
+}
+
 /*
  * Are two partition bound collections logically equal?
  *
@@ -684,41 +744,8 @@ partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
 
 	if (b1->strategy == PARTITION_STRATEGY_HASH)
 	{
-		int			greatest_modulus = get_hash_partition_greatest_modulus(b1);
-
-		/*
-		 * If two hash partitioned tables have different greatest moduli,
-		 * their partition schemes don't match.
-		 */
-		if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
+		if (!partition_hbounds_equal(b1, b2))
 			return false;
-
-		/*
-		 * We arrange the partitions in the ascending order of their moduli
-		 * and remainders.  Also every modulus is factor of next larger
-		 * modulus.  Therefore we can safely store index of a given partition
-		 * in indexes array at remainder of that partition.  Also entries at
-		 * (remainder + N * modulus) positions in indexes array are all same
-		 * for (modulus, remainder) specification for any partition.  Thus
-		 * datums array from both the given bounds are same, if and only if
-		 * their indexes array will be same.  So, it suffices to compare
-		 * indexes array.
-		 */
-		for (i = 0; i < greatest_modulus; i++)
-			if (b1->indexes[i] != b2->indexes[i])
-				return false;
-
-#ifdef USE_ASSERT_CHECKING
-
-		/*
-		 * Nonetheless make sure that the bounds are indeed same when the
-		 * indexes match.  Hash partition bound stores modulus and remainder
-		 * at b1->datums[i][0] and b1->datums[i][1] position respectively.
-		 */
-		for (i = 0; i < b1->ndatums; i++)
-			Assert((b1->datums[i][0] == b2->datums[i][0] &&
-					b1->datums[i][1] == b2->datums[i][1]));
-#endif
 	}
 	else
 	{
-- 
2.18.0

0002-Partition-wise-join-for-1-1-1-0-0-1-partition-matchi-v22.patchapplication/octet-stream; name=0002-Partition-wise-join-for-1-1-1-0-0-1-partition-matchi-v22.patchDownload
From 36b5af88f9ee49c780e19102535b1c9f671b3dac Mon Sep 17 00:00:00 2001
From: Amul Sul <amul.sul@enterprisedb.com>
Date: Mon, 1 Jul 2019 07:10:49 -0400
Subject: [PATCH 2/3] Partition-wise join for 1:1, 1:0, 0:1 partition matching.

Earlier version of partition-wise join implementation allowed
partition-wise join between two relations with exactly same partition
bounds. This commit allows partition-wise join to be applied under
following conditions

1. the partition bounds of joining relations are such that rows from
given partition on one side join can join with rows from maximum one
partition on the other side i.e. bounds of a given partition on one
side match/overlap with those of maximum one partition on the other
side. If the mapping happens to be m:n where m > 1 or n > 1, we have
to gang multiple partition relations together into a single relation.
This means that we have to add simple relations during join
processing, something which is not supported right now.  ALso, in such
a case, different pairs of joining relations can produce different
partition bounds for the same join relation, which again is not
supported right now.

2. For every partition on outer side that can contribute to the result
of an OUTER join, there exists at least one (taken along with item 1,
it means exactly one)  matching partition on the inner side. To
support partition-wise join when the inner matching partition doesn't
exist, we have to add a dummy base relation corresponding to the
non-existent inner partition. We don't have support to add base
relations during join processing.

3. For hash partitioned tables however, we support partition-wise join
only when the bounds exactly match. For hash partitioning it's unusual
to have missing partitions and hence generic partition matching is not
required.

With advanced partition matching, we won't be able to apply
partition-wise join when there are missing matching partitions. So for
a given N-way join, some sub-joins may not be partitioned, while the
overall N-way join is partitioned. We can not try partition-wise join
when one or both of the joining relations are not partitioned in one
of the pairs of joining relation. Skip partition-wise join for that
pair.

An example is A IJ B LJ C where B has an extra partition compared to A
and C. Join B LJ C requires dummy relation to join the extra partition
from B to a missing partition from C, and hence can not use
partition-wise join right now and hence BC is not partitioned.  The
extra partition in B gets eliminated in A IJ B and thus it can use
partition-wise join and is partitioned. Hence (AB)C can use
partition-wise join but not A(BC).

Not every pair of joining relation (including the one presented to
build_joinrel_partition_info()) for the same joinrel can use
partition-wise join or has both the relations partitioned. Hence we
calculate the partition bounds for the join relation in
try_partition_wise_join() instead of doing it here.

The partition bounds produced for the first pair that can use
partition-wise are saved in the RelOptInfo of the joinrel. Partition
bounds produced for subsequent pairs should match that produced by the
first pair.

Support for default partition in list partitioned tables
========================================================

When a list value is present in one of the joining relations and not
the other, and the other relation has default partition, match (join)
the partition containing that list value with the default partition.
If there are multiple matches, we can not proceed with the
partition-wise join similar to the case of matching non-default
partition. If the default partition happens to be on the outer side of
the join, the resulting join partition acts as a default partition as
it will contain all the values from the default partition. If
the partition containing the list value happens to be on the outer
side of the join, the resulting join partition is associated with the
list value, since no other partition key value from the default
partition makes it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join.  Such a
default partition may get joined to a partition from the inner side,
if inner side has a list value that is not present in the outer side.
But if such a join partner does not exist, we won't be able to use
partition-wise join since there is no (dummy) relation on the inner
side to join to.

Support for matching default partition in range partitioned tables
==================================================================

When a range is present in one of the joining relations and not the
other, and the other relation has default partition, match (join) the
partition corresponding range with the default partition. If two
ranges overlap but their ranges don't match exactly, the
non-overlapping portion from one range may join the previous
(non-overlapping portion near the lower bound, if exists), next
(non-overlapping portion near the upper bound, if exists) ranges from
the other relation or default partition from the other relation. This
causes multiple partitions on the other side to be joined with a
single partition on the first side; a case we don't support. Any range
from one side which doesn't overlap with any range on the other side,
may find its join partner in the default partition and the
corresponding range partition will need to be joined with the default
partition.  If the default partition happens to be on the outer side
of the join, the resulting join partition acts as a default partition
as it will contain all the values from the default partition. If the
non-partition corresponding to the range happens to be on the outer
side of the join, the resulting join partition is associated with that
range, since partition key values from the default partition outside
that range won't make it to the join result.

If both the relations have default partition, match (join) the default
partition with each other and deem the resulting join partition as
default partition. If one of the relations has default partition but
not the other, and the default partition happens to be on the outer
side of the join, all its rows will make it to the join.  Such a
default partition may get joined to a non-default partition from the
inner side, if inner side has a range missing in the outer side.  But
if such a join partner does not exist, we won't be able to use
partition-wise join since there is no (dummy) relation on the inner
side to join to.

Ashutosh Bapat, reviewed by Dmitry Dolgov
---
 src/backend/optimizer/path/joinrels.c |  103 +-
 src/backend/optimizer/util/relnode.c  |   33 +-
 src/backend/partitioning/partbounds.c | 1682 ++++++++++++++++++++++++-
 src/include/partitioning/partbounds.h |    7 +
 4 files changed, 1775 insertions(+), 50 deletions(-)

diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 43c3b7ea48..4b28e890ab 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1339,25 +1339,31 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	PartitionScheme part_scheme;
+	PartitionBoundInfo join_boundinfo;
+	List	   *parts1;
+	List	   *parts2;
+	ListCell   *lc1;
+	ListCell   *lc2;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1370,36 +1376,65 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	 */
 	Assert(joinrel->part_scheme == rel1->part_scheme &&
 		   joinrel->part_scheme == rel2->part_scheme);
+	part_scheme = joinrel->part_scheme;
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * Get the list of matching partitions to be joined along with the
+	 * partition bounds of the join relation. Because of the restrictions
+	 * imposed by partition matching algorithm, not every pair of joining
+	 * relations for this join will be able to use partition-wise join. But all
+	 * those pairs which can use partition-wise join will produce the same
+	 * partition bounds for the join relation.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	join_boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											part_scheme->partsupfunc,
+											part_scheme->partcollation,
+											rel1, rel2,
+											parent_sjinfo->jointype,
+											&parts1, &parts2);
+
+	if (join_boundinfo == NULL)
+		return;
 
-	nparts = joinrel->nparts;
+	if (joinrel->boundinfo == NULL)
+	{
+		Assert(joinrel->nparts == 0 && joinrel->part_rels == NULL);
+		joinrel->boundinfo = join_boundinfo;
+		joinrel->nparts = list_length(parts1);
+		Assert(joinrel->nparts == list_length(parts2));
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) *
+									joinrel->nparts);
+	}
+	else
+	{
+		Assert(partition_bounds_equal(part_scheme->partnatts,
+									  part_scheme->parttyplen,
+									  part_scheme->parttypbyval,
+									  join_boundinfo, joinrel->boundinfo));
+		/*
+		 * Every pair of joining relations should result in the same number
+		 * of child-joins.
+		 */
+		Assert(joinrel->nparts == list_length(parts1));
+		Assert(joinrel->nparts == list_length(parts2));
+		Assert(joinrel->part_rels);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	cnt_parts = -1;
+	forboth(lc1, parts1, lc2, parts2)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
-		bool		rel1_empty = (child_rel1 == NULL ||
-								  IS_DUMMY_REL(child_rel1));
-		bool		rel2_empty = (child_rel2 == NULL ||
-								  IS_DUMMY_REL(child_rel2));
+		int			part1 = lfirst_int(lc1);
+		int			part2 = lfirst_int(lc2);
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		bool		rel1_empty;
+		bool		rel2_empty;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1407,6 +1442,13 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		Assert(part1 >= 0 && part2 >= 0);
+		child_rel1 = rel1->part_rels[part1];
+		child_rel2 = rel2->part_rels[part2];
+		rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1));
+		rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2));
+		cnt_parts++;
+
 		/*
 		 * Check for cases where we can prove that this segment of the join
 		 * returns no rows, due to one or both inputs being empty (including
@@ -1506,12 +1548,23 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 			joinrel->part_rels[cnt_parts] = child_joinrel;
 		}
 
+		/*
+		 * For every pair of joining relations, the set of matching partitions
+		 * would change. However, the base relation partitions constituting
+		 * the given child should remain same for all the joining pairs. Since
+		 * the order in which children are stored in the array of child-joins,
+		 * depends upon partition bounds of the join, which are same for all
+		 * the joining pairs, every joining pair yields the child-joins in the
+		 * same order.
+		 */
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
 
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
 	}
+
+	Assert(cnt_parts == (joinrel->nparts - 1));
 }
 
 /*
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 6054bd2b53..ead871180f 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1623,7 +1623,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1636,24 +1636,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1665,17 +1647,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index aa863a30e3..c42fe88724 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -25,6 +25,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/pathnodes.h"
 #include "parser/parse_coerce.h"
 #include "partitioning/partbounds.h"
 #include "partitioning/partdesc.h"
@@ -69,6 +70,12 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+typedef struct PartitionMap
+{
+	int from;
+	int to;
+} PartitionMap;
+
 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);
@@ -110,7 +117,58 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 static List *get_range_nulltest(PartitionKey key);
 static bool partition_hbounds_equal(PartitionBoundInfo b1,
 									PartitionBoundInfo b2);
-
+static PartitionBoundInfo partition_range_bounds_merge(
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 List **outer_parts, List **inner_parts,
+							 JoinType jointype, int partnatts,
+							 FmgrInfo *supfuncs, Oid *collations);
+static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype);
+static PartitionBoundInfo partition_hash_bounds_merge(RelOptInfo *outer_rel,
+							RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype);
+static void generate_matching_part_pairs(PartitionMap *outer_maps,
+										 PartitionMap *inner_maps,
+										 int nparts1, int nparts2,
+										 JoinType jointype, int nparts,
+										 List **parts1, List **parts2);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int map_and_merge_partitions(PartitionMap *outer_maps,
+										PartitionMap *inner_maps,
+										int index1, int index2, int *next_index);
+static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *collations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2);
+static bool partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
+						   Oid *collations, PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, int *ub_cmpval,
+						   int *lb_cmpval);
+static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes);
+static bool merge_default_partitions(PartitionBoundInfo outer_bi,
+						 PartitionBoundInfo inner_bi,
+						 PartitionMap *outer_maps,
+						 PartitionMap *inner_maps,
+						 JoinType jointype,
+						 int *next_index, int *default_index);
+static bool merge_null_partitions(PartitionBoundInfo outer_bi,
+								  PartitionBoundInfo inner_bi,
+								  PartitionMap *outer_maps,
+								  PartitionMap *inner_maps,
+								  JoinType jointype,
+								  int *next_index, int *null_index,
+								  int *default_index);
 
 /*
  * get_qual_from_partbound
@@ -3021,3 +3079,1625 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * The function produces the partition bounds for a join between two relations
+ * whose partition bounds are given. The function also returns two lists of
+ * partition indexes one for each of the joining relations. Both the lists
+ * contain the same number of elements. The partition indexes at the same
+ * positions in the lists indicate the pair partitions, one from each side, to
+ * be joined and the position itself corresponds to the index of partition
+ * produced by that child-join in the partitioned join.
+ *
+ * The function returns NULL if we can not find the matching pair of
+ * partitions. This happens if 1. multiple partitions on one side match with
+ * one partition on the other side. 2. a given partition on the outer side
+ * doesn't have a matching partition on the inner side. We can not support the
+ * first case since we don't have a way to represent multiple partitions as a
+ * single relation (RelOptInfo) and then perform join using the ganged
+ * relation. We can not support the second case since the missing inner
+ * partition needs to be represented as an empty relation and we don't have a
+ * way to introduce empty relation during join planning after creating paths
+ * for all the base relations.
+ */
+extern PartitionBoundInfo
+partition_bounds_merge(int partnatts, FmgrInfo *partsupfunc,
+					   Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
+{
+	PartitionBoundInfo 	merged_bounds;
+	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
+						inner_binfo = inner_rel->boundinfo;
+	char				strategy = outer_binfo->strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partsupfunc,
+														partcollation,
+														outer_rel, inner_rel,
+														outer_parts, inner_parts,
+														jointype);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(outer_rel, inner_rel,
+														 outer_parts, inner_parts,
+														 jointype, partnatts,
+														 partsupfunc,
+														 partcollation);
+			break;
+
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = partition_hash_bounds_merge(outer_rel, inner_rel,
+														outer_parts, inner_parts,
+														jointype);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	Assert((*outer_parts == NIL || *inner_parts != NIL) &&
+		   (*inner_parts == NIL || *outer_parts != NIL));
+
+	return merged_bounds;
+}
+
+/*
+ * partition_get_range_bounds
+ *
+ * Given the index of lower bound in datums array, return lower and upper
+ * bounds and the index of the partition with that lower bound.
+ */
+static int
+partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
+						   PartitionRangeBound *lower,
+						   PartitionRangeBound *upper)
+{
+	int			part_index;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The lower bound should correspond to a valid partition. */
+	part_index = bi->indexes[lb_index + 1];
+	Assert(part_index >= 0);
+
+	lower->kind = bi->kind[lb_index];
+	lower->datums = bi->datums[lb_index];
+	lower->lower = true;
+	upper->kind = bi->kind[lb_index + 1];
+	upper->datums = bi->datums[lb_index + 1];
+	upper->lower = false;
+
+	return part_index;
+}
+
+/*
+ * partition_range_get_next_lb_index
+ *
+ * Given the index of lower bound in datums array return the
+ * index of lower bound of the next partition. When the given index corresponds
+ * to the last partition, return number of datums (ndatums).
+ */
+static int
+partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
+{
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The partition index corresponding to the upper bound should be valid. */
+	Assert(bi->indexes[lb_index + 1] >= 0);
+
+	/*
+	 * If there are no bounds left beyond the upper bound, we have reached the
+	 * last partition.
+	 */
+	if (lb_index + 2 < bi->ndatums)
+	{
+		/*
+		 * If the bound next to the upper bound corresponds to no partition,
+		 * that's the next lower bound of the next partition. Otherwise, the
+		 * current upper bound is the lower bound of the next partition.
+		 */
+		if (bi->indexes[lb_index + 2] < 0)
+			return lb_index + 2;
+		else
+			return lb_index + 1;
+	}
+	else
+		return bi->ndatums;
+}
+
+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *partcollations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2)
+{
+	return partition_rbound_cmp(partnatts, partsupfunc, partcollations,
+								bound1->datums, bound1->kind, bound1->lower,
+								bound2);
+}
+
+/*
+ * partition_range_cmp
+ *
+ * Compare the bounds of two range partitions. Set ub_cmpval <, = or > 0, if the
+ * first partition's upper bound is lower than, equal to or higher than the
+ * second partition's upper bound resp. Similarly set lb_cmpval <, =  or > 0,
+ * if the first partition's lower bound is lower than, equal to or higher than
+ * the second partition's lower bound resp.
+ *
+ * Return true, if the ranges overlap, otherwise return false.
+ */
+static bool
+partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
+					PartitionRangeBound *lower_bound1,
+					PartitionRangeBound *upper_bound1,
+					PartitionRangeBound *lower_bound2,
+					PartitionRangeBound *upper_bound2, int *ub_cmpval,
+					int *lb_cmpval)
+{
+	bool		overlap;
+
+	/*
+	 * Compare upper bound of the first partition with the lower bound of the
+	 * second and vice-versa. If lower bound is higher than the upper bound,
+	 * the partitions are not overlapping. All other cases indicate overlapping
+	 * partitions.
+	 */
+	if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+								  lower_bound1, upper_bound2) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = 1;
+		*lb_cmpval = 1;
+	}
+	else if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+									   lower_bound2, upper_bound1) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = -1;
+		*lb_cmpval = -1;
+	}
+	else
+	{
+		overlap = true;
+		*ub_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, upper_bound1,
+											   upper_bound2);
+		*lb_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, lower_bound1,
+											   lower_bound2);
+	}
+
+	return overlap;
+}
+
+/*
+ * partition_range_merge
+ *
+ * Merge the partition bounds of given two partitions such that the join
+ * between the given two partitions fits merged bounds.
+ *
+ * "merged_upper" will be set to one of the given upper bounds and
+ * "merged_lower" will be set to one of the given lower bounds.
+ */
+static void
+partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
+					  Oid *partcollations, JoinType jointype,
+					  PartitionRangeBound *left_lb,
+					  PartitionRangeBound *left_ub,
+					  PartitionRangeBound *right_lb,
+					  PartitionRangeBound *right_ub,
+					  PartitionRangeBound **merged_lb,
+					  PartitionRangeBound **merged_ub)
+{
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = left_ub;
+			*merged_lb = left_lb;
+			break;
+
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) < 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) > 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		case JOIN_FULL:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) > 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) < 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+
+	return;
+}
+
+/*
+ * Add the lower bound of the next range to the list of bounds, if the lower
+ * bound is higher or equal to the previous upper bound. If successful return
+ * true, otherwise false.
+ */
+static bool
+partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
+							  Oid *partcollations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		prev_ub.datums = llast(*merged_datums);
+		prev_ub.kind = llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  next_lb_datums, next_lb_kind, false,
+									  &prev_ub);
+	}
+
+	/*
+	 * The lower bound is lower than the last upper bound, thus does not fit
+	 * the bounds created so far and hence can not be merged with the existing
+	 * bounds.
+	 */
+	if (cmpval < 0)
+		return false;
+
+	/*
+	 * Add bounds of the new merged partition. If the next lower bound is
+	 * higher than the last upper bound, add new range with index
+	 * corresponding to the lower bound as -1. If the merged lower bound
+	 * is same as the last merged upper bound, the last upper bound will be
+	 * reused as the lower bound of the next range.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, next_lb_datums);
+		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	return true;
+}
+
+/*
+ * handle_missing_partition
+ *
+ * If a range appears in one of the joining relations but not the other, a row
+ * in the corresponding partition will not have any join partner in the other
+ * relation, unless the other relation has a default partition. If a given list
+ * value is present in one joining relation but not the other, the default
+ * partition on the other side may contain that value.
+ *
+ * In both these cases, such an extra partition forms a joining pair with the
+ * default partition, if any,  on the other side.
+ *
+ * If the default partition happens to be on the outer side of the join, the
+ * resultant partition will act as the default partition of the join relation.
+ * Otherwise the resultant partition will be associated with the range.
+ *
+ * When the default partition is not present in the other relation, the rows in
+ * the extra partition will be included in the bounds of the join result, if it
+ * appears on the outer side of the join, since all rows from the outer side
+ * are included in the join result.
+ *
+ * This function handles all these cases.
+ *
+ * maps_with_missing and missing_side_default are the partition maps (See
+ * partition_range/list_bounds_merge() for details) and the index of default
+ * partition respectively corresponding the side with missing partition.
+ *
+ * maps_with_extra and extra_part are the partition maps (See
+ * partition_range/list_bounds_merge() for details) and the index of extra
+ * partition respectively corresponding to the side with the extra partition.
+ *
+ * It returns true if the matching succeeds, otherwise returns false.
+ */
+static bool
+handle_missing_partition(PartitionMap *maps_with_missing,
+						 PartitionMap *maps_with_extra,
+						 int missing_side_default,
+						 int extra_part,
+						 bool missing_side_outer,
+						 bool missing_side_inner,
+						 int *next_index, int *default_index,
+						 int *merged_index)
+{
+	bool missing_has_default = (missing_side_default != -1);
+
+	if (missing_has_default)
+	{
+		*merged_index = map_and_merge_partitions(maps_with_missing,
+												 maps_with_extra,
+												 missing_side_default,
+												 extra_part,
+												 next_index);
+		if (*merged_index < 0)
+			return false;
+
+		if (missing_side_outer)
+		{
+			/*
+			 * Default partition on the outer side forms the default
+			 * partition of the join result.
+			 */
+			if (*default_index < 0)
+				*default_index = *merged_index;
+			else if(*default_index != *merged_index)
+			{
+				/*
+				 * Ended up with default partition on the outer side
+				 * being joined with multiple partitions on the inner
+				 * side. We don't support this case.
+				 */
+				return false;
+			}
+
+			/*
+			 * Since the merged partition acts as a default partition, it
+			 * doesn't need a separate index.
+			 */
+			*merged_index = -1;
+		}
+	}
+	else if (missing_side_inner)
+	{
+		/*
+		 * If this partition has already been mapped (say because we
+		 * found an overlapping range earlier), we know where does it
+		 * fit in the join result. Nothing to do in that case. Else
+		 * create a new merged partition.
+		 */
+		PartitionMap *extra_map = &maps_with_extra[extra_part];
+		if (extra_map->to < 0)
+		{
+			extra_map->to = *next_index;
+			*next_index = *next_index + 1;
+			*merged_index = extra_map->to;
+		}
+	}
+	else
+		*merged_index = -1;
+
+	return true;
+}
+
+static PartitionMap*
+init_partition_map(RelOptInfo *rel)
+{
+	int i, nparts = rel->nparts;
+	PartitionMap *map;
+
+	map = (PartitionMap *) palloc(sizeof(PartitionMap) * nparts);
+
+	for (i = 0; i < nparts; i++)
+	{
+		map[i].from = -1;
+		map[i].to = -1;
+	}
+
+	return map;
+}
+
+/*
+ * Allocate and initialize partition maps. We maintain four maps, two maps
+ * for each joining relation. pmap[i] gives the partition from the other
+ * relation which would join with ith partition of the given relation.
+ * Partition i from the given relation will join with partition pmap[i]
+ * from the other relation to produce partition mmap[i] of the join (merged
+ * partition).
+ *
+ * pmap[i] = -1 indicates that ith partition of a given relation does not
+ * have a matching partition from the other relation.
+ *
+ * mmap[i] = -1 indicates that ith partition of a given relation does not
+ * contribute to the join result. That can happen only when the given
+ * relation is the inner relation and it doesn't have a matching partition
+ * from the outer relation, hence pmap[i] should be -1.
+ *
+ * In case of an outer join, every partition of the outer join will appear
+ * in the join result, and thus has mmap[i] set for all i. But it's not
+ * necessary that every partition on the outer side will have a matching
+ * partition on the inner side. In such a case, we end up with pmap[i] = -1
+ * and mmap[i] != -1.
+ */
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 List **outer_parts, List **inner_parts,
+							 JoinType jointype, int partnatts,
+							 FmgrInfo *partsupfuncs, Oid *partcollations)
+
+{
+	PartitionMap *outer_maps = NULL;
+	PartitionMap *inner_maps = NULL;
+	int			outer_part = 0;
+	int			inner_part = 0;
+	PartitionBoundInfo merged_bounds = NULL;
+	int			outer_lb_index;
+	int			inner_lb_index;
+	int			next_index;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	List	   *merged_kinds = NIL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			inner_default = inner_bi->default_index;
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	outer_maps = init_partition_map(outer_rel);
+	inner_maps = init_partition_map(inner_rel);
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = 0;
+	inner_lb_index = 0;
+	next_index = 0;
+	while (outer_lb_index < outer_bi->ndatums ||
+		   inner_lb_index < inner_bi->ndatums)
+	{
+		PartitionRangeBound outer_lb, outer_ub,
+							inner_lb, inner_ub,
+							*merged_lb = NULL,
+							*merged_ub = NULL;
+
+		int			merged_index = -1;
+		bool		overlap;
+		bool		finished_outer = false;
+		bool		finished_inner = false;
+
+		/* Result of bounds comparison per partition_rbound_cmp(). */
+		int			ub_cmpval;	/* Upper bounds comparison result. */
+		int			lb_cmpval;	/* Lower bounds comparison result. */
+
+		/* Get the range bounds of the next pair of partitions. */
+		if (outer_lb_index < outer_bi->ndatums)
+			outer_part = partition_get_range_bounds(outer_bi, outer_lb_index,
+												&outer_lb, &outer_ub);
+		else
+			finished_outer = true;
+
+		if (inner_lb_index < inner_bi->ndatums)
+			inner_part = partition_get_range_bounds(inner_bi, inner_lb_index,
+												&inner_lb, &inner_ub);
+		else
+			finished_inner = true;
+
+		Assert(!finished_outer || !finished_inner);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (finished_outer)
+		{
+			overlap = false;
+			ub_cmpval = 1;
+			lb_cmpval = 1;
+		}
+		else if (finished_inner)
+		{
+			overlap = false;
+			ub_cmpval = -1;
+			lb_cmpval = -1;
+		}
+		else
+			overlap = partition_range_cmp(partnatts, partsupfuncs, partcollations,
+										  &outer_lb, &outer_ub, &inner_lb,
+										  &inner_ub, &ub_cmpval, &lb_cmpval);
+
+		if (overlap)
+		{
+			/*
+			 * The rows from overlapping portion of ranges on both sides may
+			 * join, hence the corresponding pair of partitions form a joining
+			 * pair. Match them and produce the bounds of the joint partition
+			 * and its index by merging the bounds according to the type of
+			 * join.
+			 */
+			partition_range_merge(partnatts, partsupfuncs, partcollations,
+								  jointype, &outer_lb, &outer_ub, &inner_lb,
+								  &inner_ub, &merged_lb, &merged_ub);
+
+			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+													outer_part, inner_part,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				/* Failed to match the partitions. */
+				return NULL;
+			}
+
+			/*
+			 * If the ranges overlap but don't exactly match, a row from
+			 * non-overlapping portion of the range from one side of join may
+			 * find its join partner in the previous or next overlapping
+			 * partition or default partition on the other side , if such a
+			 * partition exists. All those cases, if true, will cause one
+			 * partition from that side to match at least two partitions on the
+			 * other side; a case that we do not support now. Previous
+			 * partition has been delt with in the previous iteration of this
+			 * loop, next partition will be delt in the next iteration. We will
+			 * deal with the default partition here.
+			 */
+			if ((lb_cmpval < 0 && inner_has_default) ||
+				/* Non-overlapping range on the lower side of outer range. */
+				(lb_cmpval > 0 && outer_has_default) ||
+				/* Non-overlapping range on the lower side of inner range. */
+				(ub_cmpval < 0 && outer_has_default) ||
+				/* Non-overlapping range on the upper side of inner range. */
+				(ub_cmpval > 0 && inner_has_default))
+				/* Non-overlapping range on the upper side of outer range. */
+				return NULL;
+		}
+
+		if (ub_cmpval == 0)
+		{
+			/* Upper bounds of both the ranges match. */
+			Assert(overlap);
+
+			/* Move to the next pair of partitions. */
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the inner side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &outer_lb;
+				merged_ub = &outer_ub;
+
+				/*
+				 * For a FULL join, inner relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the inner relation acts as
+				 * INNER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_inner = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_outer = (jointype == JOIN_FULL);
+				if (!handle_missing_partition(inner_maps,
+											  outer_maps,
+											  inner_default,
+											  outer_part,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the outer side. */
+			Assert(!finished_outer);
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+		}
+		else
+		{
+			Assert(ub_cmpval > 0);
+
+			/* Upper bound of outer range higher than that of the inner. */
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the outer side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &inner_lb;
+				merged_ub = &inner_ub;
+
+				/*
+				 * For a FULL join, outer relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the outer relation acts as
+				 * OUTER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_outer = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_inner = (jointype == JOIN_FULL);
+
+				if (!handle_missing_partition(outer_maps,
+											  inner_maps,
+											  outer_default,
+											  inner_part,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the inner side. */
+			Assert (!finished_inner);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+
+		if (merged_index < 0)
+		{
+			/* We didn't find a new merged partition. */
+			continue;
+		}
+
+		/*
+		 * We have a valid partition index for the next partition of join. The
+		 * partition should have valid range.
+		 */
+		Assert(merged_lb && merged_ub);
+
+		/* Try merging new lower bound with the last upper bound. */
+		if (!partition_range_merge_next_lb(partnatts, partsupfuncs,
+										   partcollations,
+										   merged_lb->datums,
+										   merged_lb->kind, &merged_datums,
+										   &merged_kinds, &merged_indexes))
+			return NULL;
+
+		/* Add upper bound with the merged partition index. */
+		merged_datums = lappend(merged_datums, merged_ub->datums);
+		merged_kinds = lappend(merged_kinds, merged_ub->kind);
+		merged_indexes = lappend_int(merged_indexes, merged_index);
+	}
+
+	if (!merge_default_partitions(outer_bi, inner_bi,
+										  outer_maps, inner_maps,
+										  jointype, &next_index,
+										  &default_index))
+		return NULL;
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_maps, inner_maps,
+								 outer_nparts, inner_nparts,
+								 jointype, next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes,
+													  merged_kinds,
+													  -1, default_index);
+	}
+
+	/* Free any memory we used in this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype)
+{
+	PartitionMap *outer_maps = NULL;
+	PartitionMap *inner_maps = NULL;
+	int			cnto;
+	int			cnti;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			next_index = 0;
+	int			null_index = -1;
+	int			default_index = -1;
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			      *outer_indexes = outer_bi->indexes;
+	int			      *inner_indexes = inner_bi->indexes;
+	int				   outer_default = outer_bi->default_index;
+	int				   inner_default = inner_bi->default_index;
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	outer_maps = init_partition_map(outer_rel);
+	inner_maps = init_partition_map(inner_rel);
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			o_index;
+		int			i_index;
+		int			cmpval;
+		int			merged_index = -1;
+		Datum	   *merged_datum;
+		bool		finished_inner;
+		bool		finished_outer;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+		{
+			finished_outer = true;
+			odatums = NULL;
+			o_index = -1;
+		}
+		else
+		{
+			finished_outer = false;
+			odatums = outer_bi->datums[cnto];
+			o_index = outer_indexes[cnto];
+		}
+
+		if (cnti >= inner_bi->ndatums)
+		{
+			finished_inner = true;
+			idatums = NULL;
+			i_index = -1;
+		}
+		else
+		{
+			finished_inner = false;
+			idatums = inner_bi->datums[cnti];
+			i_index = inner_indexes[cnti];
+		}
+
+		/* If we exhausted both the sides, we won't enter the loop. */
+		Assert(!finished_inner || !finished_outer);
+
+		if (finished_outer)
+			cmpval = 1;
+		else if (finished_inner)
+			cmpval = -1;
+		else
+		{
+			/* Every list datum should map to a valid partition index. */
+			Assert(o_index >= 0 && i_index >= 0 &&
+				   odatums != NULL && idatums != NULL);
+
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0], idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+													o_index, i_index,
+													&next_index);
+
+			if (merged_index < 0)
+				return NULL;
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			/* A datum missing from the inner side. */
+			merged_index = -1;
+			merged_datum = odatums;
+
+			/*
+			 * For a FULL join, inner relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the inner relation acts as
+			 * INNER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_inner = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_outer = (jointype == JOIN_FULL);
+
+			if (!handle_missing_partition(inner_maps,
+										  outer_maps,
+										  inner_default,
+										  o_index,
+										  missing_side_outer,
+										  missing_side_inner,
+										  &next_index,
+										  &default_index,
+										  &merged_index))
+				return NULL;
+
+			/* Move to the next datum on the outer side. */
+			Assert(!finished_outer);
+			cnto++;
+		}
+		else
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			Assert(cmpval > 0);
+
+			/* A datum missing from the outer side. */
+			merged_index = -1;
+			merged_datum = idatums;
+
+			/*
+			 * For a FULL join, outer relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the outer relation acts as
+			 * OUTER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_outer = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_inner = (jointype == JOIN_FULL);
+
+			if (!handle_missing_partition(outer_maps,
+										  inner_maps,
+										  outer_default,
+										  i_index,
+										  missing_side_outer,
+										  missing_side_inner,
+										  &next_index,
+										  &default_index,
+										  &merged_index))
+				return NULL;
+
+			/* Move to the next datum on the right side. */
+			Assert(!finished_inner);
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	if (!merge_null_partitions(outer_bi, inner_bi,
+							   outer_maps, inner_maps,
+							   jointype, &next_index, &null_index,
+							   &default_index))
+		return NULL;
+
+	if (!merge_default_partitions(outer_bi, inner_bi,
+								  outer_maps, inner_maps,
+								  jointype, &next_index,
+								  &default_index))
+		return NULL;
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_maps, inner_maps,
+								 outer_nparts, inner_nparts,
+								 jointype, next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes, NIL,
+													  null_index, default_index);
+	}
+
+	/* Free up all extra memory before returning from this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_bounds_merge()'s arm for hash partitioned tables.
+ *
+ * If the given two hash bounds are same, the function returns the first one
+ * without any change, alongwith the lists of matching partitions. Otherwise it
+ * returns NULL.
+ *
+ * We could try merging the bounds when both the bounds have same greatest
+ * modulii. But there seems to be hardly any requirement for the same.
+ */
+static PartitionBoundInfo
+partition_hash_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype)
+{
+	int			nparts;
+	int			cnt;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * Hash partitioned table does not have explicit NULL accepting partition
+	 * and also does not have a default partition.
+	 */
+	Assert(!partition_bound_has_default(outer_bi) &&
+		   !partition_bound_has_default(inner_bi));
+	Assert(!partition_bound_accepts_nulls(outer_bi) &&
+		   !partition_bound_accepts_nulls(inner_bi));
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+
+	if (outer_nparts != inner_nparts)
+		return NULL;
+	nparts = outer_nparts;
+
+	if (outer_bi->ndatums != inner_bi->ndatums ||
+		!partition_hbounds_equal(outer_bi, inner_bi))
+		return NULL;
+
+	 /*
+	  * Cook up list of matching partitions. Since bounds are exactly same the
+	  * partitions at the same position from both the relations match.
+	  */
+	for (cnt = 0; cnt < nparts; cnt++)
+	{
+		*outer_parts = lappend_int(*outer_parts, cnt);
+		*inner_parts = lappend_int(*inner_parts, cnt);
+	}
+
+	return outer_bi;
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * If the two given partitions (given by index1 and index2 resp.) are
+ * already mapped to each other return the index of corresponding partition in
+ * the merged set of partitions.  If they do not have a merged partition
+ * associated with them, assign a new merged partition index.  If the
+ * partitions are already mapped and their mapped partitions are different from
+ * each other, they can not be merged, so return -1.
+ *
+ * partmaps1[i] gives the mapping of partitions for both relations. It
+ * describes which partition of relation 2 matches ith partition of relation 1,
+ * and which partition in the merged set matches ith partition of relation 1
+ * maps to. Similarly for partmap2.
+ *
+ * index1 and index2 are the indexes of matching partition from respective
+ * relations.
+ *
+ * *next_index is used and incremented when the given partitions require a new
+ * merged partition.
+ */
+
+static int
+map_and_merge_partitions(PartitionMap *partmaps1, PartitionMap *partmaps2,
+						 int index1, int index2, int *next_index)
+{
+	PartitionMap 	*partmap1 = &partmaps1[index1];
+	PartitionMap 	*partmap2 = &partmaps2[index2];
+	int				merged_index;
+
+	/*
+	 * If both the partitions are not mapped to each other, update the
+	 * maps.
+	 */
+	if (partmap1->from < 0 && partmap2->from < 0)
+	{
+		partmap1->from = index2;
+		partmap2->from = index1;
+	}
+
+	/*
+	 * If the given to partitions map to each other, find the corresponding
+	 * merged partition index .
+	 */
+	if (partmap1->from == index2 && partmap2->from == index1)
+	{
+		/*
+		 * If both the partitions are mapped to the same merged partition, get
+		 * the index of merged partition.
+		 */
+		if (partmap1->to == partmap2->to)
+		{
+			merged_index = partmap1->to;
+
+			/*
+			 * If the given two partitions do not have a merged partition
+			 * associated with them, allocate a new merged partition.
+			 */
+			if (merged_index < 0)
+			{
+				merged_index = *next_index;
+				*next_index = *next_index + 1;
+				partmap1->to = merged_index;
+				partmap2->to = merged_index;
+			}
+		}
+
+		/*
+		 * If partition from one relation was mapped to a merged partition but
+		 * not the partition from the other relation, map the same merged
+		 * partition to the partition from other relation, since matching
+		 * partitions map to the same merged partition.
+		 */
+		else if (partmap1->to >= 0 && partmap2->to < 0)
+		{
+			partmap2->to = partmap1->to;
+			merged_index = partmap1->to;
+		}
+		else if (partmap1->to < 0 && partmap2->to >= 0)
+		{
+			partmap1->to = partmap2->to;
+			merged_index = partmap2->to;
+		}
+		else
+		{
+			Assert(partmap1->to != partmap2->to &&
+				   partmap1->to >= 0 && partmap2->to >= 0);
+
+			/*
+			 * Both the partitions map to different merged partitions. This
+			 * means that multiple partitions from one relation matches to one
+			 * partition from the other relation. Partition-wise join does not
+			 * handle this case right now, since it requires ganging multiple
+			 * partitions together (into one RelOptInfo).
+			 */
+			merged_index = -1;
+		}
+	}
+	else
+	{
+		/*
+		 * Multiple partitions from one relation map to one partition from the
+		 * other relation. Partition-wise join does not handle this case right
+		 * now, since it requires ganging multiple partitions together (into
+		 * one RelOptInfo).
+		 */
+		merged_index = -1;
+	}
+
+	return merged_index;
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * partmaps1 map each partition from either side of the join to a merged
+ * partition resp. E.g. partmaps1[i].to gives the merged partition to which ith
+ * partition of first relation maps. Similarly for partmap2. If
+ * partmaps1[i].to == partmaps2[j].to, i and j form the matching pair of
+ * partitions.
+ *
+ * Given these maps this function produces the list pairs of partitions which
+ * when joined produce the merged partitions in the order of merged partition
+ * indexes.
+ *
+ * nparts1 and nparts2 are the number of partitions of the joining relations
+ * resp.
+ *
+ * nparts is the number of merged partitions.
+ *
+ * If successful, the pairs of partitions are returned as two separate lists,
+ * parts1 and parts2 resp., one for each side. Otherwise, those lists will be
+ * set to NIL.
+ */
+static void
+generate_matching_part_pairs(PartitionMap *partmaps1, PartitionMap *partmaps2,
+							 int nparts1, int nparts2,
+							 JoinType jointype, int nparts,
+							 List **matched_parts1, List **matched_parts2)
+{
+	bool		merged = true;
+	int		   *matching1,
+			   *matching2;
+	int 		i;
+	int			max_nparts;
+
+	*matched_parts1 = NIL;
+	*matched_parts2 = NIL;
+
+	matching1 = (int *) palloc(sizeof(int) * nparts),
+	matching2 = (int *) palloc(sizeof(int) * nparts);
+	for (i = 0; i < nparts; i++)
+		matching1[i] = matching2[i] = -1;
+
+	/* Set pairs of matching partitions. */
+	max_nparts = Max(nparts1, nparts2);
+	for (i = 0; i < max_nparts; i++)
+	{
+		if (i < nparts1)
+		{
+			PartitionMap outer_map = partmaps1[i];
+
+			if (outer_map.to >= 0)
+			{
+				Assert(outer_map.to < nparts);
+				matching1[outer_map.to] = i;
+			}
+		}
+
+		if (i < nparts2)
+		{
+			PartitionMap inner_map = partmaps2[i];
+
+			if (inner_map.to >= 0)
+			{
+				Assert(inner_map.to < nparts);
+				matching2[inner_map.to] = i;
+			}
+		}
+	}
+
+	/*
+	 * If we have a partition missing on an inner side, we need to add a dummy
+	 * relation which joins with the outer partition. If the inner relation
+	 * happens to be a base relation, it will require adding a dummy child
+	 * base relation during join processing. Right now, we freeze the base
+	 * relation arrays like PlannerInfo::simple_rte_array after planning for
+	 * base relations. Adding a new (dummy) base relation would require some
+	 * changes to that. So, right now, we do not implement partition-wise join
+	 * in such cases.
+	 */
+	for (i = 0; i < nparts; i++)
+	{
+		int			part1 = matching1[i];
+		int			part2 = matching2[i];
+
+		/* At least one of the partitions should exist. */
+		Assert(part1 >= 0 || part2 >= 0);
+
+		switch (jointype)
+		{
+			case JOIN_INNER:
+			case JOIN_SEMI:
+
+				/*
+				 * An inner or semi join can not return any row when the
+				 * matching partition on either side is missing. We should
+				 * have eliminated all such cases while merging the bounds.
+				 */
+				Assert(part1 >= 0 && part2 >= 0);
+				break;
+
+			case JOIN_LEFT:
+			case JOIN_ANTI:
+				Assert(part1 >= 0);
+				if (part2 < 0)
+					merged = false;
+				break;
+
+			case JOIN_FULL:
+				if (part1 < 0 || part2 < 0)
+					merged = false;
+				break;
+
+			default:
+				elog(ERROR, "unrecognized join type: %d", (int) jointype);
+		}
+
+		if (!merged)
+			break;
+
+		*matched_parts1 = lappend_int(*matched_parts1, part1);
+		*matched_parts2 = lappend_int(*matched_parts2, part2);
+	}
+
+	pfree(matching1);
+	pfree(matching2);
+
+	if (!merged)
+	{
+		list_free(*matched_parts1);
+		list_free(*matched_parts2);
+		*matched_parts1 = NIL;
+		*matched_parts2 = NIL;
+	}
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * Merge default partitions from both sides, if any, and assign the default
+ * partition for the join result, if necessary.
+ *
+ * If both the relations have default partitions, try mapping those to each
+ * other. If the mapping succeeds corresponding merged partition will act as
+ * the default partition of the join result.
+ *
+ * If inner side of the join has default but not the outer side, rows in it
+ * won't appear in the join result. So don't create a default partition. If
+ * outer side of the join has default but not the inner side, rows in it will
+ * appear in the join result, so create a default merged partition.
+ */
+static bool
+merge_default_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+						 PartitionMap *outer_maps, PartitionMap *inner_maps,
+						 JoinType jointype, int *next_index, int *default_index)
+{
+	int				outer_default = outer_bi->default_index;
+	int				inner_default = inner_bi->default_index;
+	bool			outer_has_default = partition_bound_has_default(outer_bi);
+	bool			inner_has_default = partition_bound_has_default(inner_bi);
+	bool			merged = true;
+	PartitionMap 	*outer_default_map = NULL;
+	PartitionMap 	*inner_default_map = NULL;
+
+	if (outer_has_default)
+		outer_default_map = &outer_maps[outer_default];
+
+	if (inner_has_default)
+		inner_default_map = &inner_maps[inner_default];
+
+	if (!outer_has_default && !inner_has_default)
+		Assert(*default_index < 0);
+	else if (outer_default_map != NULL && inner_default_map == NULL)
+	{
+		if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+			jointype == JOIN_ANTI)
+		{
+			if (outer_default_map->to < 0)
+			{
+				outer_default_map->to = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = outer_default_map->to;
+			}
+			else
+				Assert(*default_index == outer_default_map->to);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else if (outer_default_map == NULL && inner_default_map != NULL)
+	{
+		if (jointype == JOIN_FULL)
+		{
+			if (inner_default_map->to < 0)
+			{
+				inner_default_map->to = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = inner_default_map->to;
+			}
+			else
+				Assert(*default_index == inner_default_map->to);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		*default_index = map_and_merge_partitions(outer_maps, inner_maps,
+												  outer_default, inner_default,
+												  next_index);
+
+		if (*default_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
+
+/*
+ * merge_null_partitions
+ *
+ * Merge NULL partitions, i.e. a partition that can hold NULL values for a list
+ * partitioned table, if any. Find the index of merged partition to which the
+ * NULL values would belong in the join result. If one joining relation has a
+ * NULL partition but not the other, try matching it with the default partition
+ * from the other relation since the default partition may have rows with NULL
+ * partition key. We can eliminate a NULL partition when it appears only on the
+ * inner side of the join and the outer side doesn't have a default partition.
+ *
+ * When the equality operator used for join is strict, two NULL values will not
+ * be considered as equal, and thus a NULL partition can be eliminated for an
+ * inner join. But we don't check the strictness operator here.
+ */
+static bool
+merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_maps, PartitionMap *inner_maps,
+					  JoinType jointype, int *next_index,
+					  int *null_index, int *default_index)
+{
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	int			outer_ni = outer_bi->null_index;
+	int			inner_ni = inner_bi->null_index;
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	bool		merged = true;
+
+	if (!outer_has_null && !inner_has_null)
+		Assert(*null_index < 0);
+	else if (outer_has_null && !inner_has_null)
+	{
+		int			merged_index = -1;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, inner relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the inner relation acts as
+		 * INNER relation. For INNER and SEMI join OUTER and INNER
+		 * differentiation is immaterial.
+		 */
+		missing_side_inner = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_outer = (jointype == JOIN_FULL);
+
+		merged = handle_missing_partition(inner_maps,
+										  outer_maps,
+										  inner_default,
+										  outer_ni,
+										  missing_side_outer,
+										  missing_side_inner, next_index,
+										  default_index, &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join to which the outer null partition maps
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = outer_maps[outer_ni].to;
+	}
+	else if (!outer_has_null && inner_has_null)
+	{
+		int			merged_index = -1;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, outer relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the outer relation acts as OUTER
+		 * relation. For INNER and SEMI join OUTER and INNER differentiation is
+		 * immaterial.
+		 */
+		missing_side_outer = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_inner = (jointype == JOIN_FULL);
+		merged = handle_missing_partition(outer_maps,
+										  inner_maps,
+										  outer_default,
+										  inner_ni,
+										  missing_side_outer,
+										  missing_side_inner,
+										  next_index, default_index,
+										  &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join, to which the outer side null partition maps,
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = inner_maps[inner_ni].to;
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		*null_index = map_and_merge_partitions(outer_maps,
+											   inner_maps,
+											   outer_ni,
+											   inner_ni,
+											   next_index);
+		if (*null_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index 8585c29c92..914c3a6c73 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -16,6 +16,7 @@
 #include "nodes/pg_list.h"
 #include "partitioning/partdefs.h"
 #include "utils/relcache.h"
+struct RelOptInfo;				/* avoid including pathnodes.h here */
 
 
 /*
@@ -109,5 +110,11 @@ extern int	partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 										  int nvalues, Datum *values, bool *is_equal);
 extern int	partition_hash_bsearch(PartitionBoundInfo boundinfo,
 								   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   struct RelOptInfo *outer_rel,
+					   struct RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
-- 
2.18.0

0003-Tests-for-0-1-1-1-and-1-0-partition-matching-v22.patchapplication/octet-stream; name=0003-Tests-for-0-1-1-1-and-1-0-partition-matching-v22.patchDownload
From bef2bfabef16fd4ab76d3fb436af4f3fd15eab20 Mon Sep 17 00:00:00 2001
From: Amul Sul <amul.sul@enterprisedb.com>
Date: Wed, 24 Apr 2019 05:28:25 -0400
Subject: [PATCH 3/3] Tests for 0:1, 1:1 and 1:0 partition matching

Rajkumar Raghuvanshi and Ashutosh Bapat.
---
 src/test/regress/expected/partition_join.out | 3944 +++++++++++++++---
 src/test/regress/sql/partition_join.sql      |  427 +-
 2 files changed, 3751 insertions(+), 620 deletions(-)

diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 1296edcdae..d26be918aa 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -8,59 +8,86 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-(4 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+(7 rows)
 
 -- left outer join, with whole-row reference; partitionwise join does not apply
 EXPLAIN (COSTS OFF)
@@ -72,35 +99,50 @@ SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER
    ->  Hash Right Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(22 rows)
 
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-      t1      |      t2      
---------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
- (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
- (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
- (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
- (550,0,0550) | 
-(12 rows)
+       t1       |       t2       
+----------------+----------------
+ (-250,0,-0250) | 
+ (-200,0,-0200) | 
+ (-150,0,-0150) | (0,-150,-0150)
+ (-100,0,-0100) | 
+ (-50,0,-0050)  | 
+ (0,0,0000)     | (0,0,0000)
+ (50,0,0050)    | 
+ (100,0,0100)   | 
+ (150,0,0150)   | (0,150,0150)
+ (200,0,0200)   | 
+ (250,0,0250)   | 
+ (300,0,0300)   | (0,300,0300)
+ (350,0,0350)   | 
+ (400,0,0400)   | 
+ (450,0,0450)   | (0,450,0450)
+ (500,0,0500)   | 
+ (550,0,0550)   | 
+ (600,0,0600)   | (0,600,0600)
+ (650,0,0650)   | 
+ (700,0,0700)   | 
+ (750,0,0750)   | (0,750,0750)
+(21 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
@@ -112,35 +154,53 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
    ->  Append
          ->  Hash Right Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Right Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t2_2.b)
-(20 rows)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+      |       | -225 | -0225
+      |       |  -75 | -0075
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(14 rows)
 
 -- full outer join, with placeholder vars
 EXPLAIN (COSTS OFF)
@@ -148,8 +208,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                             QUERY PLAN                            
 ------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               Filter: (((50) = prt1_p0.a) OR ((75) = prt2_p0.b))
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
@@ -174,7 +242,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               Filter: (((50) = prt1_p4.a) OR ((75) = prt2_p4.b))
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
@@ -211,35 +287,44 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Hash Right Join
-         Hash Cond: (prt2_p2.b = prt1_p1.a)
+         Hash Cond: (prt2_p2.b = prt1_p0.a)
          ->  Append
                ->  Seq Scan on prt2_p2
                      Filter: (b > 250)
                ->  Seq Scan on prt2_p3
                      Filter: (b > 250)
+               ->  Seq Scan on prt2_p4
+                     Filter: (b > 250)
          ->  Hash
                ->  Append
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p1
                            Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p2
                            Filter: ((a < 450) AND (b = 0))
-(15 rows)
+(19 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-(9 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+(14 rows)
 
 -- Currently we can't do partitioned join if nullable-side partitions are pruned
 EXPLAIN (COSTS OFF)
@@ -247,11 +332,13 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                       QUERY PLAN                      
 ------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Hash Full Join
-         Hash Cond: (prt1_p1.a = prt2_p2.b)
-         Filter: ((prt1_p1.b = 0) OR (prt2_p2.a = 0))
+         Hash Cond: (prt1_p0.a = prt2_p2.b)
+         Filter: ((prt1_p0.b = 0) OR (prt2_p2.a = 0))
          ->  Append
+               ->  Seq Scan on prt1_p0
+                     Filter: (a < 450)
                ->  Seq Scan on prt1_p1
                      Filter: (a < 450)
                ->  Seq Scan on prt1_p2
@@ -262,64 +349,147 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                            Filter: (b > 250)
                      ->  Seq Scan on prt2_p3
                            Filter: (b > 250)
-(16 rows)
+                     ->  Seq Scan on prt2_p4
+                           Filter: (b > 250)
+(20 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
-(12 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+      |       | 375 | 0375
+      |       | 450 | 0450
+      |       | 525 | 0525
+      |       | 600 | 0600
+      |       | 675 | 0675
+      |       | 750 | 0750
+(20 rows)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Semi Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Semi Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                            Filter: (a = 0)
-         ->  Nested Loop Semi Join
-               Join Filter: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
-               ->  Materialize
-                     ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
-(24 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt1_p3 t2_3
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+                           Filter: (b = 0)
+(37 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
 
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
@@ -330,27 +500,82 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
    ->  Append
          ->  Hash Anti Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Hash Anti Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Hash Anti Join
                Hash Cond: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
-(17 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(27 rows)
 
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
-  sum  |         avg          | sum  |         avg         
--------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+  sum  |         avg          | sum  |        avg         
+-------+----------------------+------+--------------------
+ 95550 | 273.0000000000000000 | 2200 | 6.2857142857142857
 (1 row)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Append
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p0 t1
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+               Index Cond: (a = t1.b)
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.b = t2_1.a)
+         ->  Seq Scan on prt2_p1 t1_1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p1 t2_1
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p2 t1_2
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+               Index Cond: (a = t1_2.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p3 t1_3
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+               Index Cond: (a = t1_3.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p4 t1_4
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+               Index Cond: (a = t1_4.b)
+(27 rows)
+
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+  b   |   c   
+------+-------
+ -225 | -0225
+  -75 | -0075
+   75 | 0075
+  225 | 0225
+  375 | 0375
+  525 | 0525
+  675 | 0675
+(7 rows)
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -362,49 +587,74 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
+                     ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
                            Index Cond: (a = t1.a)
-                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t3
                            Index Cond: (b = t2.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_1
+                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
                            Index Cond: (a = t1_1.a)
-                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3_1
                            Index Cond: (b = t2_1.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_2
+                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
                            Index Cond: (a = t1_2.a)
-                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_2
                            Index Cond: (b = t2_2.a)
-(27 rows)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                           Index Cond: (a = t1_3.a)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_3
+                           Index Cond: (b = t2_3.a)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                           Index Cond: (a = t1_4.a)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t3_4
+                           Index Cond: (b = t2_4.a)
+(43 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   | t2a | t3a | least 
------+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
- 100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
- 250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
- 400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
- 550 | 0 | 0550 |     |     |      
-(12 rows)
+  a   | b |   c   | t2a  | t3a | least 
+------+---+-------+------+-----+-------
+ -250 | 0 | -0250 |      |     |      
+ -200 | 0 | -0200 |      |     |      
+ -150 | 0 | -0150 | -150 |   0 |  -150
+ -100 | 0 | -0100 |      |     |      
+  -50 | 0 | -0050 |      |     |      
+    0 | 0 | 0000  |    0 |   0 |     0
+   50 | 0 | 0050  |      |     |      
+  100 | 0 | 0100  |      |     |      
+  150 | 0 | 0150  |  150 |   0 |   150
+  200 | 0 | 0200  |      |     |      
+  250 | 0 | 0250  |      |     |      
+  300 | 0 | 0300  |  300 |   0 |   300
+  350 | 0 | 0350  |      |     |      
+  400 | 0 | 0400  |      |     |      
+  450 | 0 | 0450  |  450 |   0 |   450
+  500 | 0 | 0500  |      |     |      
+  550 | 0 | 0550  |      |     |      
+  600 | 0 | 0600  |  600 |   0 |   600
+  650 | 0 | 0650  |      |     |      
+  700 | 0 | 0700  |      |     |      
+  750 | 0 | 0750  |  750 |   0 |   750
+(21 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
@@ -418,64 +668,95 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
          Hash Cond: ((t1.c)::text = (t2.c)::text)
          Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Seq Scan on prt1_p1 t1
-               ->  Seq Scan on prt1_p2 t1_1
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
          ->  Hash
                ->  Append
                      ->  Hash Join
                            Hash Cond: (t2.a = t3.b)
-                           ->  Seq Scan on prt1_p1 t2
+                           ->  Seq Scan on prt1_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t3
+                                 ->  Seq Scan on prt2_p0 t3
                      ->  Hash Join
                            Hash Cond: (t2_1.a = t3_1.b)
-                           ->  Seq Scan on prt1_p2 t2_1
+                           ->  Seq Scan on prt1_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t3_1
+                                 ->  Seq Scan on prt2_p1 t3_1
                      ->  Hash Join
                            Hash Cond: (t2_2.a = t3_2.b)
-                           ->  Seq Scan on prt1_p3 t2_2
+                           ->  Seq Scan on prt1_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p3 t3_2
-(26 rows)
+                                 ->  Seq Scan on prt2_p2 t3_2
+                     ->  Hash Join
+                           Hash Cond: (t2_3.a = t3_3.b)
+                           ->  Seq Scan on prt1_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t3_3
+                     ->  Hash Join
+                           Hash Cond: (t2_4.a = t3_4.b)
+                           ->  Seq Scan on prt1_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t3_4
+(38 rows)
 
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, t2.b t2b, t2.c t2c, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
-  a  | t2a | t2c  
------+-----+------
-   0 |   0 | 0000
-  50 |     | 
- 100 |     | 
- 150 | 150 | 0150
- 200 |     | 
- 250 |     | 
- 300 | 300 | 0300
- 350 |     | 
- 400 |     | 
- 450 | 450 | 0450
- 500 |     | 
- 550 |     | 
-(12 rows)
+  a   | t2a  |  t2c  
+------+------+-------
+ -250 |      | 
+ -200 |      | 
+ -150 | -150 | -0150
+ -100 |      | 
+  -50 |      | 
+    0 |    0 | 0000
+   50 |      | 
+  100 |      | 
+  150 |  150 | 0150
+  200 |      | 
+  250 |      | 
+  300 |  300 | 0300
+  350 |      | 
+  400 |      | 
+  450 |  450 | 0450
+  500 |      | 
+  550 |      | 
+  600 |  600 | 0600
+  650 |      | 
+  700 |      | 
+  750 |  750 | 0750
+(21 rows)
 
 --
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
@@ -486,32 +767,49 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
    ->  Append
          ->  Hash Join
                Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
-               ->  Seq Scan on prt2_e_p1 t2
+               ->  Seq Scan on prt2_e_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 t1
+                     ->  Seq Scan on prt1_e_p0 t1
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
-               ->  Seq Scan on prt2_e_p2 t2_1
+               Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+               ->  Seq Scan on prt1_e_p1 t1_1
+                     Filter: (c = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 t1_1
-                           Filter: (c = 0)
+                     ->  Seq Scan on prt2_e_p1 t2_1
          ->  Hash Join
                Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
-               ->  Seq Scan on prt2_e_p3 t2_2
+               ->  Seq Scan on prt2_e_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 t1_2
+                     ->  Seq Scan on prt1_e_p2 t1_2
                            Filter: (c = 0)
-(21 rows)
+         ->  Hash Join
+               Hash Cond: (((t2_3.b + t2_3.a) / 2) = ((t1_3.a + t1_3.b) / 2))
+               ->  Seq Scan on prt2_e_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p3 t1_3
+                           Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (((t2_4.b + t2_4.a) / 2) = ((t1_4.a + t1_4.b) / 2))
+               ->  Seq Scan on prt2_e_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4 t1_4
+                           Filter: (c = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
- 150 | 0 | 150 | 0
- 300 | 0 | 300 | 0
- 450 | 0 | 450 | 0
-(4 rows)
+  a   | c |  b   | c 
+------+---+------+---
+ -250 | 0 | -250 | 0
+ -100 | 0 | -100 | 0
+    0 | 0 |    0 | 0
+    0 | 0 |    0 | 0
+  150 | 0 |  150 | 0
+  300 | 0 |  300 | 0
+  450 | 0 |  450 | 0
+  600 | 0 |  600 | 0
+  750 | 0 |  750 | 0
+(9 rows)
 
 --
 -- N-way join
@@ -524,154 +822,232 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = ((t3.a + t3.b) / 2))
+               Join Filter: (t1.a = t2.b)
                ->  Hash Join
-                     Hash Cond: (t2.b = t1.a)
-                     ->  Seq Scan on prt2_p1 t2
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on prt1_e_p0 t3
                      ->  Hash
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                     Index Cond: (((a + b) / 2) = t2.b)
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
+                     Index Cond: (b = ((t3.a + t3.b) / 2))
          ->  Nested Loop
                Join Filter: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_1.b = t1_1.a)
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
+               ->  Index Scan using iprt1_e_p4_ab2 on prt1_e_p1 t3_1
                      Index Cond: (((a + b) / 2) = t2_1.b)
          ->  Nested Loop
                Join Filter: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_2.b = t1_2.a)
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
+               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_2
                      Index Cond: (((a + b) / 2) = t2_2.b)
-(33 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = ((t3_3.a + t3_3.b) / 2))
+               ->  Nested Loop
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t2_3.b)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t2_4.b)
+               ->  Hash Join
+                     Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                     ->  Seq Scan on prt1_e_p4 t3_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_p4 t1_4
+                                 Filter: (b = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = ((t3_4.a + t3_4.b) / 2))
+(52 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
-(4 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(8 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                          QUERY PLAN                          
---------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Hash Right Join
                Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
-               ->  Seq Scan on prt1_e_p1 t3
+               ->  Seq Scan on prt1_e_p0 t3
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2.b = t1.a)
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_1.a)
-               ->  Seq Scan on prt1_e_p2 t3_1
+               ->  Seq Scan on prt1_e_p1 t3_1
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_1.b = t1_1.a)
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_2.a)
-               ->  Seq Scan on prt1_e_p3 t3_2
+               ->  Seq Scan on prt1_e_p2 t3_2
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_2.b = t1_2.a)
-                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                                        Filter: (b = 0)
-(33 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+               ->  Seq Scan on prt1_e_p4 t3_4
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_4.b = t1_4.a)
+                           ->  Seq Scan on prt2_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_p4 t1_4
+                                       Filter: (b = 0)
+(51 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                            QUERY PLAN                             
--------------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p1 t3
+                           ->  Seq Scan on prt1_e_p0 t3
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p1_b on prt2_p1 t2
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
                      Index Cond: (b = t1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p2 t3_1
+                           ->  Seq Scan on prt1_e_p1 t3_1
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
+               ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
                      Index Cond: (b = t1_1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p3 t3_2
+                           ->  Seq Scan on prt1_e_p2 t3_2
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
                      Index Cond: (b = t1_2.a)
-(30 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_e_p3 t3_3
+                           Filter: (c = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                           Index Cond: (a = ((t3_3.a + t3_3.b) / 2))
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Nested Loop Left Join
+               ->  Hash Right Join
+                     Hash Cond: (t1_4.a = ((t3_4.a + t3_4.b) / 2))
+                     ->  Seq Scan on prt1_p4 t1_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p4 t3_4
+                                 Filter: (c = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = t1_4.a)
+(47 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -680,21 +1056,34 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                                                    QUERY PLAN                                                   
 ----------------------------------------------------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   Sort Key: prt1_p0.a, prt2_p0.b, ((prt1_e_p0.a + prt1_e_p0.b))
    ->  Append
          ->  Hash Full Join
-               Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
-               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               Hash Cond: (prt1_p0.a = ((prt1_e_p0.a + prt1_e_p0.b) / 2))
+               Filter: ((prt1_p0.a = (50)) OR (prt2_p0.b = (75)) OR (((prt1_e_p0.a + prt1_e_p0.b) / 2) = (50)))
                ->  Hash Full Join
-                     Hash Cond: (prt1_p1.a = prt2_p1.b)
-                     ->  Seq Scan on prt1_p1
+                     Hash Cond: (prt1_p0.a = prt2_p0.b)
+                     ->  Seq Scan on prt1_p0
                            Filter: (b = 0)
                      ->  Hash
-                           ->  Seq Scan on prt2_p1
+                           ->  Seq Scan on prt2_p0
                                  Filter: (a = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1
+                     ->  Seq Scan on prt1_e_p0
                            Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = prt1_p1.a)
+               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               ->  Seq Scan on prt1_e_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on prt1_p1
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1
+                                       Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
                Filter: ((prt1_p2.a = (50)) OR (prt2_p2.b = (75)) OR (((prt1_e_p2.a + prt1_e_p2.b) / 2) = (50)))
@@ -721,7 +1110,20 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                ->  Hash
                      ->  Seq Scan on prt1_e_p3
                            Filter: (c = 0)
-(42 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = ((prt1_e_p4.a + prt1_e_p4.b) / 2))
+               Filter: ((prt1_p4.a = (50)) OR (prt2_p4.b = (75)) OR (((prt1_e_p4.a + prt1_e_p4.b) / 2) = (50)))
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = prt2_p4.b)
+                     ->  Seq Scan on prt1_p4
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4
+                                 Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4
+                           Filter: (c = 0)
+(68 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
@@ -739,172 +1141,260 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = t1_3.b)
+               Join Filter: (t1.a = t1_5.b)
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Join
-                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
-                           ->  Seq Scan on prt1_e_p1 t2
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t1_3
+                                 ->  Seq Scan on prt2_p0 t1_5
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
                      Index Cond: (a = ((t2.a + t2.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_1.a = t1_4.b)
+               Join Filter: (t1_1.a = t1_6.b)
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Join
-                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
-                           ->  Seq Scan on prt1_e_p2 t2_1
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_6.b)
+                           ->  Seq Scan on prt1_e_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t1_4
+                                 ->  Seq Scan on prt2_p1 t1_6
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
                      Index Cond: (a = ((t2_1.a + t2_1.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_2.a = t1_5.b)
+               Join Filter: (t1_2.a = t1_7.b)
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
+                           ->  Seq Scan on prt2_p2 t1_7
                                  Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
+                           ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t2_2
+                                 Index Cond: (((a + b) / 2) = t1_7.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t1_8.b)
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Nested Loop
+                           ->  Seq Scan on prt2_p3 t1_8
+                                 Filter: (a = 0)
+                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_3
+                                 Index Cond: (((a + b) / 2) = t1_8.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = ((t2_3.a + t2_3.b) / 2))
+                     Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t1_9.b)
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Join
+                           Hash Cond: (((t2_4.a + t2_4.b) / 2) = t1_9.b)
+                           ->  Seq Scan on prt1_e_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t1_9
+                                       Filter: (a = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = ((t2_4.a + t2_4.b) / 2))
+                     Filter: (b = 0)
+(66 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                               QUERY PLAN                                
--------------------------------------------------------------------------
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
-                           ->  Seq Scan on prt2_p1 t1_3
+                           Hash Cond: (t1_5.b = ((t1_10.a + t1_10.b) / 2))
+                           ->  Seq Scan on prt2_p0 t1_5
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t1_6
+                                 ->  Seq Scan on prt1_e_p0 t1_10
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
-                     Index Cond: (a = t1_3.b)
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t1_5.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
-                           ->  Seq Scan on prt2_p2 t1_4
+                           Hash Cond: (t1_6.b = ((t1_11.a + t1_11.b) / 2))
+                           ->  Seq Scan on prt2_p1 t1_6
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t1_7
+                                 ->  Seq Scan on prt1_e_p1 t1_11
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
-                     Index Cond: (a = t1_4.b)
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
+                     Index Cond: (a = t1_6.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
-                           ->  Seq Scan on prt2_p3 t1_5
+                           Hash Cond: (t1_7.b = ((t1_12.a + t1_12.b) / 2))
+                           ->  Seq Scan on prt2_p2 t1_7
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p3 t1_8
+                                 ->  Seq Scan on prt1_e_p2 t1_12
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t1_5.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t1_7.b)
                      Filter: (b = 0)
-(39 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_8.b = ((t1_13.a + t1_13.b) / 2))
+                           ->  Seq Scan on prt2_p3 t1_8
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p3 t1_13
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t1_8.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_9.b = ((t1_14.a + t1_14.b) / 2))
+                           ->  Seq Scan on prt2_p4 t1_9
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t1_14
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = t1_9.b)
+                     Filter: (b = 0)
+(63 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 -- test merge joins
 SET enable_hashjoin TO off;
 SET enable_nestloop TO off;
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                           QUERY PLAN                           
-----------------------------------------------------------------
+                            QUERY PLAN                            
+------------------------------------------------------------------
  Merge Append
    Sort Key: t1.a
    ->  Merge Semi Join
-         Merge Cond: (t1.a = t1_3.b)
+         Merge Cond: (t1.a = t1_5.b)
          ->  Sort
                Sort Key: t1.a
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+               Merge Cond: (t1_5.b = (((t1_10.a + t1_10.b) / 2)))
                ->  Sort
-                     Sort Key: t1_3.b
-                     ->  Seq Scan on prt2_p1 t1_3
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on prt2_p0 t1_5
                ->  Sort
-                     Sort Key: (((t1_6.a + t1_6.b) / 2))
-                     ->  Seq Scan on prt1_e_p1 t1_6
+                     Sort Key: (((t1_10.a + t1_10.b) / 2))
+                     ->  Seq Scan on prt1_e_p0 t1_10
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_1.a = t1_4.b)
+         Merge Cond: (t1_1.a = t1_6.b)
          ->  Sort
                Sort Key: t1_1.a
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+               Merge Cond: (t1_6.b = (((t1_11.a + t1_11.b) / 2)))
                ->  Sort
-                     Sort Key: t1_4.b
-                     ->  Seq Scan on prt2_p2 t1_4
+                     Sort Key: t1_6.b
+                     ->  Seq Scan on prt2_p1 t1_6
                ->  Sort
-                     Sort Key: (((t1_7.a + t1_7.b) / 2))
-                     ->  Seq Scan on prt1_e_p2 t1_7
+                     Sort Key: (((t1_11.a + t1_11.b) / 2))
+                     ->  Seq Scan on prt1_e_p1 t1_11
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_2.a = t1_5.b)
+         Merge Cond: (t1_2.a = t1_7.b)
          ->  Sort
                Sort Key: t1_2.a
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+               Merge Cond: (t1_7.b = (((t1_12.a + t1_12.b) / 2)))
                ->  Sort
-                     Sort Key: t1_5.b
-                     ->  Seq Scan on prt2_p3 t1_5
+                     Sort Key: t1_7.b
+                     ->  Seq Scan on prt2_p2 t1_7
                ->  Sort
-                     Sort Key: (((t1_8.a + t1_8.b) / 2))
-                     ->  Seq Scan on prt1_e_p3 t1_8
+                     Sort Key: (((t1_12.a + t1_12.b) / 2))
+                     ->  Seq Scan on prt1_e_p2 t1_12
                            Filter: (c = 0)
-(47 rows)
+   ->  Merge Semi Join
+         Merge Cond: (t1_3.a = t1_8.b)
+         ->  Sort
+               Sort Key: t1_3.a
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_8.b = (((t1_13.a + t1_13.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_8.b
+                     ->  Seq Scan on prt2_p3 t1_8
+               ->  Sort
+                     Sort Key: (((t1_13.a + t1_13.b) / 2))
+                     ->  Seq Scan on prt1_e_p3 t1_13
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_4.a = t1_9.b)
+         ->  Sort
+               Sort Key: t1_4.a
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_9.b = (((t1_14.a + t1_14.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_9.b
+                     ->  Seq Scan on prt2_p4 t1_9
+               ->  Sort
+                     Sort Key: (((t1_14.a + t1_14.b) / 2))
+                     ->  Seq Scan on prt1_e_p4 t1_14
+                           Filter: (c = 0)
+(77 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
@@ -921,14 +1411,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
                            ->  Sort
                                  Sort Key: (((t3.a + t3.b) / 2))
-                                 ->  Seq Scan on prt1_e_p1 t3
+                                 ->  Seq Scan on prt1_e_p0 t3
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1.a
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                ->  Sort
                      Sort Key: t2.b
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Merge Left Join
                Merge Cond: (t1_1.a = t2_1.b)
                ->  Sort
@@ -937,14 +1427,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
                            ->  Sort
                                  Sort Key: (((t3_1.a + t3_1.b) / 2))
-                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                 ->  Seq Scan on prt1_e_p1 t3_1
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_1.a
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                ->  Sort
                      Sort Key: t2_1.b
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Merge Left Join
                Merge Cond: (t1_2.a = t2_2.b)
                ->  Sort
@@ -953,32 +1443,74 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
                            ->  Sort
                                  Sort Key: (((t3_2.a + t3_2.b) / 2))
-                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                 ->  Seq Scan on prt1_e_p2 t3_2
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_2.a
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                ->  Sort
                      Sort Key: t2_2.b
-                     ->  Seq Scan on prt2_p3 t2_2
-(51 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Merge Left Join
+               Merge Cond: (t1_3.a = t2_3.b)
+               ->  Sort
+                     Sort Key: t1_3.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_3.a + t3_3.b) / 2)) = t1_3.a)
+                           ->  Sort
+                                 Sort Key: (((t3_3.a + t3_3.b) / 2))
+                                 ->  Seq Scan on prt1_e_p3 t3_3
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_3.a
+                                 ->  Seq Scan on prt1_p3 t1_3
+               ->  Sort
+                     Sort Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Merge Left Join
+               Merge Cond: (t1_4.a = t2_4.b)
+               ->  Sort
+                     Sort Key: t1_4.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_4.a + t3_4.b) / 2)) = t1_4.a)
+                           ->  Sort
+                                 Sort Key: (((t3_4.a + t3_4.b) / 2))
+                                 ->  Seq Scan on prt1_e_p4 t3_4
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_4.a
+                                 ->  Seq Scan on prt1_p4 t1_4
+               ->  Sort
+                     Sort Key: t2_4.b
+                     ->  Seq Scan on prt2_p4 t2_4
+(83 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- MergeAppend on nullable column
 -- This should generate a partitionwise join, but currently fails to
@@ -987,12 +1519,14 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Merge Left Join
-         Merge Cond: (prt1_p1.a = prt2_p2.b)
+         Merge Cond: (prt1_p0.a = prt2_p2.b)
          ->  Sort
-               Sort Key: prt1_p1.a
+               Sort Key: prt1_p0.a
                ->  Append
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p1
                            Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p2
@@ -1004,21 +1538,28 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                            Filter: (b > 250)
                      ->  Seq Scan on prt2_p3
                            Filter: (b > 250)
-(18 rows)
+                     ->  Seq Scan on prt2_p4
+                           Filter: (b > 250)
+(22 rows)
 
 SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  b  
------+-----
-   0 |    
-  50 |    
- 100 |    
- 150 |    
- 200 |    
- 250 |    
- 300 | 300
- 350 |    
- 400 |    
-(9 rows)
+  a   |  b  
+------+-----
+ -250 |    
+ -200 |    
+ -150 |    
+ -100 |    
+  -50 |    
+    0 |    
+   50 |    
+  100 |    
+  150 |    
+  200 |    
+  250 |    
+  300 | 300
+  350 |    
+  400 |    
+(14 rows)
 
 -- merge join when expression with whole-row reference needs to be sorted;
 -- partitionwise join does not apply
@@ -1032,30 +1573,797 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
          Sort Key: t1.a, ((((t1.*)::prt1))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
    ->  Sort
          Sort Key: t2.b, ((((t2.*)::prt2))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt2_p1 t2
-                     ->  Seq Scan on prt2_p2 t2_1
-                     ->  Seq Scan on prt2_p3 t2_2
-(16 rows)
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+(20 rows)
 
 SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.b ORDER BY t1.a;
- a  | b  
-----+----
-  0 |  0
-  6 |  6
- 12 | 12
- 18 | 18
- 24 | 24
-(5 rows)
+  a  |  b  
+-----+-----
+ -24 | -24
+ -18 | -18
+ -12 | -12
+  -6 |  -6
+   0 |   0
+   6 |   6
+  12 |  12
+  18 |  18
+  24 |  24
+(9 rows)
 
 RESET enable_hashjoin;
 RESET enable_nestloop;
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p4 t2_3
+               ->  Seq Scan on prt2_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_4
+                           Filter: (b = 0)
+(22 rows)
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p3 t1_2
+                           Filter: (b = 0)
+(21 rows)
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -150 | -0150 | 0 | -0150
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+(7 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 |   | 
+ -200 | -0200 |   | 
+ -150 | -0150 | 0 | -0150
+ -100 | -0100 |   | 
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+(21 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -200 | 0 | -0200
+ -100 | 0 | -0100
+  -50 | 0 | -0050
+   50 | 0 | 0050
+  100 | 0 | 0100
+  200 | 0 | 0200
+  250 | 0 | 0250
+  350 | 0 | 0350
+  400 | 0 | 0400
+  500 | 0 | 0500
+  550 | 0 | 0550
+  650 | 0 | 0650
+  700 | 0 | 0700
+(14 rows)
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+                             QUERY PLAN                              
+---------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t3.c
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p0_a on prt1_p0 t3
+                           Index Cond: (a = t2.b)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t2.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_1.a = t2_1.b)
+                           ->  Seq Scan on prt1_p1 t3_1
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1 t2_1
+                                       Filter: (a = 0)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p2_a on prt1_p2 t3_2
+                           Index Cond: (a = t2_2.b)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t3_3
+                           Index Cond: (a = t2_3.b)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_4.a = t2_4.b)
+                           ->  Seq Scan on prt1_p4 t3_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t2_4
+                                       Filter: (a = 0)
+(47 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+  a   | a |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.b
+   ->  Hash Right Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p5 t2_5
+                           Filter: (a = 0)
+(24 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: (t1.a = t2.b)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.a, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
+                     Filter: (a = 0)
+(22 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
 --
 -- partitioned by multiple columns
 --
@@ -1125,82 +2433,1552 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 -- test partition matching with N-way join
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                   QUERY PLAN                                   
---------------------------------------------------------------------------------
+                                           QUERY PLAN                                           
+------------------------------------------------------------------------------------------------
  GroupAggregate
    Group Key: t1.c, t2.c, t3.c
    ->  Sort
          Sort Key: t1.c, t3.c
          ->  Append
                ->  Hash Join
-                     Hash Cond: (t1.c = ltrim(t3.c, 'A'::text))
+                     Hash Cond: ((t1.c)::text = ltrim(t3.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1.b = t2.b) AND (t1.c = t2.c))
-                           ->  Seq Scan on plt1_p1 t1
+                           Hash Cond: ((t2.b = t1.b) AND ((t2.c)::text = (t1.c)::text))
+                           ->  Seq Scan on plt2_p4 t2
                            ->  Hash
-                                 ->  Seq Scan on plt2_p1 t2
+                                 ->  Seq Scan on plt1_p4 t1
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p1 t3
+                           ->  Seq Scan on plt1_e_p4 t3
                ->  Hash Join
-                     Hash Cond: (t1_1.c = ltrim(t3_1.c, 'A'::text))
+                     Hash Cond: ((t1_1.c)::text = ltrim(t3_1.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
-                           ->  Seq Scan on plt1_p2 t1_1
+                           Hash Cond: ((t1_1.b = t2_1.b) AND ((t1_1.c)::text = (t2_1.c)::text))
+                           ->  Seq Scan on plt1_p1 t1_1
                            ->  Hash
-                                 ->  Seq Scan on plt2_p2 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p2 t3_1
+                           ->  Seq Scan on plt1_e_p1 t3_1
                ->  Hash Join
-                     Hash Cond: (t1_2.c = ltrim(t3_2.c, 'A'::text))
+                     Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
-                           ->  Seq Scan on plt1_p3 t1_2
+                           Hash Cond: ((t1_2.b = t2_2.b) AND ((t1_2.c)::text = (t2_2.c)::text))
+                           ->  Seq Scan on plt1_p2 t1_2
                            ->  Hash
-                                 ->  Seq Scan on plt2_p3 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p3 t3_2
-(32 rows)
+                           ->  Seq Scan on plt1_e_p2 t3_2
+               ->  Hash Join
+                     Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2_3.b = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text))
+                           ->  Seq Scan on plt2_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p3 t3_3
+(41 rows)
 
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  75.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 123.0000000000000000 | 123.0000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 225.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 273.0000000000000000 | 273.0000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 375.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 423.0000000000000000 | 423.0000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 525.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 573.0000000000000000 | 573.0000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p5 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_3
+                                 ->  Seq Scan on plt2_p3 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Seq Scan on plt2_p2 t2_2
+                                 ->  Seq Scan on plt2_p3 t2_3
+(22 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1225,22 +4003,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
 --------------------------------------------------
  Hash Left Join
    Hash Cond: (t2.b = a)
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t3.a = t2.b)
-               ->  Seq Scan on prt1_p1 t3
-               ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
-         ->  Hash Join
-               Hash Cond: (t3_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t3_1
-               ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
-         ->  Hash Join
-               Hash Cond: (t3_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t3_2
-               ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
+   ->  Hash Join
+         Hash Cond: (t3.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t3
+               ->  Seq Scan on prt1_p1 t3_1
+               ->  Seq Scan on prt1_p2 t3_2
+               ->  Seq Scan on prt1_p3 t3_3
+               ->  Seq Scan on prt1_p4 t3_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
    ->  Hash
          ->  Result
                One-Time Filter: false
@@ -1255,16 +4033,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 --
 -- tests for hash partitioned tables.
@@ -1340,41 +4124,9 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  273.0000000000000000 | 273.0000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
 (6 rows)
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
- Sort
-   Sort Key: t1.a
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
-               ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
-               ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
-                           Filter: (b = 0)
-(21 rows)
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
@@ -1389,26 +4141,24 @@ SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c
    Sort Key: t1.c
    ->  HashAggregate
          Group Key: t1.c, t2.c
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t2.c = t1.c)
-                     ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p1 t1
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on plt1_p4 t1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_1.c = t1_1.c)
-                     ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_p2 t1_1
+                           ->  Seq Scan on plt1_p1 t1_1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_2.c = t1_2.c)
-                     ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Seq Scan on plt1_p2 t1_2
                                  Filter: ((a % 25) = 0)
-(23 rows)
+                           ->  Seq Scan on plt1_p3 t1_3
+                                 Filter: ((a % 25) = 0)
+(21 rows)
 
 --
 -- multiple levels of partitioning
@@ -1804,64 +4554,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
  Hash Join
    Hash Cond: (t1.a = t2.a)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt4_n_p1 t2
                ->  Seq Scan on prt4_n_p2 t2_1
                ->  Seq Scan on prt4_n_p3 t2_2
-(11 rows)
+(13 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a and t1.a = t3.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                        QUERY PLAN                        
+----------------------------------------------------------
  Hash Join
-   Hash Cond: (t2.a = t1.a)
+   Hash Cond: (t3.b = t1.a)
    ->  Append
-         ->  Seq Scan on prt4_n_p1 t2
-         ->  Seq Scan on prt4_n_p2 t2_1
-         ->  Seq Scan on prt4_n_p3 t2_2
+         ->  Seq Scan on prt2_p0 t3
+         ->  Seq Scan on prt2_p1 t3_1
+         ->  Seq Scan on prt2_p2 t3_2
+         ->  Seq Scan on prt2_p3 t3_3
+         ->  Seq Scan on prt2_p4 t3_4
+         ->  Seq Scan on prt2_p5 t3_5
    ->  Hash
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.a = t3_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_2.b)
-                     ->  Seq Scan on prt1_p3 t1_2
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3 t3_2
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt4_n_p1 t2
+                           ->  Seq Scan on prt4_n_p2 t2_1
+                           ->  Seq Scan on prt4_n_p3 t2_2
 (23 rows)
 
 -- partitionwise join can not be applied if there are no equi-join conditions
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (b > t1.a)
-(12 rows)
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+(16 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partitionwise join
@@ -1947,16 +4703,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
-                     ->  Seq Scan on plt1_p1 t3
-                     ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3
+                     ->  Seq Scan on plt1_p1 t3_1
+                     ->  Seq Scan on plt1_p2 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partitionwise join can not be applied for a join between list and range
 -- partitioned table
@@ -1967,14 +4724,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
  Hash Full Join
    Hash Cond: ((t2.c)::text = (t1.c)::text)
    ->  Append
-         ->  Seq Scan on prt1_p1 t2
-         ->  Seq Scan on prt1_p2 t2_1
-         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p0 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p2 t2_2
+         ->  Seq Scan on prt1_p3 t2_3
+         ->  Seq Scan on prt1_p4 t2_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt1_n_p1 t1
                ->  Seq Scan on prt1_n_p2 t1_1
-(10 rows)
+(12 rows)
 
 -- partitionwise join can not be applied if only one of joining table has
 -- default partition
@@ -1990,16 +4749,23 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
    ->  Hash Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index db9a6b4a96..7597b069fa 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -10,25 +10,39 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -69,11 +83,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -95,20 +117,30 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 
 EXPLAIN (COSTS OFF)
@@ -172,6 +204,128 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -196,28 +350,79 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
 -- test partition matching with N-way join
@@ -225,6 +430,175 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
@@ -270,27 +644,18 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
 ALTER TABLE plt2 DETACH PARTITION plt2_p3;
 ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
 ANALYZE plt2;
-
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.a % 25 = 0 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
 --
 -- multiple levels of partitioning
 --
-- 
2.18.0

#80Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: amul sul (#79)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Tue, Jul 2, 2019 at 1:47 PM amul sul <sulamul@gmail.com> wrote:

Attached version is rebase atop of the latest master head(c74d49d41c), thanks.

Thanks! Will review.

Best regards,
Etsuro Fujita

#81Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Etsuro Fujita (#80)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Wed, Jul 3, 2019 at 3:44 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

On Tue, Jul 2, 2019 at 1:47 PM amul sul <sulamul@gmail.com> wrote:

Attached version is rebase atop of the latest master head(c74d49d41c), thanks.

Thanks! Will review.

I started reviewing this. Here is my initial review comments:

* 0001-Hash-partition-bound-equality-refactoring-v22.patch

First of all, I agree with your view on hash partitioning:

"3. For hash partitioned tables however, we support partition-wise join
only when the bounds exactly match. For hash partitioning it's unusual
to have missing partitions and hence generic partition matching is not
required."

which is cited from the commit message for the main patch
"0002-Partition-wise-join-for-1-1-1-0-0-1-partition-matchi-v22.patch".
(I think it would be better if we can extend the partition matching to
the hash-partitioning case where there are missing partitions in
future, though.) However, I don't think it's a good idea to do this
refactoring, because that would lead to duplicating the code to check
whether two given hash bound collections are equal in
partition_bounds_equal() and partition_hash_bounds_merge() that will
be added by the main patch, after all. To avoid that, how about
calling partition_bounds_equal() from partition_hash_bounds_merge() in
the main patch, like the attached? I also did some cleanup for
partition_hash_bounds_merge(). This change makes the refactoring
patch unnecessary, so I dropped it. Also, I included the
regression-test patch
"0003-Tests-for-0-1-1-1-and-1-0-partition-matching-v22.patch" in the
attached, because make check didn't succeed without the
regression-test patch.

That's it for now. I'll review the remaining parts (ie,
"0002-Partition-wise-join-for-1-1-1-0-0-1-partition-matchi-v22.patch"
and "0003-Tests-for-0-1-1-1-and-1-0-partition-matching-v22.patch")
closely next.

Best regards,
Etsuro Fujita

Attachments:

Improve-partition-matching-for-partitionwise-joins-v23.patchapplication/octet-stream; name=Improve-partition-matching-for-partitionwise-joins-v23.patchDownload
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 43c3b7ea48..4ab218641c 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1339,25 +1339,31 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	PartitionScheme part_scheme;
+	PartitionBoundInfo join_boundinfo;
+	List	   *parts1;
+	List	   *parts2;
+	ListCell   *lc1;
+	ListCell   *lc2;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1370,36 +1376,67 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	 */
 	Assert(joinrel->part_scheme == rel1->part_scheme &&
 		   joinrel->part_scheme == rel2->part_scheme);
+	part_scheme = joinrel->part_scheme;
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * Get the list of matching partitions to be joined along with the
+	 * partition bounds of the join relation. Because of the restrictions
+	 * imposed by partition matching algorithm, not every pair of joining
+	 * relations for this join will be able to use partition-wise join. But all
+	 * those pairs which can use partition-wise join will produce the same
+	 * partition bounds for the join relation.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	join_boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											part_scheme->parttyplen,
+											part_scheme->parttypbyval,
+											part_scheme->partsupfunc,
+											part_scheme->partcollation,
+											rel1, rel2,
+											parent_sjinfo->jointype,
+											&parts1, &parts2);
+
+	if (join_boundinfo == NULL)
+		return;
 
-	nparts = joinrel->nparts;
+	if (joinrel->boundinfo == NULL)
+	{
+		Assert(joinrel->nparts == 0 && joinrel->part_rels == NULL);
+		joinrel->boundinfo = join_boundinfo;
+		joinrel->nparts = list_length(parts1);
+		Assert(joinrel->nparts == list_length(parts2));
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) *
+									joinrel->nparts);
+	}
+	else
+	{
+		Assert(partition_bounds_equal(part_scheme->partnatts,
+									  part_scheme->parttyplen,
+									  part_scheme->parttypbyval,
+									  join_boundinfo, joinrel->boundinfo));
+		/*
+		 * Every pair of joining relations should result in the same number
+		 * of child-joins.
+		 */
+		Assert(joinrel->nparts == list_length(parts1));
+		Assert(joinrel->nparts == list_length(parts2));
+		Assert(joinrel->part_rels);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	cnt_parts = -1;
+	forboth(lc1, parts1, lc2, parts2)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
-		bool		rel1_empty = (child_rel1 == NULL ||
-								  IS_DUMMY_REL(child_rel1));
-		bool		rel2_empty = (child_rel2 == NULL ||
-								  IS_DUMMY_REL(child_rel2));
+		int			part1 = lfirst_int(lc1);
+		int			part2 = lfirst_int(lc2);
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		bool		rel1_empty;
+		bool		rel2_empty;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1407,6 +1444,13 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		Assert(part1 >= 0 && part2 >= 0);
+		child_rel1 = rel1->part_rels[part1];
+		child_rel2 = rel2->part_rels[part2];
+		rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1));
+		rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2));
+		cnt_parts++;
+
 		/*
 		 * Check for cases where we can prove that this segment of the join
 		 * returns no rows, due to one or both inputs being empty (including
@@ -1506,12 +1550,23 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 			joinrel->part_rels[cnt_parts] = child_joinrel;
 		}
 
+		/*
+		 * For every pair of joining relations, the set of matching partitions
+		 * would change. However, the base relation partitions constituting
+		 * the given child should remain same for all the joining pairs. Since
+		 * the order in which children are stored in the array of child-joins,
+		 * depends upon partition bounds of the join, which are same for all
+		 * the joining pairs, every joining pair yields the child-joins in the
+		 * same order.
+		 */
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
 
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
 	}
+
+	Assert(cnt_parts == (joinrel->nparts - 1));
 }
 
 /*
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 6054bd2b53..ead871180f 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1623,7 +1623,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1636,24 +1636,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1665,17 +1647,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 13d576c379..1084ec8f69 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -25,6 +25,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/pathnodes.h"
 #include "parser/parse_coerce.h"
 #include "partitioning/partbounds.h"
 #include "partitioning/partdesc.h"
@@ -69,6 +70,12 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+typedef struct PartitionMap
+{
+	int from;
+	int to;
+} PartitionMap;
+
 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);
@@ -108,6 +115,58 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 									 Expr **keyCol,
 									 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static PartitionBoundInfo partition_range_bounds_merge(
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 List **outer_parts, List **inner_parts,
+							 JoinType jointype, int partnatts,
+							 FmgrInfo *supfuncs, Oid *collations);
+static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype);
+static PartitionBoundInfo partition_hash_bounds_merge(int partnatts,
+							int16 *parttyplen, bool *parttypbyval,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts);
+static void generate_matching_part_pairs(PartitionMap *outer_maps,
+										 PartitionMap *inner_maps,
+										 int nparts1, int nparts2,
+										 JoinType jointype, int nparts,
+										 List **parts1, List **parts2);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int map_and_merge_partitions(PartitionMap *outer_maps,
+										PartitionMap *inner_maps,
+										int index1, int index2, int *next_index);
+static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *collations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2);
+static bool partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
+						   Oid *collations, PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, int *ub_cmpval,
+						   int *lb_cmpval);
+static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes);
+static bool merge_default_partitions(PartitionBoundInfo outer_bi,
+						 PartitionBoundInfo inner_bi,
+						 PartitionMap *outer_maps,
+						 PartitionMap *inner_maps,
+						 JoinType jointype,
+						 int *next_index, int *default_index);
+static bool merge_null_partitions(PartitionBoundInfo outer_bi,
+								  PartitionBoundInfo inner_bi,
+								  PartitionMap *outer_maps,
+								  PartitionMap *inner_maps,
+								  JoinType jointype,
+								  int *next_index, int *null_index,
+								  int *default_index);
 
 /*
  * get_qual_from_partbound
@@ -2995,3 +3054,1621 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * The function produces the partition bounds for a join between two relations
+ * whose partition bounds are given. The function also returns two lists of
+ * partition indexes one for each of the joining relations. Both the lists
+ * contain the same number of elements. The partition indexes at the same
+ * positions in the lists indicate the pair partitions, one from each side, to
+ * be joined and the position itself corresponds to the index of partition
+ * produced by that child-join in the partitioned join.
+ *
+ * The function returns NULL if we can not find the matching pair of
+ * partitions. This happens if 1. multiple partitions on one side match with
+ * one partition on the other side. 2. a given partition on the outer side
+ * doesn't have a matching partition on the inner side. We can not support the
+ * first case since we don't have a way to represent multiple partitions as a
+ * single relation (RelOptInfo) and then perform join using the ganged
+ * relation. We can not support the second case since the missing inner
+ * partition needs to be represented as an empty relation and we don't have a
+ * way to introduce empty relation during join planning after creating paths
+ * for all the base relations.
+ */
+PartitionBoundInfo
+partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype,
+					   List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo 	merged_bounds;
+	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
+						inner_binfo = inner_rel->boundinfo;
+	char				strategy = outer_binfo->strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partsupfunc,
+														partcollation,
+														outer_rel, inner_rel,
+														outer_parts, inner_parts,
+														jointype);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(outer_rel, inner_rel,
+														 outer_parts, inner_parts,
+														 jointype, partnatts,
+														 partsupfunc,
+														 partcollation);
+			break;
+
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = partition_hash_bounds_merge(partnatts, 
+														parttyplen, parttypbyval,
+														outer_rel, inner_rel,
+														outer_parts, inner_parts);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	Assert((*outer_parts == NIL || *inner_parts != NIL) &&
+		   (*inner_parts == NIL || *outer_parts != NIL));
+
+	return merged_bounds;
+}
+
+/*
+ * partition_get_range_bounds
+ *
+ * Given the index of lower bound in datums array, return lower and upper
+ * bounds and the index of the partition with that lower bound.
+ */
+static int
+partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
+						   PartitionRangeBound *lower,
+						   PartitionRangeBound *upper)
+{
+	int			part_index;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The lower bound should correspond to a valid partition. */
+	part_index = bi->indexes[lb_index + 1];
+	Assert(part_index >= 0);
+
+	lower->kind = bi->kind[lb_index];
+	lower->datums = bi->datums[lb_index];
+	lower->lower = true;
+	upper->kind = bi->kind[lb_index + 1];
+	upper->datums = bi->datums[lb_index + 1];
+	upper->lower = false;
+
+	return part_index;
+}
+
+/*
+ * partition_range_get_next_lb_index
+ *
+ * Given the index of lower bound in datums array return the
+ * index of lower bound of the next partition. When the given index corresponds
+ * to the last partition, return number of datums (ndatums).
+ */
+static int
+partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
+{
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The partition index corresponding to the upper bound should be valid. */
+	Assert(bi->indexes[lb_index + 1] >= 0);
+
+	/*
+	 * If there are no bounds left beyond the upper bound, we have reached the
+	 * last partition.
+	 */
+	if (lb_index + 2 < bi->ndatums)
+	{
+		/*
+		 * If the bound next to the upper bound corresponds to no partition,
+		 * that's the next lower bound of the next partition. Otherwise, the
+		 * current upper bound is the lower bound of the next partition.
+		 */
+		if (bi->indexes[lb_index + 2] < 0)
+			return lb_index + 2;
+		else
+			return lb_index + 1;
+	}
+	else
+		return bi->ndatums;
+}
+
+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *partcollations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2)
+{
+	return partition_rbound_cmp(partnatts, partsupfunc, partcollations,
+								bound1->datums, bound1->kind, bound1->lower,
+								bound2);
+}
+
+/*
+ * partition_range_cmp
+ *
+ * Compare the bounds of two range partitions. Set ub_cmpval <, = or > 0, if the
+ * first partition's upper bound is lower than, equal to or higher than the
+ * second partition's upper bound resp. Similarly set lb_cmpval <, =  or > 0,
+ * if the first partition's lower bound is lower than, equal to or higher than
+ * the second partition's lower bound resp.
+ *
+ * Return true, if the ranges overlap, otherwise return false.
+ */
+static bool
+partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
+					PartitionRangeBound *lower_bound1,
+					PartitionRangeBound *upper_bound1,
+					PartitionRangeBound *lower_bound2,
+					PartitionRangeBound *upper_bound2, int *ub_cmpval,
+					int *lb_cmpval)
+{
+	bool		overlap;
+
+	/*
+	 * Compare upper bound of the first partition with the lower bound of the
+	 * second and vice-versa. If lower bound is higher than the upper bound,
+	 * the partitions are not overlapping. All other cases indicate overlapping
+	 * partitions.
+	 */
+	if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+								  lower_bound1, upper_bound2) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = 1;
+		*lb_cmpval = 1;
+	}
+	else if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+									   lower_bound2, upper_bound1) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = -1;
+		*lb_cmpval = -1;
+	}
+	else
+	{
+		overlap = true;
+		*ub_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, upper_bound1,
+											   upper_bound2);
+		*lb_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, lower_bound1,
+											   lower_bound2);
+	}
+
+	return overlap;
+}
+
+/*
+ * partition_range_merge
+ *
+ * Merge the partition bounds of given two partitions such that the join
+ * between the given two partitions fits merged bounds.
+ *
+ * "merged_upper" will be set to one of the given upper bounds and
+ * "merged_lower" will be set to one of the given lower bounds.
+ */
+static void
+partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
+					  Oid *partcollations, JoinType jointype,
+					  PartitionRangeBound *left_lb,
+					  PartitionRangeBound *left_ub,
+					  PartitionRangeBound *right_lb,
+					  PartitionRangeBound *right_ub,
+					  PartitionRangeBound **merged_lb,
+					  PartitionRangeBound **merged_ub)
+{
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = left_ub;
+			*merged_lb = left_lb;
+			break;
+
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) < 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) > 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		case JOIN_FULL:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) > 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) < 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+
+	return;
+}
+
+/*
+ * Add the lower bound of the next range to the list of bounds, if the lower
+ * bound is higher or equal to the previous upper bound. If successful return
+ * true, otherwise false.
+ */
+static bool
+partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
+							  Oid *partcollations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		prev_ub.datums = llast(*merged_datums);
+		prev_ub.kind = llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  next_lb_datums, next_lb_kind, false,
+									  &prev_ub);
+	}
+
+	/*
+	 * The lower bound is lower than the last upper bound, thus does not fit
+	 * the bounds created so far and hence can not be merged with the existing
+	 * bounds.
+	 */
+	if (cmpval < 0)
+		return false;
+
+	/*
+	 * Add bounds of the new merged partition. If the next lower bound is
+	 * higher than the last upper bound, add new range with index
+	 * corresponding to the lower bound as -1. If the merged lower bound
+	 * is same as the last merged upper bound, the last upper bound will be
+	 * reused as the lower bound of the next range.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, next_lb_datums);
+		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	return true;
+}
+
+/*
+ * handle_missing_partition
+ *
+ * If a range appears in one of the joining relations but not the other, a row
+ * in the corresponding partition will not have any join partner in the other
+ * relation, unless the other relation has a default partition. If a given list
+ * value is present in one joining relation but not the other, the default
+ * partition on the other side may contain that value.
+ *
+ * In both these cases, such an extra partition forms a joining pair with the
+ * default partition, if any,  on the other side.
+ *
+ * If the default partition happens to be on the outer side of the join, the
+ * resultant partition will act as the default partition of the join relation.
+ * Otherwise the resultant partition will be associated with the range.
+ *
+ * When the default partition is not present in the other relation, the rows in
+ * the extra partition will be included in the bounds of the join result, if it
+ * appears on the outer side of the join, since all rows from the outer side
+ * are included in the join result.
+ *
+ * This function handles all these cases.
+ *
+ * maps_with_missing and missing_side_default are the partition maps (See
+ * partition_range/list_bounds_merge() for details) and the index of default
+ * partition respectively corresponding the side with missing partition.
+ *
+ * maps_with_extra and extra_part are the partition maps (See
+ * partition_range/list_bounds_merge() for details) and the index of extra
+ * partition respectively corresponding to the side with the extra partition.
+ *
+ * It returns true if the matching succeeds, otherwise returns false.
+ */
+static bool
+handle_missing_partition(PartitionMap *maps_with_missing,
+						 PartitionMap *maps_with_extra,
+						 int missing_side_default,
+						 int extra_part,
+						 bool missing_side_outer,
+						 bool missing_side_inner,
+						 int *next_index, int *default_index,
+						 int *merged_index)
+{
+	bool missing_has_default = (missing_side_default != -1);
+
+	if (missing_has_default)
+	{
+		*merged_index = map_and_merge_partitions(maps_with_missing,
+												 maps_with_extra,
+												 missing_side_default,
+												 extra_part,
+												 next_index);
+		if (*merged_index < 0)
+			return false;
+
+		if (missing_side_outer)
+		{
+			/*
+			 * Default partition on the outer side forms the default
+			 * partition of the join result.
+			 */
+			if (*default_index < 0)
+				*default_index = *merged_index;
+			else if(*default_index != *merged_index)
+			{
+				/*
+				 * Ended up with default partition on the outer side
+				 * being joined with multiple partitions on the inner
+				 * side. We don't support this case.
+				 */
+				return false;
+			}
+
+			/*
+			 * Since the merged partition acts as a default partition, it
+			 * doesn't need a separate index.
+			 */
+			*merged_index = -1;
+		}
+	}
+	else if (missing_side_inner)
+	{
+		/*
+		 * If this partition has already been mapped (say because we
+		 * found an overlapping range earlier), we know where does it
+		 * fit in the join result. Nothing to do in that case. Else
+		 * create a new merged partition.
+		 */
+		PartitionMap *extra_map = &maps_with_extra[extra_part];
+		if (extra_map->to < 0)
+		{
+			extra_map->to = *next_index;
+			*next_index = *next_index + 1;
+			*merged_index = extra_map->to;
+		}
+	}
+	else
+		*merged_index = -1;
+
+	return true;
+}
+
+static PartitionMap*
+init_partition_map(RelOptInfo *rel)
+{
+	int i, nparts = rel->nparts;
+	PartitionMap *map;
+
+	map = (PartitionMap *) palloc(sizeof(PartitionMap) * nparts);
+
+	for (i = 0; i < nparts; i++)
+	{
+		map[i].from = -1;
+		map[i].to = -1;
+	}
+
+	return map;
+}
+
+/*
+ * Allocate and initialize partition maps. We maintain four maps, two maps
+ * for each joining relation. pmap[i] gives the partition from the other
+ * relation which would join with ith partition of the given relation.
+ * Partition i from the given relation will join with partition pmap[i]
+ * from the other relation to produce partition mmap[i] of the join (merged
+ * partition).
+ *
+ * pmap[i] = -1 indicates that ith partition of a given relation does not
+ * have a matching partition from the other relation.
+ *
+ * mmap[i] = -1 indicates that ith partition of a given relation does not
+ * contribute to the join result. That can happen only when the given
+ * relation is the inner relation and it doesn't have a matching partition
+ * from the outer relation, hence pmap[i] should be -1.
+ *
+ * In case of an outer join, every partition of the outer join will appear
+ * in the join result, and thus has mmap[i] set for all i. But it's not
+ * necessary that every partition on the outer side will have a matching
+ * partition on the inner side. In such a case, we end up with pmap[i] = -1
+ * and mmap[i] != -1.
+ */
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 List **outer_parts, List **inner_parts,
+							 JoinType jointype, int partnatts,
+							 FmgrInfo *partsupfuncs, Oid *partcollations)
+
+{
+	PartitionMap *outer_maps = NULL;
+	PartitionMap *inner_maps = NULL;
+	int			outer_part = 0;
+	int			inner_part = 0;
+	PartitionBoundInfo merged_bounds = NULL;
+	int			outer_lb_index;
+	int			inner_lb_index;
+	int			next_index;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	List	   *merged_kinds = NIL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			inner_default = inner_bi->default_index;
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	outer_maps = init_partition_map(outer_rel);
+	inner_maps = init_partition_map(inner_rel);
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = 0;
+	inner_lb_index = 0;
+	next_index = 0;
+	while (outer_lb_index < outer_bi->ndatums ||
+		   inner_lb_index < inner_bi->ndatums)
+	{
+		PartitionRangeBound outer_lb, outer_ub,
+							inner_lb, inner_ub,
+							*merged_lb = NULL,
+							*merged_ub = NULL;
+
+		int			merged_index = -1;
+		bool		overlap;
+		bool		finished_outer = false;
+		bool		finished_inner = false;
+
+		/* Result of bounds comparison per partition_rbound_cmp(). */
+		int			ub_cmpval;	/* Upper bounds comparison result. */
+		int			lb_cmpval;	/* Lower bounds comparison result. */
+
+		/* Get the range bounds of the next pair of partitions. */
+		if (outer_lb_index < outer_bi->ndatums)
+			outer_part = partition_get_range_bounds(outer_bi, outer_lb_index,
+												&outer_lb, &outer_ub);
+		else
+			finished_outer = true;
+
+		if (inner_lb_index < inner_bi->ndatums)
+			inner_part = partition_get_range_bounds(inner_bi, inner_lb_index,
+												&inner_lb, &inner_ub);
+		else
+			finished_inner = true;
+
+		Assert(!finished_outer || !finished_inner);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (finished_outer)
+		{
+			overlap = false;
+			ub_cmpval = 1;
+			lb_cmpval = 1;
+		}
+		else if (finished_inner)
+		{
+			overlap = false;
+			ub_cmpval = -1;
+			lb_cmpval = -1;
+		}
+		else
+			overlap = partition_range_cmp(partnatts, partsupfuncs, partcollations,
+										  &outer_lb, &outer_ub, &inner_lb,
+										  &inner_ub, &ub_cmpval, &lb_cmpval);
+
+		if (overlap)
+		{
+			/*
+			 * The rows from overlapping portion of ranges on both sides may
+			 * join, hence the corresponding pair of partitions form a joining
+			 * pair. Match them and produce the bounds of the joint partition
+			 * and its index by merging the bounds according to the type of
+			 * join.
+			 */
+			partition_range_merge(partnatts, partsupfuncs, partcollations,
+								  jointype, &outer_lb, &outer_ub, &inner_lb,
+								  &inner_ub, &merged_lb, &merged_ub);
+
+			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+													outer_part, inner_part,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				/* Failed to match the partitions. */
+				return NULL;
+			}
+
+			/*
+			 * If the ranges overlap but don't exactly match, a row from
+			 * non-overlapping portion of the range from one side of join may
+			 * find its join partner in the previous or next overlapping
+			 * partition or default partition on the other side , if such a
+			 * partition exists. All those cases, if true, will cause one
+			 * partition from that side to match at least two partitions on the
+			 * other side; a case that we do not support now. Previous
+			 * partition has been delt with in the previous iteration of this
+			 * loop, next partition will be delt in the next iteration. We will
+			 * deal with the default partition here.
+			 */
+			if ((lb_cmpval < 0 && inner_has_default) ||
+				/* Non-overlapping range on the lower side of outer range. */
+				(lb_cmpval > 0 && outer_has_default) ||
+				/* Non-overlapping range on the lower side of inner range. */
+				(ub_cmpval < 0 && outer_has_default) ||
+				/* Non-overlapping range on the upper side of inner range. */
+				(ub_cmpval > 0 && inner_has_default))
+				/* Non-overlapping range on the upper side of outer range. */
+				return NULL;
+		}
+
+		if (ub_cmpval == 0)
+		{
+			/* Upper bounds of both the ranges match. */
+			Assert(overlap);
+
+			/* Move to the next pair of partitions. */
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the inner side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &outer_lb;
+				merged_ub = &outer_ub;
+
+				/*
+				 * For a FULL join, inner relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the inner relation acts as
+				 * INNER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_inner = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_outer = (jointype == JOIN_FULL);
+				if (!handle_missing_partition(inner_maps,
+											  outer_maps,
+											  inner_default,
+											  outer_part,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the outer side. */
+			Assert(!finished_outer);
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+		}
+		else
+		{
+			Assert(ub_cmpval > 0);
+
+			/* Upper bound of outer range higher than that of the inner. */
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the outer side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &inner_lb;
+				merged_ub = &inner_ub;
+
+				/*
+				 * For a FULL join, outer relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the outer relation acts as
+				 * OUTER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_outer = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_inner = (jointype == JOIN_FULL);
+
+				if (!handle_missing_partition(outer_maps,
+											  inner_maps,
+											  outer_default,
+											  inner_part,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the inner side. */
+			Assert (!finished_inner);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+
+		if (merged_index < 0)
+		{
+			/* We didn't find a new merged partition. */
+			continue;
+		}
+
+		/*
+		 * We have a valid partition index for the next partition of join. The
+		 * partition should have valid range.
+		 */
+		Assert(merged_lb && merged_ub);
+
+		/* Try merging new lower bound with the last upper bound. */
+		if (!partition_range_merge_next_lb(partnatts, partsupfuncs,
+										   partcollations,
+										   merged_lb->datums,
+										   merged_lb->kind, &merged_datums,
+										   &merged_kinds, &merged_indexes))
+			return NULL;
+
+		/* Add upper bound with the merged partition index. */
+		merged_datums = lappend(merged_datums, merged_ub->datums);
+		merged_kinds = lappend(merged_kinds, merged_ub->kind);
+		merged_indexes = lappend_int(merged_indexes, merged_index);
+	}
+
+	if (!merge_default_partitions(outer_bi, inner_bi,
+										  outer_maps, inner_maps,
+										  jointype, &next_index,
+										  &default_index))
+		return NULL;
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_maps, inner_maps,
+								 outer_nparts, inner_nparts,
+								 jointype, next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes,
+													  merged_kinds,
+													  -1, default_index);
+	}
+
+	/* Free any memory we used in this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype)
+{
+	PartitionMap *outer_maps = NULL;
+	PartitionMap *inner_maps = NULL;
+	int			cnto;
+	int			cnti;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			next_index = 0;
+	int			null_index = -1;
+	int			default_index = -1;
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			      *outer_indexes = outer_bi->indexes;
+	int			      *inner_indexes = inner_bi->indexes;
+	int				   outer_default = outer_bi->default_index;
+	int				   inner_default = inner_bi->default_index;
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	outer_maps = init_partition_map(outer_rel);
+	inner_maps = init_partition_map(inner_rel);
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			o_index;
+		int			i_index;
+		int			cmpval;
+		int			merged_index = -1;
+		Datum	   *merged_datum;
+		bool		finished_inner;
+		bool		finished_outer;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+		{
+			finished_outer = true;
+			odatums = NULL;
+			o_index = -1;
+		}
+		else
+		{
+			finished_outer = false;
+			odatums = outer_bi->datums[cnto];
+			o_index = outer_indexes[cnto];
+		}
+
+		if (cnti >= inner_bi->ndatums)
+		{
+			finished_inner = true;
+			idatums = NULL;
+			i_index = -1;
+		}
+		else
+		{
+			finished_inner = false;
+			idatums = inner_bi->datums[cnti];
+			i_index = inner_indexes[cnti];
+		}
+
+		/* If we exhausted both the sides, we won't enter the loop. */
+		Assert(!finished_inner || !finished_outer);
+
+		if (finished_outer)
+			cmpval = 1;
+		else if (finished_inner)
+			cmpval = -1;
+		else
+		{
+			/* Every list datum should map to a valid partition index. */
+			Assert(o_index >= 0 && i_index >= 0 &&
+				   odatums != NULL && idatums != NULL);
+
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0], idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+													o_index, i_index,
+													&next_index);
+
+			if (merged_index < 0)
+				return NULL;
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			/* A datum missing from the inner side. */
+			merged_index = -1;
+			merged_datum = odatums;
+
+			/*
+			 * For a FULL join, inner relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the inner relation acts as
+			 * INNER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_inner = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_outer = (jointype == JOIN_FULL);
+
+			if (!handle_missing_partition(inner_maps,
+										  outer_maps,
+										  inner_default,
+										  o_index,
+										  missing_side_outer,
+										  missing_side_inner,
+										  &next_index,
+										  &default_index,
+										  &merged_index))
+				return NULL;
+
+			/* Move to the next datum on the outer side. */
+			Assert(!finished_outer);
+			cnto++;
+		}
+		else
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			Assert(cmpval > 0);
+
+			/* A datum missing from the outer side. */
+			merged_index = -1;
+			merged_datum = idatums;
+
+			/*
+			 * For a FULL join, outer relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the outer relation acts as
+			 * OUTER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_outer = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_inner = (jointype == JOIN_FULL);
+
+			if (!handle_missing_partition(outer_maps,
+										  inner_maps,
+										  outer_default,
+										  i_index,
+										  missing_side_outer,
+										  missing_side_inner,
+										  &next_index,
+										  &default_index,
+										  &merged_index))
+				return NULL;
+
+			/* Move to the next datum on the right side. */
+			Assert(!finished_inner);
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	if (!merge_null_partitions(outer_bi, inner_bi,
+							   outer_maps, inner_maps,
+							   jointype, &next_index, &null_index,
+							   &default_index))
+		return NULL;
+
+	if (!merge_default_partitions(outer_bi, inner_bi,
+								  outer_maps, inner_maps,
+								  jointype, &next_index,
+								  &default_index))
+		return NULL;
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_maps, inner_maps,
+								 outer_nparts, inner_nparts,
+								 jointype, next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes, NIL,
+													  null_index, default_index);
+	}
+
+	/* Free up all extra memory before returning from this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_bounds_merge()'s arm for hash partitioned tables.
+ *
+ * If the given two hash bounds are same, the function returns the first one
+ * without any change, alongwith the lists of matching partitions. Otherwise it
+ * returns NULL.
+ *
+ * We could try merging the bounds when both the bounds have same greatest
+ * modulii. But there seems to be hardly any requirement for the same.
+ */
+static PartitionBoundInfo
+partition_hash_bounds_merge(int partnatts,
+							int16 *parttyplen, bool *parttypbyval,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			cnt;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_HASH);
+
+	/*
+	 * Hash partitioned table does not have explicit NULL accepting partition
+	 * and also does not have a default partition.
+	 */
+	Assert(!partition_bound_has_default(outer_bi) &&
+		   !partition_bound_has_default(inner_bi));
+	Assert(!partition_bound_accepts_nulls(outer_bi) &&
+		   !partition_bound_accepts_nulls(inner_bi));
+
+	if (outer_rel->nparts != inner_rel->nparts)
+		return NULL;
+
+	if (!partition_bounds_equal(partnatts, parttyplen, parttypbyval,
+								outer_bi, inner_bi))
+		return NULL;
+
+	 /*
+	  * Cook up list of matching partitions. Since bounds are exactly same the
+	  * partitions at the same position from both the relations match.
+	  */
+	for (cnt = 0; cnt < outer_rel->nparts; cnt++)
+	{
+		*outer_parts = lappend_int(*outer_parts, cnt);
+		*inner_parts = lappend_int(*inner_parts, cnt);
+	}
+
+	return outer_bi;
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * If the two given partitions (given by index1 and index2 resp.) are
+ * already mapped to each other return the index of corresponding partition in
+ * the merged set of partitions.  If they do not have a merged partition
+ * associated with them, assign a new merged partition index.  If the
+ * partitions are already mapped and their mapped partitions are different from
+ * each other, they can not be merged, so return -1.
+ *
+ * partmaps1[i] gives the mapping of partitions for both relations. It
+ * describes which partition of relation 2 matches ith partition of relation 1,
+ * and which partition in the merged set matches ith partition of relation 1
+ * maps to. Similarly for partmap2.
+ *
+ * index1 and index2 are the indexes of matching partition from respective
+ * relations.
+ *
+ * *next_index is used and incremented when the given partitions require a new
+ * merged partition.
+ */
+
+static int
+map_and_merge_partitions(PartitionMap *partmaps1, PartitionMap *partmaps2,
+						 int index1, int index2, int *next_index)
+{
+	PartitionMap 	*partmap1 = &partmaps1[index1];
+	PartitionMap 	*partmap2 = &partmaps2[index2];
+	int				merged_index;
+
+	/*
+	 * If both the partitions are not mapped to each other, update the
+	 * maps.
+	 */
+	if (partmap1->from < 0 && partmap2->from < 0)
+	{
+		partmap1->from = index2;
+		partmap2->from = index1;
+	}
+
+	/*
+	 * If the given to partitions map to each other, find the corresponding
+	 * merged partition index .
+	 */
+	if (partmap1->from == index2 && partmap2->from == index1)
+	{
+		/*
+		 * If both the partitions are mapped to the same merged partition, get
+		 * the index of merged partition.
+		 */
+		if (partmap1->to == partmap2->to)
+		{
+			merged_index = partmap1->to;
+
+			/*
+			 * If the given two partitions do not have a merged partition
+			 * associated with them, allocate a new merged partition.
+			 */
+			if (merged_index < 0)
+			{
+				merged_index = *next_index;
+				*next_index = *next_index + 1;
+				partmap1->to = merged_index;
+				partmap2->to = merged_index;
+			}
+		}
+
+		/*
+		 * If partition from one relation was mapped to a merged partition but
+		 * not the partition from the other relation, map the same merged
+		 * partition to the partition from other relation, since matching
+		 * partitions map to the same merged partition.
+		 */
+		else if (partmap1->to >= 0 && partmap2->to < 0)
+		{
+			partmap2->to = partmap1->to;
+			merged_index = partmap1->to;
+		}
+		else if (partmap1->to < 0 && partmap2->to >= 0)
+		{
+			partmap1->to = partmap2->to;
+			merged_index = partmap2->to;
+		}
+		else
+		{
+			Assert(partmap1->to != partmap2->to &&
+				   partmap1->to >= 0 && partmap2->to >= 0);
+
+			/*
+			 * Both the partitions map to different merged partitions. This
+			 * means that multiple partitions from one relation matches to one
+			 * partition from the other relation. Partition-wise join does not
+			 * handle this case right now, since it requires ganging multiple
+			 * partitions together (into one RelOptInfo).
+			 */
+			merged_index = -1;
+		}
+	}
+	else
+	{
+		/*
+		 * Multiple partitions from one relation map to one partition from the
+		 * other relation. Partition-wise join does not handle this case right
+		 * now, since it requires ganging multiple partitions together (into
+		 * one RelOptInfo).
+		 */
+		merged_index = -1;
+	}
+
+	return merged_index;
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * partmaps1 map each partition from either side of the join to a merged
+ * partition resp. E.g. partmaps1[i].to gives the merged partition to which ith
+ * partition of first relation maps. Similarly for partmap2. If
+ * partmaps1[i].to == partmaps2[j].to, i and j form the matching pair of
+ * partitions.
+ *
+ * Given these maps this function produces the list pairs of partitions which
+ * when joined produce the merged partitions in the order of merged partition
+ * indexes.
+ *
+ * nparts1 and nparts2 are the number of partitions of the joining relations
+ * resp.
+ *
+ * nparts is the number of merged partitions.
+ *
+ * If successful, the pairs of partitions are returned as two separate lists,
+ * parts1 and parts2 resp., one for each side. Otherwise, those lists will be
+ * set to NIL.
+ */
+static void
+generate_matching_part_pairs(PartitionMap *partmaps1, PartitionMap *partmaps2,
+							 int nparts1, int nparts2,
+							 JoinType jointype, int nparts,
+							 List **matched_parts1, List **matched_parts2)
+{
+	bool		merged = true;
+	int		   *matching1,
+			   *matching2;
+	int 		i;
+	int			max_nparts;
+
+	*matched_parts1 = NIL;
+	*matched_parts2 = NIL;
+
+	matching1 = (int *) palloc(sizeof(int) * nparts),
+	matching2 = (int *) palloc(sizeof(int) * nparts);
+	for (i = 0; i < nparts; i++)
+		matching1[i] = matching2[i] = -1;
+
+	/* Set pairs of matching partitions. */
+	max_nparts = Max(nparts1, nparts2);
+	for (i = 0; i < max_nparts; i++)
+	{
+		if (i < nparts1)
+		{
+			PartitionMap outer_map = partmaps1[i];
+
+			if (outer_map.to >= 0)
+			{
+				Assert(outer_map.to < nparts);
+				matching1[outer_map.to] = i;
+			}
+		}
+
+		if (i < nparts2)
+		{
+			PartitionMap inner_map = partmaps2[i];
+
+			if (inner_map.to >= 0)
+			{
+				Assert(inner_map.to < nparts);
+				matching2[inner_map.to] = i;
+			}
+		}
+	}
+
+	/*
+	 * If we have a partition missing on an inner side, we need to add a dummy
+	 * relation which joins with the outer partition. If the inner relation
+	 * happens to be a base relation, it will require adding a dummy child
+	 * base relation during join processing. Right now, we freeze the base
+	 * relation arrays like PlannerInfo::simple_rte_array after planning for
+	 * base relations. Adding a new (dummy) base relation would require some
+	 * changes to that. So, right now, we do not implement partition-wise join
+	 * in such cases.
+	 */
+	for (i = 0; i < nparts; i++)
+	{
+		int			part1 = matching1[i];
+		int			part2 = matching2[i];
+
+		/* At least one of the partitions should exist. */
+		Assert(part1 >= 0 || part2 >= 0);
+
+		switch (jointype)
+		{
+			case JOIN_INNER:
+			case JOIN_SEMI:
+
+				/*
+				 * An inner or semi join can not return any row when the
+				 * matching partition on either side is missing. We should
+				 * have eliminated all such cases while merging the bounds.
+				 */
+				Assert(part1 >= 0 && part2 >= 0);
+				break;
+
+			case JOIN_LEFT:
+			case JOIN_ANTI:
+				Assert(part1 >= 0);
+				if (part2 < 0)
+					merged = false;
+				break;
+
+			case JOIN_FULL:
+				if (part1 < 0 || part2 < 0)
+					merged = false;
+				break;
+
+			default:
+				elog(ERROR, "unrecognized join type: %d", (int) jointype);
+		}
+
+		if (!merged)
+			break;
+
+		*matched_parts1 = lappend_int(*matched_parts1, part1);
+		*matched_parts2 = lappend_int(*matched_parts2, part2);
+	}
+
+	pfree(matching1);
+	pfree(matching2);
+
+	if (!merged)
+	{
+		list_free(*matched_parts1);
+		list_free(*matched_parts2);
+		*matched_parts1 = NIL;
+		*matched_parts2 = NIL;
+	}
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * Merge default partitions from both sides, if any, and assign the default
+ * partition for the join result, if necessary.
+ *
+ * If both the relations have default partitions, try mapping those to each
+ * other. If the mapping succeeds corresponding merged partition will act as
+ * the default partition of the join result.
+ *
+ * If inner side of the join has default but not the outer side, rows in it
+ * won't appear in the join result. So don't create a default partition. If
+ * outer side of the join has default but not the inner side, rows in it will
+ * appear in the join result, so create a default merged partition.
+ */
+static bool
+merge_default_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+						 PartitionMap *outer_maps, PartitionMap *inner_maps,
+						 JoinType jointype, int *next_index, int *default_index)
+{
+	int				outer_default = outer_bi->default_index;
+	int				inner_default = inner_bi->default_index;
+	bool			outer_has_default = partition_bound_has_default(outer_bi);
+	bool			inner_has_default = partition_bound_has_default(inner_bi);
+	bool			merged = true;
+	PartitionMap 	*outer_default_map = NULL;
+	PartitionMap 	*inner_default_map = NULL;
+
+	if (outer_has_default)
+		outer_default_map = &outer_maps[outer_default];
+
+	if (inner_has_default)
+		inner_default_map = &inner_maps[inner_default];
+
+	if (!outer_has_default && !inner_has_default)
+		Assert(*default_index < 0);
+	else if (outer_default_map != NULL && inner_default_map == NULL)
+	{
+		if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+			jointype == JOIN_ANTI)
+		{
+			if (outer_default_map->to < 0)
+			{
+				outer_default_map->to = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = outer_default_map->to;
+			}
+			else
+				Assert(*default_index == outer_default_map->to);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else if (outer_default_map == NULL && inner_default_map != NULL)
+	{
+		if (jointype == JOIN_FULL)
+		{
+			if (inner_default_map->to < 0)
+			{
+				inner_default_map->to = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = inner_default_map->to;
+			}
+			else
+				Assert(*default_index == inner_default_map->to);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		*default_index = map_and_merge_partitions(outer_maps, inner_maps,
+												  outer_default, inner_default,
+												  next_index);
+
+		if (*default_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
+
+/*
+ * merge_null_partitions
+ *
+ * Merge NULL partitions, i.e. a partition that can hold NULL values for a list
+ * partitioned table, if any. Find the index of merged partition to which the
+ * NULL values would belong in the join result. If one joining relation has a
+ * NULL partition but not the other, try matching it with the default partition
+ * from the other relation since the default partition may have rows with NULL
+ * partition key. We can eliminate a NULL partition when it appears only on the
+ * inner side of the join and the outer side doesn't have a default partition.
+ *
+ * When the equality operator used for join is strict, two NULL values will not
+ * be considered as equal, and thus a NULL partition can be eliminated for an
+ * inner join. But we don't check the strictness operator here.
+ */
+static bool
+merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_maps, PartitionMap *inner_maps,
+					  JoinType jointype, int *next_index,
+					  int *null_index, int *default_index)
+{
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	int			outer_ni = outer_bi->null_index;
+	int			inner_ni = inner_bi->null_index;
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	bool		merged = true;
+
+	if (!outer_has_null && !inner_has_null)
+		Assert(*null_index < 0);
+	else if (outer_has_null && !inner_has_null)
+	{
+		int			merged_index = -1;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, inner relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the inner relation acts as
+		 * INNER relation. For INNER and SEMI join OUTER and INNER
+		 * differentiation is immaterial.
+		 */
+		missing_side_inner = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_outer = (jointype == JOIN_FULL);
+
+		merged = handle_missing_partition(inner_maps,
+										  outer_maps,
+										  inner_default,
+										  outer_ni,
+										  missing_side_outer,
+										  missing_side_inner, next_index,
+										  default_index, &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join to which the outer null partition maps
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = outer_maps[outer_ni].to;
+	}
+	else if (!outer_has_null && inner_has_null)
+	{
+		int			merged_index = -1;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, outer relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the outer relation acts as OUTER
+		 * relation. For INNER and SEMI join OUTER and INNER differentiation is
+		 * immaterial.
+		 */
+		missing_side_outer = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_inner = (jointype == JOIN_FULL);
+		merged = handle_missing_partition(outer_maps,
+										  inner_maps,
+										  outer_default,
+										  inner_ni,
+										  missing_side_outer,
+										  missing_side_inner,
+										  next_index, default_index,
+										  &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join, to which the outer side null partition maps,
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = inner_maps[inner_ni].to;
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		*null_index = map_and_merge_partitions(outer_maps,
+											   inner_maps,
+											   outer_ni,
+											   inner_ni,
+											   next_index);
+		if (*null_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index 8585c29c92..636087d83e 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -16,6 +16,7 @@
 #include "nodes/pg_list.h"
 #include "partitioning/partdefs.h"
 #include "utils/relcache.h"
+struct RelOptInfo;				/* avoid including pathnodes.h here */
 
 
 /*
@@ -109,5 +110,11 @@ extern int	partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 										  int nvalues, Datum *values, bool *is_equal);
 extern int	partition_hash_bsearch(PartitionBoundInfo boundinfo,
 								   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   struct RelOptInfo *outer_rel, struct RelOptInfo *inner_rel,
+					   JoinType jointype,
+					   List **outer_parts, List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 1296edcdae..d26be918aa 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -8,59 +8,86 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-(4 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+(7 rows)
 
 -- left outer join, with whole-row reference; partitionwise join does not apply
 EXPLAIN (COSTS OFF)
@@ -72,35 +99,50 @@ SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER
    ->  Hash Right Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(22 rows)
 
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-      t1      |      t2      
---------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
- (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
- (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
- (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
- (550,0,0550) | 
-(12 rows)
+       t1       |       t2       
+----------------+----------------
+ (-250,0,-0250) | 
+ (-200,0,-0200) | 
+ (-150,0,-0150) | (0,-150,-0150)
+ (-100,0,-0100) | 
+ (-50,0,-0050)  | 
+ (0,0,0000)     | (0,0,0000)
+ (50,0,0050)    | 
+ (100,0,0100)   | 
+ (150,0,0150)   | (0,150,0150)
+ (200,0,0200)   | 
+ (250,0,0250)   | 
+ (300,0,0300)   | (0,300,0300)
+ (350,0,0350)   | 
+ (400,0,0400)   | 
+ (450,0,0450)   | (0,450,0450)
+ (500,0,0500)   | 
+ (550,0,0550)   | 
+ (600,0,0600)   | (0,600,0600)
+ (650,0,0650)   | 
+ (700,0,0700)   | 
+ (750,0,0750)   | (0,750,0750)
+(21 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
@@ -112,35 +154,53 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
    ->  Append
          ->  Hash Right Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Right Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t2_2.b)
-(20 rows)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+      |       | -225 | -0225
+      |       |  -75 | -0075
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(14 rows)
 
 -- full outer join, with placeholder vars
 EXPLAIN (COSTS OFF)
@@ -148,8 +208,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                             QUERY PLAN                            
 ------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               Filter: (((50) = prt1_p0.a) OR ((75) = prt2_p0.b))
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
@@ -174,7 +242,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               Filter: (((50) = prt1_p4.a) OR ((75) = prt2_p4.b))
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
@@ -211,35 +287,44 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Hash Right Join
-         Hash Cond: (prt2_p2.b = prt1_p1.a)
+         Hash Cond: (prt2_p2.b = prt1_p0.a)
          ->  Append
                ->  Seq Scan on prt2_p2
                      Filter: (b > 250)
                ->  Seq Scan on prt2_p3
                      Filter: (b > 250)
+               ->  Seq Scan on prt2_p4
+                     Filter: (b > 250)
          ->  Hash
                ->  Append
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p1
                            Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p2
                            Filter: ((a < 450) AND (b = 0))
-(15 rows)
+(19 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-(9 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+(14 rows)
 
 -- Currently we can't do partitioned join if nullable-side partitions are pruned
 EXPLAIN (COSTS OFF)
@@ -247,11 +332,13 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                       QUERY PLAN                      
 ------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Hash Full Join
-         Hash Cond: (prt1_p1.a = prt2_p2.b)
-         Filter: ((prt1_p1.b = 0) OR (prt2_p2.a = 0))
+         Hash Cond: (prt1_p0.a = prt2_p2.b)
+         Filter: ((prt1_p0.b = 0) OR (prt2_p2.a = 0))
          ->  Append
+               ->  Seq Scan on prt1_p0
+                     Filter: (a < 450)
                ->  Seq Scan on prt1_p1
                      Filter: (a < 450)
                ->  Seq Scan on prt1_p2
@@ -262,64 +349,147 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                            Filter: (b > 250)
                      ->  Seq Scan on prt2_p3
                            Filter: (b > 250)
-(16 rows)
+                     ->  Seq Scan on prt2_p4
+                           Filter: (b > 250)
+(20 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
-(12 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+      |       | 375 | 0375
+      |       | 450 | 0450
+      |       | 525 | 0525
+      |       | 600 | 0600
+      |       | 675 | 0675
+      |       | 750 | 0750
+(20 rows)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Semi Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Semi Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                            Filter: (a = 0)
-         ->  Nested Loop Semi Join
-               Join Filter: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
-               ->  Materialize
-                     ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
-(24 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt1_p3 t2_3
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+                           Filter: (b = 0)
+(37 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
 
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
@@ -330,27 +500,82 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
    ->  Append
          ->  Hash Anti Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Hash Anti Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Hash Anti Join
                Hash Cond: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
-(17 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(27 rows)
 
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
-  sum  |         avg          | sum  |         avg         
--------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+  sum  |         avg          | sum  |        avg         
+-------+----------------------+------+--------------------
+ 95550 | 273.0000000000000000 | 2200 | 6.2857142857142857
 (1 row)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Append
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p0 t1
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+               Index Cond: (a = t1.b)
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.b = t2_1.a)
+         ->  Seq Scan on prt2_p1 t1_1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p1 t2_1
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p2 t1_2
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+               Index Cond: (a = t1_2.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p3 t1_3
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+               Index Cond: (a = t1_3.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p4 t1_4
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+               Index Cond: (a = t1_4.b)
+(27 rows)
+
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+  b   |   c   
+------+-------
+ -225 | -0225
+  -75 | -0075
+   75 | 0075
+  225 | 0225
+  375 | 0375
+  525 | 0525
+  675 | 0675
+(7 rows)
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -362,49 +587,74 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
+                     ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
                            Index Cond: (a = t1.a)
-                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t3
                            Index Cond: (b = t2.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_1
+                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
                            Index Cond: (a = t1_1.a)
-                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3_1
                            Index Cond: (b = t2_1.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_2
+                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
                            Index Cond: (a = t1_2.a)
-                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_2
                            Index Cond: (b = t2_2.a)
-(27 rows)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                           Index Cond: (a = t1_3.a)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_3
+                           Index Cond: (b = t2_3.a)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                           Index Cond: (a = t1_4.a)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t3_4
+                           Index Cond: (b = t2_4.a)
+(43 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   | t2a | t3a | least 
------+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
- 100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
- 250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
- 400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
- 550 | 0 | 0550 |     |     |      
-(12 rows)
+  a   | b |   c   | t2a  | t3a | least 
+------+---+-------+------+-----+-------
+ -250 | 0 | -0250 |      |     |      
+ -200 | 0 | -0200 |      |     |      
+ -150 | 0 | -0150 | -150 |   0 |  -150
+ -100 | 0 | -0100 |      |     |      
+  -50 | 0 | -0050 |      |     |      
+    0 | 0 | 0000  |    0 |   0 |     0
+   50 | 0 | 0050  |      |     |      
+  100 | 0 | 0100  |      |     |      
+  150 | 0 | 0150  |  150 |   0 |   150
+  200 | 0 | 0200  |      |     |      
+  250 | 0 | 0250  |      |     |      
+  300 | 0 | 0300  |  300 |   0 |   300
+  350 | 0 | 0350  |      |     |      
+  400 | 0 | 0400  |      |     |      
+  450 | 0 | 0450  |  450 |   0 |   450
+  500 | 0 | 0500  |      |     |      
+  550 | 0 | 0550  |      |     |      
+  600 | 0 | 0600  |  600 |   0 |   600
+  650 | 0 | 0650  |      |     |      
+  700 | 0 | 0700  |      |     |      
+  750 | 0 | 0750  |  750 |   0 |   750
+(21 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
@@ -418,64 +668,95 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
          Hash Cond: ((t1.c)::text = (t2.c)::text)
          Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Seq Scan on prt1_p1 t1
-               ->  Seq Scan on prt1_p2 t1_1
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
          ->  Hash
                ->  Append
                      ->  Hash Join
                            Hash Cond: (t2.a = t3.b)
-                           ->  Seq Scan on prt1_p1 t2
+                           ->  Seq Scan on prt1_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t3
+                                 ->  Seq Scan on prt2_p0 t3
                      ->  Hash Join
                            Hash Cond: (t2_1.a = t3_1.b)
-                           ->  Seq Scan on prt1_p2 t2_1
+                           ->  Seq Scan on prt1_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t3_1
+                                 ->  Seq Scan on prt2_p1 t3_1
                      ->  Hash Join
                            Hash Cond: (t2_2.a = t3_2.b)
-                           ->  Seq Scan on prt1_p3 t2_2
+                           ->  Seq Scan on prt1_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p3 t3_2
-(26 rows)
+                                 ->  Seq Scan on prt2_p2 t3_2
+                     ->  Hash Join
+                           Hash Cond: (t2_3.a = t3_3.b)
+                           ->  Seq Scan on prt1_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t3_3
+                     ->  Hash Join
+                           Hash Cond: (t2_4.a = t3_4.b)
+                           ->  Seq Scan on prt1_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t3_4
+(38 rows)
 
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, t2.b t2b, t2.c t2c, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
-  a  | t2a | t2c  
------+-----+------
-   0 |   0 | 0000
-  50 |     | 
- 100 |     | 
- 150 | 150 | 0150
- 200 |     | 
- 250 |     | 
- 300 | 300 | 0300
- 350 |     | 
- 400 |     | 
- 450 | 450 | 0450
- 500 |     | 
- 550 |     | 
-(12 rows)
+  a   | t2a  |  t2c  
+------+------+-------
+ -250 |      | 
+ -200 |      | 
+ -150 | -150 | -0150
+ -100 |      | 
+  -50 |      | 
+    0 |    0 | 0000
+   50 |      | 
+  100 |      | 
+  150 |  150 | 0150
+  200 |      | 
+  250 |      | 
+  300 |  300 | 0300
+  350 |      | 
+  400 |      | 
+  450 |  450 | 0450
+  500 |      | 
+  550 |      | 
+  600 |  600 | 0600
+  650 |      | 
+  700 |      | 
+  750 |  750 | 0750
+(21 rows)
 
 --
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
@@ -486,32 +767,49 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
    ->  Append
          ->  Hash Join
                Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
-               ->  Seq Scan on prt2_e_p1 t2
+               ->  Seq Scan on prt2_e_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 t1
+                     ->  Seq Scan on prt1_e_p0 t1
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
-               ->  Seq Scan on prt2_e_p2 t2_1
+               Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+               ->  Seq Scan on prt1_e_p1 t1_1
+                     Filter: (c = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 t1_1
-                           Filter: (c = 0)
+                     ->  Seq Scan on prt2_e_p1 t2_1
          ->  Hash Join
                Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
-               ->  Seq Scan on prt2_e_p3 t2_2
+               ->  Seq Scan on prt2_e_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 t1_2
+                     ->  Seq Scan on prt1_e_p2 t1_2
                            Filter: (c = 0)
-(21 rows)
+         ->  Hash Join
+               Hash Cond: (((t2_3.b + t2_3.a) / 2) = ((t1_3.a + t1_3.b) / 2))
+               ->  Seq Scan on prt2_e_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p3 t1_3
+                           Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (((t2_4.b + t2_4.a) / 2) = ((t1_4.a + t1_4.b) / 2))
+               ->  Seq Scan on prt2_e_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4 t1_4
+                           Filter: (c = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
- 150 | 0 | 150 | 0
- 300 | 0 | 300 | 0
- 450 | 0 | 450 | 0
-(4 rows)
+  a   | c |  b   | c 
+------+---+------+---
+ -250 | 0 | -250 | 0
+ -100 | 0 | -100 | 0
+    0 | 0 |    0 | 0
+    0 | 0 |    0 | 0
+  150 | 0 |  150 | 0
+  300 | 0 |  300 | 0
+  450 | 0 |  450 | 0
+  600 | 0 |  600 | 0
+  750 | 0 |  750 | 0
+(9 rows)
 
 --
 -- N-way join
@@ -524,154 +822,232 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = ((t3.a + t3.b) / 2))
+               Join Filter: (t1.a = t2.b)
                ->  Hash Join
-                     Hash Cond: (t2.b = t1.a)
-                     ->  Seq Scan on prt2_p1 t2
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on prt1_e_p0 t3
                      ->  Hash
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                     Index Cond: (((a + b) / 2) = t2.b)
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
+                     Index Cond: (b = ((t3.a + t3.b) / 2))
          ->  Nested Loop
                Join Filter: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_1.b = t1_1.a)
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
+               ->  Index Scan using iprt1_e_p4_ab2 on prt1_e_p1 t3_1
                      Index Cond: (((a + b) / 2) = t2_1.b)
          ->  Nested Loop
                Join Filter: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_2.b = t1_2.a)
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
+               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_2
                      Index Cond: (((a + b) / 2) = t2_2.b)
-(33 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = ((t3_3.a + t3_3.b) / 2))
+               ->  Nested Loop
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t2_3.b)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t2_4.b)
+               ->  Hash Join
+                     Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                     ->  Seq Scan on prt1_e_p4 t3_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_p4 t1_4
+                                 Filter: (b = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = ((t3_4.a + t3_4.b) / 2))
+(52 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
-(4 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(8 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                          QUERY PLAN                          
---------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Hash Right Join
                Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
-               ->  Seq Scan on prt1_e_p1 t3
+               ->  Seq Scan on prt1_e_p0 t3
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2.b = t1.a)
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_1.a)
-               ->  Seq Scan on prt1_e_p2 t3_1
+               ->  Seq Scan on prt1_e_p1 t3_1
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_1.b = t1_1.a)
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_2.a)
-               ->  Seq Scan on prt1_e_p3 t3_2
+               ->  Seq Scan on prt1_e_p2 t3_2
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_2.b = t1_2.a)
-                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                                        Filter: (b = 0)
-(33 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+               ->  Seq Scan on prt1_e_p4 t3_4
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_4.b = t1_4.a)
+                           ->  Seq Scan on prt2_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_p4 t1_4
+                                       Filter: (b = 0)
+(51 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                            QUERY PLAN                             
--------------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p1 t3
+                           ->  Seq Scan on prt1_e_p0 t3
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p1_b on prt2_p1 t2
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
                      Index Cond: (b = t1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p2 t3_1
+                           ->  Seq Scan on prt1_e_p1 t3_1
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
+               ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
                      Index Cond: (b = t1_1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p3 t3_2
+                           ->  Seq Scan on prt1_e_p2 t3_2
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
                      Index Cond: (b = t1_2.a)
-(30 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_e_p3 t3_3
+                           Filter: (c = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                           Index Cond: (a = ((t3_3.a + t3_3.b) / 2))
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Nested Loop Left Join
+               ->  Hash Right Join
+                     Hash Cond: (t1_4.a = ((t3_4.a + t3_4.b) / 2))
+                     ->  Seq Scan on prt1_p4 t1_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p4 t3_4
+                                 Filter: (c = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = t1_4.a)
+(47 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -680,21 +1056,34 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                                                    QUERY PLAN                                                   
 ----------------------------------------------------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   Sort Key: prt1_p0.a, prt2_p0.b, ((prt1_e_p0.a + prt1_e_p0.b))
    ->  Append
          ->  Hash Full Join
-               Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
-               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               Hash Cond: (prt1_p0.a = ((prt1_e_p0.a + prt1_e_p0.b) / 2))
+               Filter: ((prt1_p0.a = (50)) OR (prt2_p0.b = (75)) OR (((prt1_e_p0.a + prt1_e_p0.b) / 2) = (50)))
                ->  Hash Full Join
-                     Hash Cond: (prt1_p1.a = prt2_p1.b)
-                     ->  Seq Scan on prt1_p1
+                     Hash Cond: (prt1_p0.a = prt2_p0.b)
+                     ->  Seq Scan on prt1_p0
                            Filter: (b = 0)
                      ->  Hash
-                           ->  Seq Scan on prt2_p1
+                           ->  Seq Scan on prt2_p0
                                  Filter: (a = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1
+                     ->  Seq Scan on prt1_e_p0
                            Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = prt1_p1.a)
+               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               ->  Seq Scan on prt1_e_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on prt1_p1
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1
+                                       Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
                Filter: ((prt1_p2.a = (50)) OR (prt2_p2.b = (75)) OR (((prt1_e_p2.a + prt1_e_p2.b) / 2) = (50)))
@@ -721,7 +1110,20 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                ->  Hash
                      ->  Seq Scan on prt1_e_p3
                            Filter: (c = 0)
-(42 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = ((prt1_e_p4.a + prt1_e_p4.b) / 2))
+               Filter: ((prt1_p4.a = (50)) OR (prt2_p4.b = (75)) OR (((prt1_e_p4.a + prt1_e_p4.b) / 2) = (50)))
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = prt2_p4.b)
+                     ->  Seq Scan on prt1_p4
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4
+                                 Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4
+                           Filter: (c = 0)
+(68 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
@@ -739,172 +1141,260 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = t1_3.b)
+               Join Filter: (t1.a = t1_5.b)
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Join
-                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
-                           ->  Seq Scan on prt1_e_p1 t2
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t1_3
+                                 ->  Seq Scan on prt2_p0 t1_5
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
                      Index Cond: (a = ((t2.a + t2.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_1.a = t1_4.b)
+               Join Filter: (t1_1.a = t1_6.b)
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Join
-                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
-                           ->  Seq Scan on prt1_e_p2 t2_1
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_6.b)
+                           ->  Seq Scan on prt1_e_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t1_4
+                                 ->  Seq Scan on prt2_p1 t1_6
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
                      Index Cond: (a = ((t2_1.a + t2_1.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_2.a = t1_5.b)
+               Join Filter: (t1_2.a = t1_7.b)
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
+                           ->  Seq Scan on prt2_p2 t1_7
                                  Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
+                           ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t2_2
+                                 Index Cond: (((a + b) / 2) = t1_7.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t1_8.b)
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Nested Loop
+                           ->  Seq Scan on prt2_p3 t1_8
+                                 Filter: (a = 0)
+                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_3
+                                 Index Cond: (((a + b) / 2) = t1_8.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = ((t2_3.a + t2_3.b) / 2))
+                     Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t1_9.b)
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Join
+                           Hash Cond: (((t2_4.a + t2_4.b) / 2) = t1_9.b)
+                           ->  Seq Scan on prt1_e_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t1_9
+                                       Filter: (a = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = ((t2_4.a + t2_4.b) / 2))
+                     Filter: (b = 0)
+(66 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                               QUERY PLAN                                
--------------------------------------------------------------------------
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
-                           ->  Seq Scan on prt2_p1 t1_3
+                           Hash Cond: (t1_5.b = ((t1_10.a + t1_10.b) / 2))
+                           ->  Seq Scan on prt2_p0 t1_5
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t1_6
+                                 ->  Seq Scan on prt1_e_p0 t1_10
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
-                     Index Cond: (a = t1_3.b)
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t1_5.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
-                           ->  Seq Scan on prt2_p2 t1_4
+                           Hash Cond: (t1_6.b = ((t1_11.a + t1_11.b) / 2))
+                           ->  Seq Scan on prt2_p1 t1_6
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t1_7
+                                 ->  Seq Scan on prt1_e_p1 t1_11
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
-                     Index Cond: (a = t1_4.b)
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
+                     Index Cond: (a = t1_6.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
-                           ->  Seq Scan on prt2_p3 t1_5
+                           Hash Cond: (t1_7.b = ((t1_12.a + t1_12.b) / 2))
+                           ->  Seq Scan on prt2_p2 t1_7
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p3 t1_8
+                                 ->  Seq Scan on prt1_e_p2 t1_12
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t1_5.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t1_7.b)
                      Filter: (b = 0)
-(39 rows)
-
-SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_8.b = ((t1_13.a + t1_13.b) / 2))
+                           ->  Seq Scan on prt2_p3 t1_8
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p3 t1_13
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t1_8.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_9.b = ((t1_14.a + t1_14.b) / 2))
+                           ->  Seq Scan on prt2_p4 t1_9
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t1_14
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = t1_9.b)
+                     Filter: (b = 0)
+(63 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 -- test merge joins
 SET enable_hashjoin TO off;
 SET enable_nestloop TO off;
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                           QUERY PLAN                           
-----------------------------------------------------------------
+                            QUERY PLAN                            
+------------------------------------------------------------------
  Merge Append
    Sort Key: t1.a
    ->  Merge Semi Join
-         Merge Cond: (t1.a = t1_3.b)
+         Merge Cond: (t1.a = t1_5.b)
          ->  Sort
                Sort Key: t1.a
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+               Merge Cond: (t1_5.b = (((t1_10.a + t1_10.b) / 2)))
                ->  Sort
-                     Sort Key: t1_3.b
-                     ->  Seq Scan on prt2_p1 t1_3
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on prt2_p0 t1_5
                ->  Sort
-                     Sort Key: (((t1_6.a + t1_6.b) / 2))
-                     ->  Seq Scan on prt1_e_p1 t1_6
+                     Sort Key: (((t1_10.a + t1_10.b) / 2))
+                     ->  Seq Scan on prt1_e_p0 t1_10
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_1.a = t1_4.b)
+         Merge Cond: (t1_1.a = t1_6.b)
          ->  Sort
                Sort Key: t1_1.a
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+               Merge Cond: (t1_6.b = (((t1_11.a + t1_11.b) / 2)))
                ->  Sort
-                     Sort Key: t1_4.b
-                     ->  Seq Scan on prt2_p2 t1_4
+                     Sort Key: t1_6.b
+                     ->  Seq Scan on prt2_p1 t1_6
                ->  Sort
-                     Sort Key: (((t1_7.a + t1_7.b) / 2))
-                     ->  Seq Scan on prt1_e_p2 t1_7
+                     Sort Key: (((t1_11.a + t1_11.b) / 2))
+                     ->  Seq Scan on prt1_e_p1 t1_11
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_2.a = t1_5.b)
+         Merge Cond: (t1_2.a = t1_7.b)
          ->  Sort
                Sort Key: t1_2.a
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+               Merge Cond: (t1_7.b = (((t1_12.a + t1_12.b) / 2)))
                ->  Sort
-                     Sort Key: t1_5.b
-                     ->  Seq Scan on prt2_p3 t1_5
+                     Sort Key: t1_7.b
+                     ->  Seq Scan on prt2_p2 t1_7
                ->  Sort
-                     Sort Key: (((t1_8.a + t1_8.b) / 2))
-                     ->  Seq Scan on prt1_e_p3 t1_8
+                     Sort Key: (((t1_12.a + t1_12.b) / 2))
+                     ->  Seq Scan on prt1_e_p2 t1_12
                            Filter: (c = 0)
-(47 rows)
+   ->  Merge Semi Join
+         Merge Cond: (t1_3.a = t1_8.b)
+         ->  Sort
+               Sort Key: t1_3.a
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_8.b = (((t1_13.a + t1_13.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_8.b
+                     ->  Seq Scan on prt2_p3 t1_8
+               ->  Sort
+                     Sort Key: (((t1_13.a + t1_13.b) / 2))
+                     ->  Seq Scan on prt1_e_p3 t1_13
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_4.a = t1_9.b)
+         ->  Sort
+               Sort Key: t1_4.a
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_9.b = (((t1_14.a + t1_14.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_9.b
+                     ->  Seq Scan on prt2_p4 t1_9
+               ->  Sort
+                     Sort Key: (((t1_14.a + t1_14.b) / 2))
+                     ->  Seq Scan on prt1_e_p4 t1_14
+                           Filter: (c = 0)
+(77 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
@@ -921,14 +1411,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
                            ->  Sort
                                  Sort Key: (((t3.a + t3.b) / 2))
-                                 ->  Seq Scan on prt1_e_p1 t3
+                                 ->  Seq Scan on prt1_e_p0 t3
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1.a
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                ->  Sort
                      Sort Key: t2.b
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Merge Left Join
                Merge Cond: (t1_1.a = t2_1.b)
                ->  Sort
@@ -937,14 +1427,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
                            ->  Sort
                                  Sort Key: (((t3_1.a + t3_1.b) / 2))
-                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                 ->  Seq Scan on prt1_e_p1 t3_1
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_1.a
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                ->  Sort
                      Sort Key: t2_1.b
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Merge Left Join
                Merge Cond: (t1_2.a = t2_2.b)
                ->  Sort
@@ -953,32 +1443,74 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
                            ->  Sort
                                  Sort Key: (((t3_2.a + t3_2.b) / 2))
-                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                 ->  Seq Scan on prt1_e_p2 t3_2
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_2.a
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                ->  Sort
                      Sort Key: t2_2.b
-                     ->  Seq Scan on prt2_p3 t2_2
-(51 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Merge Left Join
+               Merge Cond: (t1_3.a = t2_3.b)
+               ->  Sort
+                     Sort Key: t1_3.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_3.a + t3_3.b) / 2)) = t1_3.a)
+                           ->  Sort
+                                 Sort Key: (((t3_3.a + t3_3.b) / 2))
+                                 ->  Seq Scan on prt1_e_p3 t3_3
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_3.a
+                                 ->  Seq Scan on prt1_p3 t1_3
+               ->  Sort
+                     Sort Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Merge Left Join
+               Merge Cond: (t1_4.a = t2_4.b)
+               ->  Sort
+                     Sort Key: t1_4.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_4.a + t3_4.b) / 2)) = t1_4.a)
+                           ->  Sort
+                                 Sort Key: (((t3_4.a + t3_4.b) / 2))
+                                 ->  Seq Scan on prt1_e_p4 t3_4
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_4.a
+                                 ->  Seq Scan on prt1_p4 t1_4
+               ->  Sort
+                     Sort Key: t2_4.b
+                     ->  Seq Scan on prt2_p4 t2_4
+(83 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- MergeAppend on nullable column
 -- This should generate a partitionwise join, but currently fails to
@@ -987,12 +1519,14 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Merge Left Join
-         Merge Cond: (prt1_p1.a = prt2_p2.b)
+         Merge Cond: (prt1_p0.a = prt2_p2.b)
          ->  Sort
-               Sort Key: prt1_p1.a
+               Sort Key: prt1_p0.a
                ->  Append
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p1
                            Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p2
@@ -1004,21 +1538,28 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                            Filter: (b > 250)
                      ->  Seq Scan on prt2_p3
                            Filter: (b > 250)
-(18 rows)
+                     ->  Seq Scan on prt2_p4
+                           Filter: (b > 250)
+(22 rows)
 
 SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  b  
------+-----
-   0 |    
-  50 |    
- 100 |    
- 150 |    
- 200 |    
- 250 |    
- 300 | 300
- 350 |    
- 400 |    
-(9 rows)
+  a   |  b  
+------+-----
+ -250 |    
+ -200 |    
+ -150 |    
+ -100 |    
+  -50 |    
+    0 |    
+   50 |    
+  100 |    
+  150 |    
+  200 |    
+  250 |    
+  300 | 300
+  350 |    
+  400 |    
+(14 rows)
 
 -- merge join when expression with whole-row reference needs to be sorted;
 -- partitionwise join does not apply
@@ -1032,175 +1573,2412 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
          Sort Key: t1.a, ((((t1.*)::prt1))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
    ->  Sort
          Sort Key: t2.b, ((((t2.*)::prt2))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt2_p1 t2
-                     ->  Seq Scan on prt2_p2 t2_1
-                     ->  Seq Scan on prt2_p3 t2_2
-(16 rows)
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+(20 rows)
 
 SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.b ORDER BY t1.a;
- a  | b  
-----+----
-  0 |  0
-  6 |  6
- 12 | 12
- 18 | 18
- 24 | 24
-(5 rows)
+  a  |  b  
+-----+-----
+ -24 | -24
+ -18 | -18
+ -12 | -12
+  -6 |  -6
+   0 |   0
+   6 |   6
+  12 |  12
+  18 |  18
+  24 |  24
+(9 rows)
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p4 t2_3
+               ->  Seq Scan on prt2_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_4
+                           Filter: (b = 0)
+(22 rows)
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p3 t1_2
+                           Filter: (b = 0)
+(21 rows)
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -150 | -0150 | 0 | -0150
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+(7 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 |   | 
+ -200 | -0200 |   | 
+ -150 | -0150 | 0 | -0150
+ -100 | -0100 |   | 
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+(21 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -200 | 0 | -0200
+ -100 | 0 | -0100
+  -50 | 0 | -0050
+   50 | 0 | 0050
+  100 | 0 | 0100
+  200 | 0 | 0200
+  250 | 0 | 0250
+  350 | 0 | 0350
+  400 | 0 | 0400
+  500 | 0 | 0500
+  550 | 0 | 0550
+  650 | 0 | 0650
+  700 | 0 | 0700
+(14 rows)
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+                             QUERY PLAN                              
+---------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t3.c
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p0_a on prt1_p0 t3
+                           Index Cond: (a = t2.b)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t2.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_1.a = t2_1.b)
+                           ->  Seq Scan on prt1_p1 t3_1
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1 t2_1
+                                       Filter: (a = 0)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p2_a on prt1_p2 t3_2
+                           Index Cond: (a = t2_2.b)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t3_3
+                           Index Cond: (a = t2_3.b)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_4.a = t2_4.b)
+                           ->  Seq Scan on prt1_p4 t3_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t2_4
+                                       Filter: (a = 0)
+(47 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+  a   | a |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.b
+   ->  Hash Right Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p5 t2_5
+                           Filter: (a = 0)
+(24 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: (t1.a = t2.b)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.a, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
+                     Filter: (a = 0)
+(22 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
+               ->  Seq Scan on prt1_m_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p1
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
+               ->  Seq Scan on prt1_m_p2
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p2
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
+               ->  Seq Scan on prt1_m_p3
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p3
+                           Filter: (c = 0)
+(24 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+  a  | c |  b  | c 
+-----+---+-----+---
+   0 | 0 |   0 | 0
+  50 | 0 |     |  
+ 100 | 0 |     |  
+ 150 | 0 | 150 | 0
+ 200 | 0 |     |  
+ 250 | 0 |     |  
+ 300 | 0 | 300 | 0
+ 350 | 0 |     |  
+ 400 | 0 |     |  
+ 450 | 0 | 450 | 0
+ 500 | 0 |     |  
+ 550 | 0 |     |  
+     |   |  75 | 0
+     |   | 225 | 0
+     |   | 375 | 0
+     |   | 525 | 0
+(16 rows)
+
+--
+-- tests for list partitioned tables.
+--
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
+--
+-- list partitioned by expression
+--
+CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
+ANALYZE plt1_e;
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+                                           QUERY PLAN                                           
+------------------------------------------------------------------------------------------------
+ GroupAggregate
+   Group Key: t1.c, t2.c, t3.c
+   ->  Sort
+         Sort Key: t1.c, t3.c
+         ->  Append
+               ->  Hash Join
+                     Hash Cond: ((t1.c)::text = ltrim(t3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2.b = t1.b) AND ((t2.c)::text = (t1.c)::text))
+                           ->  Seq Scan on plt2_p4 t2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p4 t1
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p4 t3
+               ->  Hash Join
+                     Hash Cond: ((t1_1.c)::text = ltrim(t3_1.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t1_1.b = t2_1.b) AND ((t1_1.c)::text = (t2_1.c)::text))
+                           ->  Seq Scan on plt1_p1 t1_1
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p1 t2_1
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p1 t3_1
+               ->  Hash Join
+                     Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t1_2.b = t2_2.b) AND ((t1_2.c)::text = (t2_2.c)::text))
+                           ->  Seq Scan on plt1_p2 t1_2
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p2 t2_2
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p2 t3_2
+               ->  Hash Join
+                     Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2_3.b = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text))
+                           ->  Seq Scan on plt2_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p3 t3_3
+(41 rows)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p5 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_3
+                                 ->  Seq Scan on plt2_p3 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
 
-RESET enable_hashjoin;
-RESET enable_nestloop;
---
--- partitioned by multiple columns
---
-CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
-CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
-ANALYZE prt1_m;
-CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
-CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
-ANALYZE prt2_m;
+-- semi join
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-                                                             QUERY PLAN                                                             
-------------------------------------------------------------------------------------------------------------------------------------
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
  Sort
-   Sort Key: prt1_m_p1.a, prt2_m_p1.b
-   ->  Append
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
-               ->  Seq Scan on prt1_m_p1
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p1
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
-               ->  Seq Scan on prt1_m_p2
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p2
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
-               ->  Seq Scan on prt1_m_p3
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p3
-                           Filter: (c = 0)
-(24 rows)
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Seq Scan on plt2_p2 t2_2
+                                 ->  Seq Scan on plt2_p3 t2_3
+(22 rows)
 
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
-  50 | 0 |     |  
- 100 | 0 |     |  
- 150 | 0 | 150 | 0
- 200 | 0 |     |  
- 250 | 0 |     |  
- 300 | 0 | 300 | 0
- 350 | 0 |     |  
- 400 | 0 |     |  
- 450 | 0 | 450 | 0
- 500 | 0 |     |  
- 550 | 0 |     |  
-     |   |  75 | 0
-     |   | 225 | 0
-     |   | 375 | 0
-     |   | 525 | 0
-(16 rows)
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
 
---
--- tests for list partitioned tables.
---
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
---
--- list partitioned by expression
---
-CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1_e;
--- test partition matching with N-way join
+-- anti join
 EXPLAIN (COSTS OFF)
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                   QUERY PLAN                                   
---------------------------------------------------------------------------------
- GroupAggregate
-   Group Key: t1.c, t2.c, t3.c
-   ->  Sort
-         Sort Key: t1.c, t3.c
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
          ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.c = ltrim(t3.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1.b = t2.b) AND (t1.c = t2.c))
-                           ->  Seq Scan on plt1_p1 t1
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.c = ltrim(t3_1.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
-                           ->  Seq Scan on plt1_p2 t1_1
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.c = ltrim(t3_2.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
-                           ->  Seq Scan on plt1_p3 t1_2
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p3 t3_2
-(32 rows)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+(19 rows)
 
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  75.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 123.0000000000000000 | 123.0000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 225.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 273.0000000000000000 | 273.0000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 375.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 423.0000000000000000 | 423.0000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 525.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 573.0000000000000000 | 573.0000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1225,22 +4003,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
 --------------------------------------------------
  Hash Left Join
    Hash Cond: (t2.b = a)
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t3.a = t2.b)
-               ->  Seq Scan on prt1_p1 t3
-               ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
-         ->  Hash Join
-               Hash Cond: (t3_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t3_1
-               ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
-         ->  Hash Join
-               Hash Cond: (t3_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t3_2
-               ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
+   ->  Hash Join
+         Hash Cond: (t3.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t3
+               ->  Seq Scan on prt1_p1 t3_1
+               ->  Seq Scan on prt1_p2 t3_2
+               ->  Seq Scan on prt1_p3 t3_3
+               ->  Seq Scan on prt1_p4 t3_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
    ->  Hash
          ->  Result
                One-Time Filter: false
@@ -1255,16 +4033,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 --
 -- tests for hash partitioned tables.
@@ -1340,41 +4124,9 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  273.0000000000000000 | 273.0000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
 (6 rows)
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
- Sort
-   Sort Key: t1.a
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
-               ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
-               ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
-                           Filter: (b = 0)
-(21 rows)
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
@@ -1389,26 +4141,24 @@ SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c
    Sort Key: t1.c
    ->  HashAggregate
          Group Key: t1.c, t2.c
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t2.c = t1.c)
-                     ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p1 t1
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on plt1_p4 t1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_1.c = t1_1.c)
-                     ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_p2 t1_1
+                           ->  Seq Scan on plt1_p1 t1_1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_2.c = t1_2.c)
-                     ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Seq Scan on plt1_p2 t1_2
                                  Filter: ((a % 25) = 0)
-(23 rows)
+                           ->  Seq Scan on plt1_p3 t1_3
+                                 Filter: ((a % 25) = 0)
+(21 rows)
 
 --
 -- multiple levels of partitioning
@@ -1804,64 +4554,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
  Hash Join
    Hash Cond: (t1.a = t2.a)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt4_n_p1 t2
                ->  Seq Scan on prt4_n_p2 t2_1
                ->  Seq Scan on prt4_n_p3 t2_2
-(11 rows)
+(13 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a and t1.a = t3.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                        QUERY PLAN                        
+----------------------------------------------------------
  Hash Join
-   Hash Cond: (t2.a = t1.a)
+   Hash Cond: (t3.b = t1.a)
    ->  Append
-         ->  Seq Scan on prt4_n_p1 t2
-         ->  Seq Scan on prt4_n_p2 t2_1
-         ->  Seq Scan on prt4_n_p3 t2_2
+         ->  Seq Scan on prt2_p0 t3
+         ->  Seq Scan on prt2_p1 t3_1
+         ->  Seq Scan on prt2_p2 t3_2
+         ->  Seq Scan on prt2_p3 t3_3
+         ->  Seq Scan on prt2_p4 t3_4
+         ->  Seq Scan on prt2_p5 t3_5
    ->  Hash
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.a = t3_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_2.b)
-                     ->  Seq Scan on prt1_p3 t1_2
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3 t3_2
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt4_n_p1 t2
+                           ->  Seq Scan on prt4_n_p2 t2_1
+                           ->  Seq Scan on prt4_n_p3 t2_2
 (23 rows)
 
 -- partitionwise join can not be applied if there are no equi-join conditions
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (b > t1.a)
-(12 rows)
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+(16 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partitionwise join
@@ -1947,16 +4703,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
-                     ->  Seq Scan on plt1_p1 t3
-                     ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3
+                     ->  Seq Scan on plt1_p1 t3_1
+                     ->  Seq Scan on plt1_p2 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partitionwise join can not be applied for a join between list and range
 -- partitioned table
@@ -1967,14 +4724,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
  Hash Full Join
    Hash Cond: ((t2.c)::text = (t1.c)::text)
    ->  Append
-         ->  Seq Scan on prt1_p1 t2
-         ->  Seq Scan on prt1_p2 t2_1
-         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p0 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p2 t2_2
+         ->  Seq Scan on prt1_p3 t2_3
+         ->  Seq Scan on prt1_p4 t2_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt1_n_p1 t1
                ->  Seq Scan on prt1_n_p2 t1_1
-(10 rows)
+(12 rows)
 
 -- partitionwise join can not be applied if only one of joining table has
 -- default partition
@@ -1990,16 +4749,23 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
    ->  Hash Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index db9a6b4a96..7597b069fa 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -10,25 +10,39 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -69,11 +83,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -95,20 +117,30 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 
 EXPLAIN (COSTS OFF)
@@ -172,6 +204,128 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -196,28 +350,79 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
 -- test partition matching with N-way join
@@ -225,6 +430,175 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
@@ -270,27 +644,18 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
 ALTER TABLE plt2 DETACH PARTITION plt2_p3;
 ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
 ANALYZE plt2;
-
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.a % 25 = 0 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
 --
 -- multiple levels of partitioning
 --
#82Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Etsuro Fujita (#81)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Mon, Jul 8, 2019 at 8:33 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

I'll review the remaining parts (ie,
"0002-Partition-wise-join-for-1-1-1-0-0-1-partition-matchi-v22.patch"
and "0003-Tests-for-0-1-1-1-and-1-0-partition-matching-v22.patch")
closely next.

I've been reviewing the main patch
"0002-Partition-wise-join-for-1-1-1-0-0-1-partition-matchi-v22.patch",
but it's pretty large and complicated, so I'm still at the first pass,
and so maybe I'm missing something, but let me comment a few things on
the patch. First of all, I think the patch's problem setting, which
is stated in the commit message, is reasonable:

Earlier version of partition-wise join implementation allowed
partition-wise join between two relations with exactly same partition
bounds. This commit allows partition-wise join to be applied under
following conditions

1. the partition bounds of joining relations are such that rows from
given partition on one side join can join with rows from maximum one
partition on the other side i.e. bounds of a given partition on one
side match/overlap with those of maximum one partition on the other
side.

And I agree with the patch's approach to this that tries to find such
a partition mapping between two input partitioned relations for a join
relation, by trying to match their partition bounds, which is
implemented in a new function partition_bounds_merge(), in general.
But one thing that I'm concerned about most at this point is this in
try_partitionwise_join():

    /*
-    * Since we allow partitionwise join only when the partition bounds of the
-    * joining relations exactly match, the partition bounds of the join
-    * should match those of the joining relations.
+    * Get the list of matching partitions to be joined along with the
+    * partition bounds of the join relation. Because of the restrictions
+    * imposed by partition matching algorithm, not every pair of joining
+    * relations for this join will be able to use partition-wise join. But all
+    * those pairs which can use partition-wise join will produce the same
+    * partition bounds for the join relation.
     */
-   Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-                                 joinrel->part_scheme->parttyplen,
-                                 joinrel->part_scheme->parttypbyval,
-                                 joinrel->boundinfo, rel1->boundinfo));
-   Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-                                 joinrel->part_scheme->parttyplen,
-                                 joinrel->part_scheme->parttypbyval,
-                                 joinrel->boundinfo, rel2->boundinfo));
+   join_boundinfo = partition_bounds_merge(part_scheme->partnatts,
+                                           part_scheme->partsupfunc,
+                                           part_scheme->partcollation,
+                                           rel1, rel2,
+                                           parent_sjinfo->jointype,
+                                           &parts1, &parts2);
+
+   if (join_boundinfo == NULL)
+       return;

I.e., partition_bounds_merge() is performed for each pair of input
partitioned relations for a join relation in try_partitionwise_join().
Since partition_bounds_merge() would need a lot of CPU cycles, I don't
think this is acceptable; ISTM that some redesign is needed to avoid
this. I'm wondering that once we successfully merged partition bounds
from a pair of input partitioned relations for the join relation, by
using the merged partition bounds, we could get the lists of matching
to-be-joined partitions for subsequent pairs of input partitioned
relations for the join relation in a more efficient way than by
performing partition_bounds_merge() as proposed in the patch.

Best regards,
Etsuro Fujita

#83amul sul
sulamul@gmail.com
In reply to: Etsuro Fujita (#81)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Mon, Jul 8, 2019 at 5:03 PM Etsuro Fujita <etsuro.fujita@gmail.com>
wrote:

On Wed, Jul 3, 2019 at 3:44 PM Etsuro Fujita <etsuro.fujita@gmail.com>
wrote:

On Tue, Jul 2, 2019 at 1:47 PM amul sul <sulamul@gmail.com> wrote:

Attached version is rebase atop of the latest master head(c74d49d41c),

thanks.

Thanks! Will review.

I started reviewing this. Here is my initial review comments:

* 0001-Hash-partition-bound-equality-refactoring-v22.patch

First of all, I agree with your view on hash partitioning:

"3. For hash partitioned tables however, we support partition-wise join
only when the bounds exactly match. For hash partitioning it's unusual
to have missing partitions and hence generic partition matching is not
required."

which is cited from the commit message for the main patch
"0002-Partition-wise-join-for-1-1-1-0-0-1-partition-matchi-v22.patch".
(I think it would be better if we can extend the partition matching to
the hash-partitioning case where there are missing partitions in
future, though.) However, I don't think it's a good idea to do this
refactoring, because that would lead to duplicating the code to check
whether two given hash bound collections are equal in
partition_bounds_equal() and partition_hash_bounds_merge() that will
be added by the main patch, after all. To avoid that, how about
calling partition_bounds_equal() from partition_hash_bounds_merge() in
the main patch, like the attached?

Agree, your changes look good to me, thanks for working on it.

Regards,
Amul

#84Robert Haas
robertmhaas@gmail.com
In reply to: Etsuro Fujita (#82)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Thu, Jul 18, 2019 at 2:55 AM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

I.e., partition_bounds_merge() is performed for each pair of input
partitioned relations for a join relation in try_partitionwise_join().
Since partition_bounds_merge() would need a lot of CPU cycles, I don't
think this is acceptable; ISTM that some redesign is needed to avoid
this. I'm wondering that once we successfully merged partition bounds
from a pair of input partitioned relations for the join relation, by
using the merged partition bounds, we could get the lists of matching
to-be-joined partitions for subsequent pairs of input partitioned
relations for the join relation in a more efficient way than by
performing partition_bounds_merge() as proposed in the patch.

I don't know whether partition_bounds_merge() is well-implemented; I
haven't looked. But in general I don't see an alternative to doing
some kind of merging on each pair of input relations. That's just how
planning works, and I don't see why it should need to be prohibitively
expensive. I might be missing something, though; do you have an idea?

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

#85Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: amul sul (#83)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Fri, Jul 19, 2019 at 8:09 PM amul sul <sulamul@gmail.com> wrote:

On Mon, Jul 8, 2019 at 5:03 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

I started reviewing this. Here is my initial review comments:

* 0001-Hash-partition-bound-equality-refactoring-v22.patch

However, I don't think it's a good idea to do this
refactoring, because that would lead to duplicating the code to check
whether two given hash bound collections are equal in
partition_bounds_equal() and partition_hash_bounds_merge() that will
be added by the main patch, after all. To avoid that, how about
calling partition_bounds_equal() from partition_hash_bounds_merge() in
the main patch, like the attached?

Agree, your changes look good to me, thanks for working on it.

Cool! Thanks for reviewing!

Sorry for the delay. I was busy with something else recently.

Best regards,
Etsuro Fujita

#86Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Robert Haas (#84)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Fri, Jul 19, 2019 at 10:44 PM Robert Haas <robertmhaas@gmail.com> wrote:

On Thu, Jul 18, 2019 at 2:55 AM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

I.e., partition_bounds_merge() is performed for each pair of input
partitioned relations for a join relation in try_partitionwise_join().
Since partition_bounds_merge() would need a lot of CPU cycles, I don't
think this is acceptable; ISTM that some redesign is needed to avoid
this. I'm wondering that once we successfully merged partition bounds
from a pair of input partitioned relations for the join relation, by
using the merged partition bounds, we could get the lists of matching
to-be-joined partitions for subsequent pairs of input partitioned
relations for the join relation in a more efficient way than by
performing partition_bounds_merge() as proposed in the patch.

I don't know whether partition_bounds_merge() is well-implemented; I
haven't looked.

My concern about that is list partitioning. In that case that
function calls partition_list_bounds_merge(), which generates the
partition bounds for a join relation between two given input
relations, by performing merge join for a pair of the datums arrays
from both the input relations. Since the datums arrays contain all
non-null list values across all partitions, if the numbers of the list
values (ie, ndatums') are large, the merge join would require not a
few cycles, so it would be much expensive to perform the merge join
for each such pair when considering large N-way partitionwise joins of
list-partitioned tables with large ndatums. To see that, I did simple
tests using a list-partitioned table pt created with the attached,
which has 10 partitions, each with 1000 list values, so ndatums is
10000. (The tests below are performed with
enable_partitionwise_join=on.)

* 2-way self-join of pt: explain analyze select * from pt t0, pt t1
where t0.a = t1.a;
- HEAD:
Planning Time: 1.731 ms
Execution Time: 15.159 ms
- Patched:
Planning Time: 1.884 ms
Execution Time: 15.127 ms

* 4-way self-join of pt: explain analyze select * from pt t0, pt t1,
pt t2, pt t3 where t0.a = t1.a and t1.a = t2.a and t2.a = t3.a;
- HEAD:
Planning Time: 28.787 ms
Execution Time: 34.313 ms
- Patched:
Planning Time: 40.263 ms
Execution Time: 35.019 ms

* 8-way self-join of pt: explain analyze select * from pt t0, pt t1,
pt t2, pt t3, pt t4, pt t5, pt t6, pt t7 where t0.a = t1.a and t1.a =
t2.a and t2.a = t3.a and t3.a = t4.a and t4.a = t5.a and t5.a = t6.a
and t6.a = t7.a;
- HEAD:
Planning Time: 2279.653 ms
Execution Time: 63.303 ms
- Patched:
Planning Time: 3834.751 ms
Execution Time: 62.949 ms

Actually, these joins would not need the partition-matching algorithm
the patch adds; we could probably avoid this regression by modifying
the patch to plan these joins the same way as before, but ISTM that
these results imply that the cost of performing the merge join for
each such pair would not be negligible when considering large N-way
partitionwise joins mentioned above. Maybe I'm missing something,
though.

But in general I don't see an alternative to doing
some kind of merging on each pair of input relations. That's just how
planning works, and I don't see why it should need to be prohibitively
expensive. I might be missing something, though; do you have an idea?

Yes, I do; but I think I should think a little more about that.

Sorry for the delay.

Best regards,
Etsuro Fujita

Attachments:

list_parted2.sqlapplication/octet-stream; name=list_parted2.sqlDownload
#87Amit Langote
amitlangote09@gmail.com
In reply to: Etsuro Fujita (#86)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Tue, Jul 30, 2019 at 6:00 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

On Fri, Jul 19, 2019 at 10:44 PM Robert Haas <robertmhaas@gmail.com> wrote:

On Thu, Jul 18, 2019 at 2:55 AM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

I.e., partition_bounds_merge() is performed for each pair of input
partitioned relations for a join relation in try_partitionwise_join().
Since partition_bounds_merge() would need a lot of CPU cycles, I don't
think this is acceptable; ISTM that some redesign is needed to avoid
this. I'm wondering that once we successfully merged partition bounds
from a pair of input partitioned relations for the join relation, by
using the merged partition bounds, we could get the lists of matching
to-be-joined partitions for subsequent pairs of input partitioned
relations for the join relation in a more efficient way than by
performing partition_bounds_merge() as proposed in the patch.

I don't know whether partition_bounds_merge() is well-implemented; I
haven't looked.

My concern about that is list partitioning. In that case that
function calls partition_list_bounds_merge(), which generates the
partition bounds for a join relation between two given input
relations, by performing merge join for a pair of the datums arrays
from both the input relations.

I had similar thoughts upon seeing that partition_bounds_merge() will
be replacing the current way of determining if partition-wise join can
occur; that it will make the handling of currently supported cases
more expensive.

The current way is to compare the PartitionBoundInfos of joining
relations using partition_bounds_equal(), and if equal, simply join
the pairs of matching partitions if the join quals permit doing so.
There's no need to do anything extra to determine which partitions to
join with each other, because it's already established. Likewise,
partition_bounds_merge() shouldn't to have to anything extra in that
case. That is, for the cases that are already supported, we should
find a way to make partition_bounds_merge() only as expensive as just
performing partition_bounds_equals(), or maybe just slightly more.

Thanks,
Amit

#88Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Amit Langote (#87)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Amit-san,

On Wed, Jul 31, 2019 at 2:47 PM Amit Langote <amitlangote09@gmail.com> wrote:

On Tue, Jul 30, 2019 at 6:00 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

On Fri, Jul 19, 2019 at 10:44 PM Robert Haas <robertmhaas@gmail.com> wrote:

I don't know whether partition_bounds_merge() is well-implemented; I
haven't looked.

My concern about that is list partitioning. In that case that
function calls partition_list_bounds_merge(), which generates the
partition bounds for a join relation between two given input
relations, by performing merge join for a pair of the datums arrays
from both the input relations.

I had similar thoughts upon seeing that partition_bounds_merge() will
be replacing the current way of determining if partition-wise join can
occur; that it will make the handling of currently supported cases
more expensive.

The current way is to compare the PartitionBoundInfos of joining
relations using partition_bounds_equal(), and if equal, simply join
the pairs of matching partitions if the join quals permit doing so.
There's no need to do anything extra to determine which partitions to
join with each other, because it's already established. Likewise,
partition_bounds_merge() shouldn't to have to anything extra in that
case. That is, for the cases that are already supported, we should
find a way to make partition_bounds_merge() only as expensive as just
performing partition_bounds_equals(), or maybe just slightly more.

I 100% agree on that point.

One thing that was unexpected to me is this:

I wrote:

To see that, I did simple
tests using a list-partitioned table pt created with the attached,
which has 10 partitions, each with 1000 list values, so ndatums is
10000. (The tests below are performed with
enable_partitionwise_join=on.)

* 2-way self-join of pt: explain analyze select * from pt t0, pt t1
where t0.a = t1.a;
- HEAD:
Planning Time: 1.731 ms
Execution Time: 15.159 ms
- Patched:
Planning Time: 1.884 ms
Execution Time: 15.127 ms

IIUC, in this test, I think partition_bounds_equals() and
partition_bounds_merge() have been performed only once in HEAD and the
patched version respectively to plan the partitionwise join, so this
might imply that the cost of the latter is just slightly more
expensive than that of the former. I'm missing something, though.

Anyway I'll continue to review this patch, so I'll move this to the
next CF with the same status (Needs review).

Best regards,
Etsuro Fujita

#89Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Etsuro Fujita (#86)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Tue, Jul 30, 2019 at 6:00 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

On Fri, Jul 19, 2019 at 10:44 PM Robert Haas <robertmhaas@gmail.com> wrote:

On Thu, Jul 18, 2019 at 2:55 AM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

I.e., partition_bounds_merge() is performed for each pair of input
partitioned relations for a join relation in try_partitionwise_join().
Since partition_bounds_merge() would need a lot of CPU cycles, I don't
think this is acceptable; ISTM that some redesign is needed to avoid
this. I'm wondering that once we successfully merged partition bounds
from a pair of input partitioned relations for the join relation, by
using the merged partition bounds, we could get the lists of matching
to-be-joined partitions for subsequent pairs of input partitioned
relations for the join relation in a more efficient way than by
performing partition_bounds_merge() as proposed in the patch.

I don't know whether partition_bounds_merge() is well-implemented; I
haven't looked.

My concern about that is list partitioning. In that case that
function calls partition_list_bounds_merge(), which generates the
partition bounds for a join relation between two given input
relations, by performing merge join for a pair of the datums arrays
from both the input relations. Since the datums arrays contain all
non-null list values across all partitions, if the numbers of the list
values (ie, ndatums') are large, the merge join would require not a
few cycles, so it would be much expensive to perform the merge join
for each such pair when considering large N-way partitionwise joins of
list-partitioned tables with large ndatums. To see that, I did simple
tests using a list-partitioned table pt created with the attached,
which has 10 partitions, each with 1000 list values, so ndatums is
10000. (The tests below are performed with
enable_partitionwise_join=on.)

* 2-way self-join of pt: explain analyze select * from pt t0, pt t1
where t0.a = t1.a;
- HEAD:
Planning Time: 1.731 ms
Execution Time: 15.159 ms
- Patched:
Planning Time: 1.884 ms
Execution Time: 15.127 ms

* 4-way self-join of pt: explain analyze select * from pt t0, pt t1,
pt t2, pt t3 where t0.a = t1.a and t1.a = t2.a and t2.a = t3.a;
- HEAD:
Planning Time: 28.787 ms
Execution Time: 34.313 ms
- Patched:
Planning Time: 40.263 ms
Execution Time: 35.019 ms

* 8-way self-join of pt: explain analyze select * from pt t0, pt t1,
pt t2, pt t3, pt t4, pt t5, pt t6, pt t7 where t0.a = t1.a and t1.a =
t2.a and t2.a = t3.a and t3.a = t4.a and t4.a = t5.a and t5.a = t6.a
and t6.a = t7.a;
- HEAD:
Planning Time: 2279.653 ms
Execution Time: 63.303 ms
- Patched:
Planning Time: 3834.751 ms
Execution Time: 62.949 ms

Actually, these joins would not need the partition-matching algorithm
the patch adds; we could probably avoid this regression by modifying
the patch to plan these joins the same way as before, but ISTM that
these results imply that the cost of performing the merge join for
each such pair would not be negligible when considering large N-way
partitionwise joins mentioned above. Maybe I'm missing something,
though.

But in general I don't see an alternative to doing
some kind of merging on each pair of input relations. That's just how
planning works, and I don't see why it should need to be prohibitively
expensive. I might be missing something, though; do you have an idea?

Yes, I do; but I think I should think a little more about that.

I gave more thought to this. My idea is to generate the list of
matching partitions to be joined from the partbound info after
creating it with partition_bounds_merge(), as I stated before. Let me
explain using an example. Suppose that R, S and T are partitioned
tables, that R=R(a,b,c) is partitioned on ranges of a into three
partitions R1, R2 and R3, that S=S(a,b,c) is partitioned on ranges of
a into three partitions S1, S2 and S3, and that T=T(a,b,c) is
partitioned on ranges of a into three partitions T1, T2 and T3.
Consider a 3-way join query: SELECT * FROM R, S, T WHERE R.a=S.a AND
S.a=T.a; Suppose that when considering 2-way join R IJ S,
partition_bounds_merge() successfully merges the partition bounds for
R and S, and generates join pairs (R1, S1), (R2, S2) and (R3, S3), and
that when considering 3-way join (R IJ S) IJ T, that function
successfully merges the partition bounds for (R IJ S) and T, and
generates join pairs ((R1 IJ S1), T1) and ((R2 IJ S2), T2). The
question here is: do we really need to perform
partition_bounds_merge() to generate the list of matching partitions
to be joined for 3-way join R IJ (S IJ T), for example? I don't think
so; because 1) we see from the 3-way join pairs ((R1 IJ S1), T1) and
((R2 IJ S2), T2) that Ri, Si and Ti (i=1,2) have boundaries
overlapping with each other, which means that there would be (S1, T1)
and (S2, T2) as 2-way join pairs for S IJ T, and 2) we have R IJ (S IJ
T) = (R IJ S) IJ T = (R1 IJ S1) IJ T1 union (R2 IJ S2) IJ T2 = R1 IJ
(S1 IJ T1) union R2 IJ (S2 IJ T2), which means that the list of
matching partitions to be joined for R IJ (S IJ T) in question are
(R1, (S1 IJ T1)) and (R2, (S2 IJ T2)) since we see from the equation
that pairs from R and (S IJ T) other than that would not contribute to
the join result. Does that make sense? Attached is a WIP patch
implementing this created on top of the patch [1]/messages/by-id/CAPmGK177W+jNgpM5f_m-vdDKbEBu_=3CyPzFjkT_1nzf1Vqn+A@mail.gmail.com.

It seems that I performed the above tests on an assertion-enabled
build. :( So I executed the tests one more time. Here are the
results.

* 2-way self-join of pt: explain analyze select * from pt t0, pt t1
where t0.a = t1.a;
- HEAD:
Planning Time: 0.969 ms
Execution Time: 13.843 ms
- with patch:
Planning Time: 1.720 ms
Execution Time: 14.393 ms
- with patch plus attached:
Planning Time: 1.630 ms
Execution Time: 14.002 ms

* 4-way self-join of pt: explain analyze select * from pt t0, pt t1,
pt t2, pt t3 where t0.a = t1.a

and t1.a = t2.a and t2.a = t3.a;
- HEAD:
Planning Time: 12.203 ms
Execution Time: 31.784 ms
- with patch:
Planning Time: 32.102 ms
Execution Time: 32.504 ms
- with patch plus attached:
Planning Time: 19.471 ms
Execution Time: 32.582 ms

* 8-way self-join of pt: explain analyze select * from pt t0, pt t1,
pt t2, pt t3, pt t4, pt t5, pt t6, pt t7 where t0.a = t1.a and t1.a =
t2.a and t2.a = t3.a and t3.a = t4.a and t4.a = t5.a and t5.a = t6.a
and t6.a = t7.a;
- HEAD:
Planning Time: 948.131 ms
Execution Time: 55.645 ms
- with patch:
Planning Time: 2939.813 ms
Execution Time: 56.760 ms
- with patch plus attached:
Planning Time: 1108.076 ms
Execution Time: 55.750 ms

Note: the attached patch still uses the proposed partition matching
algorithm for these queries. As I said before, these queries don't
need that algorithm, so we could eliminate the planning overhead
compared to HEAD, by planning these queries as before, perhaps, but I
haven't modified the patch as such yet.

Best regards,
Etsuro Fujita

[1]: /messages/by-id/CAPmGK177W+jNgpM5f_m-vdDKbEBu_=3CyPzFjkT_1nzf1Vqn+A@mail.gmail.com

Attachments:

Modify-partition-matching-algorithm-WIP.patchapplication/octet-stream; name=Modify-partition-matching-algorithm-WIP.patchDownload
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index e6ce8e2110..a61b1741ec 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -2281,6 +2281,7 @@ _outRelOptInfo(StringInfo str, const RelOptInfo *node)
 	WRITE_BOOL_FIELD(has_eclass_joins);
 	WRITE_BOOL_FIELD(consider_partitionwise_join);
 	WRITE_BITMAPSET_FIELD(top_parent_relids);
+	WRITE_BITMAPSET_FIELD(all_partrels);
 	WRITE_NODE_FIELD(partitioned_child_rels);
 }
 
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 52abd8a12f..6607fae57b 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -48,6 +48,9 @@ static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 												Relids left_relids, Relids right_relids);
 static int	match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel,
 										 bool strict_op);
+static void	get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2);
 
 
 /*
@@ -1357,8 +1360,6 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	PartitionScheme part_scheme;
-	PartitionBoundInfo join_boundinfo;
 	List	   *parts1;
 	List	   *parts2;
 	ListCell   *lc1;
@@ -1369,7 +1370,7 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (joinrel->part_scheme == NULL)
+	if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
@@ -1394,51 +1395,51 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	 */
 	Assert(joinrel->part_scheme == rel1->part_scheme &&
 		   joinrel->part_scheme == rel2->part_scheme);
-	part_scheme = joinrel->part_scheme;
 
 	/*
-	 * Get the list of matching partitions to be joined along with the
-	 * partition bounds of the join relation. Because of the restrictions
-	 * imposed by partition matching algorithm, not every pair of joining
-	 * relations for this join will be able to use partition-wise join. But all
-	 * those pairs which can use partition-wise join will produce the same
-	 * partition bounds for the join relation.
+	 * If we haven't created the partition bounds for the join relation yet,
+	 * create it along with the the list of matching partitions to be joined;
+	 * else generate the list of such partitions from the partitioning info
+	 * for the join relation we already have.
 	 */
-	join_boundinfo = partition_bounds_merge(part_scheme->partnatts,
-											part_scheme->parttyplen,
-											part_scheme->parttypbyval,
-											part_scheme->partsupfunc,
-											part_scheme->partcollation,
-											rel1, rel2,
-											parent_sjinfo->jointype,
-											&parts1, &parts2);
-
-	if (join_boundinfo == NULL)
-		return;
-
-	if (joinrel->boundinfo == NULL)
+	if (joinrel->nparts == -1)
 	{
-		Assert(joinrel->nparts == 0 && joinrel->part_rels == NULL);
-		joinrel->boundinfo = join_boundinfo;
-		joinrel->nparts = list_length(parts1);
-		Assert(joinrel->nparts == list_length(parts2));
+		PartitionBoundInfo boundinfo;
+		int			nparts;
+
+		Assert(joinrel->boundinfo == NULL);
+		boundinfo = partition_bounds_merge(joinrel->part_scheme->partnatts,
+										   joinrel->part_scheme->parttyplen,
+										   joinrel->part_scheme->parttypbyval,
+										   joinrel->part_scheme->partsupfunc,
+										   joinrel->part_scheme->partcollation,
+										   rel1, rel2,
+										   parent_sjinfo->jointype,
+										   &parts1, &parts2);
+		if (boundinfo == NULL)
+		{
+			joinrel->nparts = 0;
+			return;
+		}
+		joinrel->boundinfo = boundinfo;
+
+		Assert(list_length(parts1) == list_length(parts2));
+		nparts = list_length(parts1);
+		Assert(nparts > 0);
+		joinrel->nparts = nparts;
+		Assert(joinrel->part_rels == NULL);
 		joinrel->part_rels =
-			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) *
-									joinrel->nparts);
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
 	}
 	else
 	{
-		Assert(partition_bounds_equal(part_scheme->partnatts,
-									  part_scheme->parttyplen,
-									  part_scheme->parttypbyval,
-									  join_boundinfo, joinrel->boundinfo));
-		/*
-		 * Every pair of joining relations should result in the same number
-		 * of child-joins.
-		 */
-		Assert(joinrel->nparts == list_length(parts1));
-		Assert(joinrel->nparts == list_length(parts2));
+		Assert(joinrel->nparts > 0);
+		Assert(joinrel->boundinfo);
 		Assert(joinrel->part_rels);
+
+		get_matching_part_pairs(root, joinrel, rel1, rel2, &parts1, &parts2);
+		Assert(list_length(parts1) == joinrel->nparts);
+		Assert(list_length(parts2) == joinrel->nparts);
 	}
 
 	/*
@@ -1449,12 +1450,12 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	cnt_parts = -1;
 	forboth(lc1, parts1, lc2, parts2)
 	{
-		int			part1 = lfirst_int(lc1);
-		int			part2 = lfirst_int(lc2);
-		RelOptInfo *child_rel1;
-		RelOptInfo *child_rel2;
-		bool		rel1_empty;
-		bool		rel2_empty;
+		RelOptInfo *child_rel1 = (RelOptInfo *) lfirst(lc1);
+		RelOptInfo *child_rel2 = (RelOptInfo *) lfirst(lc2);
+		bool		rel1_empty = (child_rel1 == NULL ||
+								  IS_DUMMY_REL(child_rel1));
+		bool		rel2_empty = (child_rel2 == NULL ||
+								  IS_DUMMY_REL(child_rel2));
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1462,11 +1463,6 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
-		Assert(part1 >= 0 && part2 >= 0);
-		child_rel1 = rel1->part_rels[part1];
-		child_rel2 = rel2->part_rels[part2];
-		rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1));
-		rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2));
 		cnt_parts++;
 
 		/*
@@ -1566,25 +1562,19 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 												 child_sjinfo,
 												 child_sjinfo->jointype);
 			joinrel->part_rels[cnt_parts] = child_joinrel;
+			joinrel->all_partrels = bms_add_members(joinrel->all_partrels,
+													child_joinrel->relids);
 		}
 
-		/*
-		 * For every pair of joining relations, the set of matching partitions
-		 * would change. However, the base relation partitions constituting
-		 * the given child should remain same for all the joining pairs. Since
-		 * the order in which children are stored in the array of child-joins,
-		 * depends upon partition bounds of the join, which are same for all
-		 * the joining pairs, every joining pair yields the child-joins in the
-		 * same order.
-		 */
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
 
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
+
 	}
 
-	Assert(cnt_parts == (joinrel->nparts - 1));
+	Assert(cnt_parts == joinrel->nparts - 1);
 }
 
 /*
@@ -1793,3 +1783,70 @@ match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op)
 
 	return -1;
 }
+
+static void
+get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2)
+{
+	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
+	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
+	int 		cnt_parts;
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
+	{
+		RelOptInfo *child_joinrel = joinrel->part_rels[cnt_parts];
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		Relids		child_relids1;
+		Relids		child_relids2;
+
+		/*
+		 * If the segment of the join is empty, it means that the segment was
+		 * ignored in the child-join creation step in try_partitionwise_join()
+		 * since it would not contribute to the join result, due to one or
+		 * both inputs being empty; add NULL for each list so that it will be
+		 * ignored again in that step.
+		 */
+		if (!child_joinrel)
+		{
+			*parts1 = lappend(*parts1, NULL);
+			*parts2 = lappend(*parts2, NULL);
+			continue;
+		}
+
+		child_relids1 = bms_intersect(child_joinrel->relids,
+									  rel1->all_partrels);
+		Assert(bms_num_members(child_relids1) == bms_num_members(rel1->relids));
+		if (rel1_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids1);
+
+			child_rel1 = find_base_rel(root, varno);
+		}
+		else
+			child_rel1 = find_join_rel(root, child_relids1);
+		/* We should have a rel for child_relids1 */
+		Assert(child_rel1);
+
+		child_relids2 = bms_intersect(child_joinrel->relids,
+									  rel2->all_partrels);
+		Assert(bms_num_members(child_relids2) == bms_num_members(rel2->relids));
+		if (rel2_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids2);
+
+			child_rel2 = find_base_rel(root, varno);
+		}
+		else
+			child_rel2 = find_join_rel(root, child_relids2);
+		/* We should have a rel for child_relids2 */
+		Assert(child_rel2);
+
+		*parts1 = lappend(*parts1, child_rel1);
+		*parts2 = lappend(*parts2, child_rel2);
+	}
+}
diff --git a/src/backend/optimizer/util/inherit.c b/src/backend/optimizer/util/inherit.c
index 38bc61e687..caf6039c10 100644
--- a/src/backend/optimizer/util/inherit.c
+++ b/src/backend/optimizer/util/inherit.c
@@ -376,6 +376,8 @@ expand_partitioned_rtentry(PlannerInfo *root, RelOptInfo *relinfo,
 		/* Create the otherrel RelOptInfo too. */
 		childrelinfo = build_simple_rel(root, childRTindex, relinfo);
 		relinfo->part_rels[i] = childrelinfo;
+		relinfo->all_partrels = bms_add_members(relinfo->all_partrels,
+												childrelinfo->relids);
 
 		/* If this child is itself partitioned, recurse */
 		if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 1e883bc994..f9ac8ab82a 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -237,10 +237,11 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->has_eclass_joins = false;
 	rel->consider_partitionwise_join = false;	/* might get changed later */
 	rel->part_scheme = NULL;
-	rel->nparts = 0;
+	rel->nparts = -1;
 	rel->boundinfo = NULL;
 	rel->partition_qual = NIL;
 	rel->part_rels = NULL;
+	rel->all_partrels = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->partitioned_child_rels = NIL;
@@ -650,10 +651,11 @@ build_join_rel(PlannerInfo *root,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -826,10 +828,11 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 88afc05a60..a5b3072c0f 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -128,11 +128,14 @@ static PartitionBoundInfo partition_hash_bounds_merge(int partnatts,
 							int16 *parttyplen, bool *parttypbyval,
 							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 							List **outer_parts, List **inner_parts);
-static void generate_matching_part_pairs(PartitionMap *outer_maps,
-										 PartitionMap *inner_maps,
-										 int nparts1, int nparts2,
-										 JoinType jointype, int nparts,
-										 List **parts1, List **parts2);
+static void	generate_matching_part_pairs(RelOptInfo *rel1,
+							 RelOptInfo *rel2,
+							 PartitionMap *partmaps1,
+							 PartitionMap *partmaps2,
+							 int nparts1, int nparts2,
+							 int nparts,
+							 List **matched_parts1,
+							 List **matched_parts2);
 static PartitionBoundInfo build_merged_partition_bounds(char strategy,
 							  List *merged_datums, List *merged_indexes,
 							  List *merged_contents, int null_index,
@@ -3839,9 +3842,10 @@ partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 		return NULL;
 
 	/* Use maps to match partition from the joining relations. */
-	generate_matching_part_pairs(outer_maps, inner_maps,
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 outer_maps, inner_maps,
 								 outer_nparts, inner_nparts,
-								 jointype, next_index,
+								 next_index,
 								 outer_parts, inner_parts);
 
 	/* Craft a PartitionBoundInfo to return. */
@@ -4099,9 +4103,10 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 		return NULL;
 
 	/* Use maps to match partition from the joining relations. */
-	generate_matching_part_pairs(outer_maps, inner_maps,
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 outer_maps, inner_maps,
 								 outer_nparts, inner_nparts,
-								 jointype, next_index,
+								 next_index,
 								 outer_parts, inner_parts);
 
 	/* Craft a PartitionBoundInfo to return. */
@@ -4170,8 +4175,8 @@ partition_hash_bounds_merge(int partnatts,
 	  */
 	for (cnt = 0; cnt < outer_rel->nparts; cnt++)
 	{
-		*outer_parts = lappend_int(*outer_parts, cnt);
-		*inner_parts = lappend_int(*inner_parts, cnt);
+		*outer_parts = lappend(*outer_parts, outer_rel->part_rels[cnt]);
+		*inner_parts = lappend(*inner_parts, inner_rel->part_rels[cnt]);
 	}
 
 	return outer_bi;
@@ -4312,12 +4317,11 @@ map_and_merge_partitions(PartitionMap *partmaps1, PartitionMap *partmaps2,
  * set to NIL.
  */
 static void
-generate_matching_part_pairs(PartitionMap *partmaps1, PartitionMap *partmaps2,
-							 int nparts1, int nparts2,
-							 JoinType jointype, int nparts,
+generate_matching_part_pairs(RelOptInfo *rel1, RelOptInfo *rel2,
+							 PartitionMap *partmaps1, PartitionMap *partmaps2,
+							 int nparts1, int nparts2, int nparts,
 							 List **matched_parts1, List **matched_parts2)
 {
-	bool		merged = true;
 	int		   *matching1,
 			   *matching2;
 	int 		i;
@@ -4358,16 +4362,6 @@ generate_matching_part_pairs(PartitionMap *partmaps1, PartitionMap *partmaps2,
 		}
 	}
 
-	/*
-	 * If we have a partition missing on an inner side, we need to add a dummy
-	 * relation which joins with the outer partition. If the inner relation
-	 * happens to be a base relation, it will require adding a dummy child
-	 * base relation during join processing. Right now, we freeze the base
-	 * relation arrays like PlannerInfo::simple_rte_array after planning for
-	 * base relations. Adding a new (dummy) base relation would require some
-	 * changes to that. So, right now, we do not implement partition-wise join
-	 * in such cases.
-	 */
 	for (i = 0; i < nparts; i++)
 	{
 		int			part1 = matching1[i];
@@ -4376,52 +4370,14 @@ generate_matching_part_pairs(PartitionMap *partmaps1, PartitionMap *partmaps2,
 		/* At least one of the partitions should exist. */
 		Assert(part1 >= 0 || part2 >= 0);
 
-		switch (jointype)
-		{
-			case JOIN_INNER:
-			case JOIN_SEMI:
-
-				/*
-				 * An inner or semi join can not return any row when the
-				 * matching partition on either side is missing. We should
-				 * have eliminated all such cases while merging the bounds.
-				 */
-				Assert(part1 >= 0 && part2 >= 0);
-				break;
-
-			case JOIN_LEFT:
-			case JOIN_ANTI:
-				Assert(part1 >= 0);
-				if (part2 < 0)
-					merged = false;
-				break;
-
-			case JOIN_FULL:
-				if (part1 < 0 || part2 < 0)
-					merged = false;
-				break;
-
-			default:
-				elog(ERROR, "unrecognized join type: %d", (int) jointype);
-		}
-
-		if (!merged)
-			break;
-
-		*matched_parts1 = lappend_int(*matched_parts1, part1);
-		*matched_parts2 = lappend_int(*matched_parts2, part2);
+		*matched_parts1 = lappend(*matched_parts1,
+								  part1 >= 0 ? rel1->part_rels[part1] : NULL);
+		*matched_parts2 = lappend(*matched_parts2,
+								  part2 >= 0 ? rel2->part_rels[part2] : NULL);
 	}
 
 	pfree(matching1);
 	pfree(matching2);
-
-	if (!merged)
-	{
-		list_free(*matched_parts1);
-		list_free(*matched_parts2);
-		*matched_parts1 = NIL;
-		*matched_parts2 = NIL;
-	}
 }
 
 static PartitionBoundInfo
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index e3c579ee44..1d2dc68201 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -581,6 +581,7 @@ typedef struct PartitionSchemeData *PartitionScheme;
  *		boundinfo - Partition bounds
  *		partition_qual - Partition constraint if not the root
  *		part_rels - RelOptInfos for each partition
+ *		all_partrels - Relids set of all partition relids
  *		partexprs, nullable_partexprs - Partition key expressions
  *		partitioned_child_rels - RT indexes of unpruned partitions of
  *								 this relation that are partitioned tables
@@ -723,6 +724,7 @@ typedef struct RelOptInfo
 	List	   *partition_qual; /* partition constraint */
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of partitions,
 									 * stored in the same order of bounds */
+	Relids		all_partrels;	/* Relids set of all partition relids */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 	List	   *partitioned_child_rels; /* List of RT indexes. */
#90Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Etsuro Fujita (#89)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Fri, Aug 16, 2019 at 10:25 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

It seems that I performed the above tests on an assertion-enabled
build. :( So I executed the tests one more time. Here are the
results.

* 2-way self-join of pt: explain analyze select * from pt t0, pt t1
where t0.a = t1.a;
- HEAD:
Planning Time: 0.969 ms
Execution Time: 13.843 ms
- with patch:
Planning Time: 1.720 ms
Execution Time: 14.393 ms
- with patch plus attached:
Planning Time: 1.630 ms
Execution Time: 14.002 ms

* 4-way self-join of pt: explain analyze select * from pt t0, pt t1,
pt t2, pt t3 where t0.a = t1.a

and t1.a = t2.a and t2.a = t3.a;
- HEAD:
Planning Time: 12.203 ms
Execution Time: 31.784 ms
- with patch:
Planning Time: 32.102 ms
Execution Time: 32.504 ms
- with patch plus attached:
Planning Time: 19.471 ms
Execution Time: 32.582 ms

* 8-way self-join of pt: explain analyze select * from pt t0, pt t1,
pt t2, pt t3, pt t4, pt t5, pt t6, pt t7 where t0.a = t1.a and t1.a =
t2.a and t2.a = t3.a and t3.a = t4.a and t4.a = t5.a and t5.a = t6.a
and t6.a = t7.a;
- HEAD:
Planning Time: 948.131 ms
Execution Time: 55.645 ms
- with patch:
Planning Time: 2939.813 ms
Execution Time: 56.760 ms
- with patch plus attached:
Planning Time: 1108.076 ms
Execution Time: 55.750 ms

Note: the attached patch still uses the proposed partition matching
algorithm for these queries. As I said before, these queries don't
need that algorithm, so we could eliminate the planning overhead
compared to HEAD, by planning these queries as before, perhaps, but I
haven't modified the patch as such yet.

I modified the patch further as such. Attached is an updated version
of the patch created on top of the patch in [1]/messages/by-id/CAPmGK177W+jNgpM5f_m-vdDKbEBu_=3CyPzFjkT_1nzf1Vqn+A@mail.gmail.com. I did the tests
again using the updated version of the patch. Here are the results:

* 2-way self-join of pt:
Planning Time: 1.043 ms
Execution Time: 13.931 ms

* 4-way self-join of pt:
Planning Time: 12.499 ms
Execution Time: 32.392 ms

* 8-way self-join of pt:
Planning Time: 968.412 ms
Execution Time: 56.328 ms

The planning time for each test case still increased slightly, but IMO
I think that would be acceptable. To see the efficiency of the
attached, I did another testing with test cases that really need the
new partition-matching algorithm:

* explain analyze select * from pt6 t6, pt7 t7 where t6.a = t7.a;
- base patch in [1]/messages/by-id/CAPmGK177W+jNgpM5f_m-vdDKbEBu_=3CyPzFjkT_1nzf1Vqn+A@mail.gmail.com
Planning Time: 1.758 ms
Execution Time: 13.977 ms
- with attached
Planning Time: 1.777 ms
Execution Time: 13.959 ms

* explain analyze select * from pt4 t4, pt5 t5, pt6 t6, pt7 t7 where
t4.a = t5.a and t5.a = t6.a and t6.a = t7.a;
- base patch in [1]/messages/by-id/CAPmGK177W+jNgpM5f_m-vdDKbEBu_=3CyPzFjkT_1nzf1Vqn+A@mail.gmail.com
Planning Time: 33.201 ms
Execution Time: 32.480 ms
- with attached
Planning Time: 21.019 ms
Execution Time: 32.777 ms

* explain analyze select * from pt0 t0, pt1 t1, pt2 t2, pt3 t3, pt4
t4, pt5 t5, pt6 t6, pt7 t7 where t0.a = t1.a and t1.a = t2.a and t2.a
= t3.a and t3.a = t4.a and t4.a = t5.a and t5.a = t6.a and t6.a =
t7.a;
- base patch in [1]/messages/by-id/CAPmGK177W+jNgpM5f_m-vdDKbEBu_=3CyPzFjkT_1nzf1Vqn+A@mail.gmail.com
Planning Time: 3060.000 ms
Execution Time: 55.553 ms
- with attached
Planning Time: 1144.996 ms
Execution Time: 56.233 ms

where pt0, pt1, pt2, pt3, pt4, pt5, pt6 and pt7 are list partitioned
tables that have slighly different list values. (The structure and
list values of ptN are almost the same as that of pt used above, but
ptN's N-th partition ptNpN has an extra list value that pt's N-th
partition ptpN doesn't have.) If anyone is interested in this
testing, I'll send a script file for producing these list partitioned
tables.

About the attached:

* The attached patch modified try_partitionwise_join() so that we call
partition_bounds_equal() then partition_bounds_merge() (if necessary)
to create the partition bounds for the join rel. We don't support for
merging the partition bounds for the hash-partitioning case, so this
makes code to handle the hash-partitioning case in
partition_bounds_merge() completely unnecessary, so I removed that
entirely.

* I removed this assertion in partition_bounds_merge(), because I
think this is covered by two assertions above this.

+   Assert((*outer_parts == NIL || *inner_parts != NIL) &&
+          (*inner_parts == NIL || *outer_parts != NIL));

* (I forgot to mention this in a previous email, but) I removed this
bit of generate_matching_part_pairs(), because we already have the
same check in try_partitionwise_join(), so this bit would be redundant
IIUC.

+       switch (jointype)
+       {
+           case JOIN_INNER:
+           case JOIN_SEMI:
+
+               /*
+                * An inner or semi join can not return any row when the
+                * matching partition on either side is missing. We should
+                * have eliminated all such cases while merging the bounds.
+                */
+               Assert(part1 >= 0 && part2 >= 0);
+               break;
+
+           case JOIN_LEFT:
+           case JOIN_ANTI:
+               Assert(part1 >= 0);
+               if (part2 < 0)
+                   merged = false;
+               break;
+
+           case JOIN_FULL:
+               if (part1 < 0 || part2 < 0)
+                   merged = false;
+               break;
+
+           default:
+               elog(ERROR, "unrecognized join type: %d", (int) jointype);
+       }

* I added more comments.

If there are no objections, I'll merge the attached with the base patch in [1]/messages/by-id/CAPmGK177W+jNgpM5f_m-vdDKbEBu_=3CyPzFjkT_1nzf1Vqn+A@mail.gmail.com.

Best regards,
Etsuro Fujita

[1]: /messages/by-id/CAPmGK177W+jNgpM5f_m-vdDKbEBu_=3CyPzFjkT_1nzf1Vqn+A@mail.gmail.com

Attachments:

Modify-partition-matching-algorithm-1.patchapplication/octet-stream; name=Modify-partition-matching-algorithm-1.patchDownload
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index e6ce8e2110..b7849560b7 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -2281,6 +2281,8 @@ _outRelOptInfo(StringInfo str, const RelOptInfo *node)
 	WRITE_BOOL_FIELD(has_eclass_joins);
 	WRITE_BOOL_FIELD(consider_partitionwise_join);
 	WRITE_BITMAPSET_FIELD(top_parent_relids);
+	WRITE_BOOL_FIELD(merged);
+	WRITE_BITMAPSET_FIELD(all_partrels);
 	WRITE_NODE_FIELD(partitioned_child_rels);
 }
 
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 52abd8a12f..8442e9d186 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -48,6 +48,9 @@ static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 												Relids left_relids, Relids right_relids);
 static int	match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel,
 										 bool strict_op);
+static void get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2);
 
 
 /*
@@ -1357,19 +1360,18 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	PartitionScheme part_scheme;
-	PartitionBoundInfo join_boundinfo;
-	List	   *parts1;
-	List	   *parts2;
-	ListCell   *lc1;
-	ListCell   *lc2;
+	bool		merged = false;
+	List	   *parts1 = NIL;
+	List	   *parts2 = NIL;
+	ListCell   *lcr1 = NULL;
+	ListCell   *lcr2 = NULL;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (joinrel->part_scheme == NULL)
+	if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
@@ -1394,51 +1396,100 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	 */
 	Assert(joinrel->part_scheme == rel1->part_scheme &&
 		   joinrel->part_scheme == rel2->part_scheme);
-	part_scheme = joinrel->part_scheme;
 
 	/*
-	 * Get the list of matching partitions to be joined along with the
-	 * partition bounds of the join relation. Because of the restrictions
-	 * imposed by partition matching algorithm, not every pair of joining
-	 * relations for this join will be able to use partition-wise join. But all
-	 * those pairs which can use partition-wise join will produce the same
-	 * partition bounds for the join relation.
+	 * If we don't have the partition bounds for the join rel yet, try to
+	 * create it along with pairs of partitions to be joined; else generate
+	 * those using the partitioning info for the join rel we already have.
 	 */
-	join_boundinfo = partition_bounds_merge(part_scheme->partnatts,
-											part_scheme->parttyplen,
-											part_scheme->parttypbyval,
-											part_scheme->partsupfunc,
-											part_scheme->partcollation,
-											rel1, rel2,
-											parent_sjinfo->jointype,
-											&parts1, &parts2);
-
-	if (join_boundinfo == NULL)
-		return;
-
-	if (joinrel->boundinfo == NULL)
+	if (joinrel->nparts == -1)
 	{
-		Assert(joinrel->nparts == 0 && joinrel->part_rels == NULL);
-		joinrel->boundinfo = join_boundinfo;
-		joinrel->nparts = list_length(parts1);
-		Assert(joinrel->nparts == list_length(parts2));
+		PartitionScheme part_scheme = joinrel->part_scheme;
+		PartitionBoundInfo boundinfo = NULL;
+		int			nparts = 0;
+
+		Assert(joinrel->boundinfo == NULL);
+		Assert(joinrel->part_rels == NULL);
+
+		/*
+		 * See if the partition bounds for inputs are exactly the same, in
+		 * which case we don't need to work hard; partitions with the same
+		 * partition indexes will form join pairs, and the join rel will have
+		 * the same partition bounds as inputs.
+		 *
+		 * In theory, the partition bounds for inputs might be exactly the
+		 * same even in the case where one or both inputs have merged
+		 * partition bounds, but it seems unlikely to be worth the cycles to
+		 * check.
+		 */
+		if (!rel1->merged &&
+			!rel2->merged &&
+			rel1->nparts == rel2->nparts &&
+			partition_bounds_equal(part_scheme->partnatts,
+								   part_scheme->parttyplen,
+								   part_scheme->parttypbyval,
+								   rel1->boundinfo, rel2->boundinfo))
+		{
+			boundinfo = rel1->boundinfo;
+			nparts = rel1->nparts;
+		}
+
+		/*
+		 * Try to create the partition bounds along with join pairs.
+		 */
+		if (boundinfo == NULL)
+		{
+			boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											   part_scheme->parttyplen,
+											   part_scheme->parttypbyval,
+											   part_scheme->partsupfunc,
+											   part_scheme->partcollation,
+											   rel1, rel2,
+											   parent_sjinfo->jointype,
+											   &parts1, &parts2);
+			Assert(list_length(parts1) == list_length(parts2));
+			if (boundinfo == NULL)
+			{
+				joinrel->nparts = 0;
+				return;
+			}
+			nparts = list_length(parts1);
+			merged = true;
+		}
+
+		Assert(nparts > 0);
+		joinrel->boundinfo = boundinfo;
+		joinrel->merged = merged;
+		joinrel->nparts = nparts;
 		joinrel->part_rels =
-			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) *
-									joinrel->nparts);
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
 	}
 	else
 	{
-		Assert(partition_bounds_equal(part_scheme->partnatts,
-									  part_scheme->parttyplen,
-									  part_scheme->parttypbyval,
-									  join_boundinfo, joinrel->boundinfo));
+		Assert(joinrel->nparts > 0);
+		Assert(joinrel->boundinfo);
+		Assert(joinrel->part_rels);
+
 		/*
-		 * Every pair of joining relations should result in the same number
-		 * of child-joins.
+		 * If the partition bounds for the join rel are not merged ones,
+		 * inputs are guaranteed to have the same partition bounds, so
+		 * partitions with the same partition indexes will form join pairs;
+		 * else let get_matching_part_pairs() do the work.
 		 */
-		Assert(joinrel->nparts == list_length(parts1));
-		Assert(joinrel->nparts == list_length(parts2));
-		Assert(joinrel->part_rels);
+		if (joinrel->merged)
+		{
+			get_matching_part_pairs(root, joinrel, rel1, rel2,
+									&parts1, &parts2);
+			Assert(list_length(parts1) == joinrel->nparts);
+			Assert(list_length(parts2) == joinrel->nparts);
+			merged = true;
+		}
+	}
+
+	if (merged)
+	{
+		lcr1 = list_head(parts1);
+		lcr2 = list_head(parts2);
 	}
 
 	/*
@@ -1446,15 +1497,14 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	cnt_parts = -1;
-	forboth(lc1, parts1, lc2, parts2)
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
 	{
-		int			part1 = lfirst_int(lc1);
-		int			part2 = lfirst_int(lc2);
-		RelOptInfo *child_rel1;
-		RelOptInfo *child_rel2;
-		bool		rel1_empty;
-		bool		rel2_empty;
+		RelOptInfo *child_rel1 = merged ? (RelOptInfo *) lfirst(lcr1) : rel1->part_rels[cnt_parts];
+		RelOptInfo *child_rel2 = merged ? (RelOptInfo *) lfirst(lcr2) : rel2->part_rels[cnt_parts];
+		bool		rel1_empty = (child_rel1 == NULL ||
+								  IS_DUMMY_REL(child_rel1));
+		bool		rel2_empty = (child_rel2 == NULL ||
+								  IS_DUMMY_REL(child_rel2));
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1462,12 +1512,11 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
-		Assert(part1 >= 0 && part2 >= 0);
-		child_rel1 = rel1->part_rels[part1];
-		child_rel2 = rel2->part_rels[part2];
-		rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1));
-		rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2));
-		cnt_parts++;
+		if (merged)
+		{
+			lcr1 = lnext(parts1, lcr1);
+			lcr2 = lnext(parts2, lcr2);
+		}
 
 		/*
 		 * Check for cases where we can prove that this segment of the join
@@ -1566,25 +1615,17 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 												 child_sjinfo,
 												 child_sjinfo->jointype);
 			joinrel->part_rels[cnt_parts] = child_joinrel;
+			joinrel->all_partrels = bms_add_members(joinrel->all_partrels,
+													child_joinrel->relids);
 		}
 
-		/*
-		 * For every pair of joining relations, the set of matching partitions
-		 * would change. However, the base relation partitions constituting
-		 * the given child should remain same for all the joining pairs. Since
-		 * the order in which children are stored in the array of child-joins,
-		 * depends upon partition bounds of the join, which are same for all
-		 * the joining pairs, every joining pair yields the child-joins in the
-		 * same order.
-		 */
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
 
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
-	}
 
-	Assert(cnt_parts == (joinrel->nparts - 1));
+	}
 }
 
 /*
@@ -1793,3 +1834,97 @@ match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op)
 
 	return -1;
 }
+
+/*
+ * get_matching_part_pairs
+ *		Generate join pairs of partitions for the two inputs for the join rel
+ */
+static void
+get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2)
+{
+	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
+	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
+	int 		cnt_parts;
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
+	{
+		RelOptInfo *child_joinrel = joinrel->part_rels[cnt_parts];
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		Relids		child_relids1;
+		Relids		child_relids2;
+
+		/*
+		 * If this segment of the join is empty, it means that this segment
+		 * was ignored when previously creating child-join paths for it in
+		 * try_partitionwise_join() as it would not contribute to the join
+		 * result, due to one or both inputs being empty; add NULL to each of
+		 * the given lists so that this segment will be ignored again in that
+		 * function.
+		 */
+		if (!child_joinrel)
+		{
+			*parts1 = lappend(*parts1, NULL);
+			*parts2 = lappend(*parts2, NULL);
+			continue;
+		}
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel1 side.
+		 */
+		child_relids1 = bms_intersect(child_joinrel->relids,
+									  rel1->all_partrels);
+		Assert(bms_num_members(child_relids1) == bms_num_members(rel1->relids));
+		/*
+		 * Get a child rel for rel1 with the relids.  Note that we should have
+		 * the child rel even if rel1 is a join rel, because in that case the
+		 * partitions specified in the relids would have matching/overlapping
+		 * boundaries, so those partitions should be considered as ones to be
+		 * joined even when planning partitionwise joins of rel1, meaning that
+		 * the child rel would have been built by the time we get here.
+		 */
+		if (rel1_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids1);
+
+			child_rel1 = find_base_rel(root, varno);
+		}
+		else
+			child_rel1 = find_join_rel(root, child_relids1);
+		Assert(child_rel1);
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel2 side.
+		 */
+		child_relids2 = bms_intersect(child_joinrel->relids,
+									  rel2->all_partrels);
+		Assert(bms_num_members(child_relids2) == bms_num_members(rel2->relids));
+		/*
+		 * Get a child rel for rel2 with the relids.  See above comments.
+		 */
+		if (rel2_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids2);
+
+			child_rel2 = find_base_rel(root, varno);
+		}
+		else
+			child_rel2 = find_join_rel(root, child_relids2);
+		Assert(child_rel2);
+
+		/*
+		 * The join of rel1 and rel2 is legal, so is the join of the child
+		 * rels obtained above; add them to the given lists as a join pair
+		 * producing this join segment.
+		 */
+		*parts1 = lappend(*parts1, child_rel1);
+		*parts2 = lappend(*parts2, child_rel2);
+	}
+}
diff --git a/src/backend/optimizer/util/inherit.c b/src/backend/optimizer/util/inherit.c
index 38bc61e687..caf6039c10 100644
--- a/src/backend/optimizer/util/inherit.c
+++ b/src/backend/optimizer/util/inherit.c
@@ -376,6 +376,8 @@ expand_partitioned_rtentry(PlannerInfo *root, RelOptInfo *relinfo,
 		/* Create the otherrel RelOptInfo too. */
 		childrelinfo = build_simple_rel(root, childRTindex, relinfo);
 		relinfo->part_rels[i] = childrelinfo;
+		relinfo->all_partrels = bms_add_members(relinfo->all_partrels,
+												childrelinfo->relids);
 
 		/* If this child is itself partitioned, recurse */
 		if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index ce85082bb3..248a9e9093 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -240,10 +240,12 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->has_eclass_joins = false;
 	rel->consider_partitionwise_join = false;	/* might get changed later */
 	rel->part_scheme = NULL;
-	rel->nparts = 0;
+	rel->nparts = -1;
 	rel->boundinfo = NULL;
+	rel->merged = false;
 	rel->partition_qual = NIL;
 	rel->part_rels = NULL;
+	rel->all_partrels = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->partitioned_child_rels = NIL;
@@ -653,10 +655,12 @@ build_join_rel(PlannerInfo *root,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -829,10 +833,12 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index a8c85966af..f06fa0827c 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -124,15 +124,14 @@ static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid
 							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 							List **outer_parts, List **inner_parts,
 							JoinType jointype);
-static PartitionBoundInfo partition_hash_bounds_merge(int partnatts,
-							int16 *parttyplen, bool *parttypbyval,
-							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
-							List **outer_parts, List **inner_parts);
-static void generate_matching_part_pairs(PartitionMap *outer_maps,
-										 PartitionMap *inner_maps,
-										 int nparts1, int nparts2,
-										 JoinType jointype, int nparts,
-										 List **parts1, List **parts2);
+static void generate_matching_part_pairs(RelOptInfo *rel1,
+							 RelOptInfo *rel2,
+							 PartitionMap *partmaps1,
+							 PartitionMap *partmaps2,
+							 int nparts1, int nparts2,
+							 int nparts,
+							 List **matched_parts1,
+							 List **matched_parts2);
 static PartitionBoundInfo build_merged_partition_bounds(char strategy,
 							  List *merged_datums, List *merged_indexes,
 							  List *merged_contents, int null_index,
@@ -3104,6 +3103,11 @@ partition_bounds_merge(int partnatts,
 	*inner_parts = NIL;
 	switch (strategy)
 	{
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = NULL;
+
+			break;
+
 		case PARTITION_STRATEGY_LIST:
 			merged_bounds = partition_list_bounds_merge(partsupfunc,
 														partcollation,
@@ -3120,13 +3124,6 @@ partition_bounds_merge(int partnatts,
 														 partcollation);
 			break;
 
-		case PARTITION_STRATEGY_HASH:
-			merged_bounds = partition_hash_bounds_merge(partnatts, 
-														parttyplen, parttypbyval,
-														outer_rel, inner_rel,
-														outer_parts, inner_parts);
-			break;
-
 		default:
 			elog(ERROR, "unexpected partition strategy: %d", strategy);
 	}
@@ -3135,9 +3132,6 @@ partition_bounds_merge(int partnatts,
 
 	Assert(list_length(*outer_parts) == list_length(*inner_parts));
 
-	Assert((*outer_parts == NIL || *inner_parts != NIL) &&
-		   (*inner_parts == NIL || *outer_parts != NIL));
-
 	return merged_bounds;
 }
 
@@ -3839,9 +3833,10 @@ partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 		return NULL;
 
 	/* Use maps to match partition from the joining relations. */
-	generate_matching_part_pairs(outer_maps, inner_maps,
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 outer_maps, inner_maps,
 								 outer_nparts, inner_nparts,
-								 jointype, next_index,
+								 next_index,
 								 outer_parts, inner_parts);
 
 	/* Craft a PartitionBoundInfo to return. */
@@ -4099,9 +4094,10 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 		return NULL;
 
 	/* Use maps to match partition from the joining relations. */
-	generate_matching_part_pairs(outer_maps, inner_maps,
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 outer_maps, inner_maps,
 								 outer_nparts, inner_nparts,
-								 jointype, next_index,
+								 next_index,
 								 outer_parts, inner_parts);
 
 	/* Craft a PartitionBoundInfo to return. */
@@ -4122,61 +4118,6 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 	return merged_bounds;
 }
 
-/*
- * partition_bounds_merge()'s arm for hash partitioned tables.
- *
- * If the given two hash bounds are same, the function returns the first one
- * without any change, alongwith the lists of matching partitions. Otherwise it
- * returns NULL.
- *
- * We could try merging the bounds when both the bounds have same greatest
- * modulii. But there seems to be hardly any requirement for the same.
- */
-static PartitionBoundInfo
-partition_hash_bounds_merge(int partnatts,
-							int16 *parttyplen, bool *parttypbyval,
-							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
-							List **outer_parts, List **inner_parts)
-{
-	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
-					   inner_bi = inner_rel->boundinfo;
-	int			cnt;
-
-	Assert(*outer_parts == NIL);
-	Assert(*inner_parts == NIL);
-
-	Assert(outer_bi->strategy == inner_bi->strategy &&
-		   outer_bi->strategy == PARTITION_STRATEGY_HASH);
-
-	/*
-	 * Hash partitioned table does not have explicit NULL accepting partition
-	 * and also does not have a default partition.
-	 */
-	Assert(!partition_bound_has_default(outer_bi) &&
-		   !partition_bound_has_default(inner_bi));
-	Assert(!partition_bound_accepts_nulls(outer_bi) &&
-		   !partition_bound_accepts_nulls(inner_bi));
-
-	if (outer_rel->nparts != inner_rel->nparts)
-		return NULL;
-
-	if (!partition_bounds_equal(partnatts, parttyplen, parttypbyval,
-								outer_bi, inner_bi))
-		return NULL;
-
-	 /*
-	  * Cook up list of matching partitions. Since bounds are exactly same the
-	  * partitions at the same position from both the relations match.
-	  */
-	for (cnt = 0; cnt < outer_rel->nparts; cnt++)
-	{
-		*outer_parts = lappend_int(*outer_parts, cnt);
-		*inner_parts = lappend_int(*inner_parts, cnt);
-	}
-
-	return outer_bi;
-}
-
 /*
  * map_and_merge_partitions
  *
@@ -4312,12 +4253,11 @@ map_and_merge_partitions(PartitionMap *partmaps1, PartitionMap *partmaps2,
  * set to NIL.
  */
 static void
-generate_matching_part_pairs(PartitionMap *partmaps1, PartitionMap *partmaps2,
-							 int nparts1, int nparts2,
-							 JoinType jointype, int nparts,
+generate_matching_part_pairs(RelOptInfo *rel1, RelOptInfo *rel2,
+							 PartitionMap *partmaps1, PartitionMap *partmaps2,
+							 int nparts1, int nparts2, int nparts,
 							 List **matched_parts1, List **matched_parts2)
 {
-	bool		merged = true;
 	int		   *matching1,
 			   *matching2;
 	int 		i;
@@ -4358,16 +4298,6 @@ generate_matching_part_pairs(PartitionMap *partmaps1, PartitionMap *partmaps2,
 		}
 	}
 
-	/*
-	 * If we have a partition missing on an inner side, we need to add a dummy
-	 * relation which joins with the outer partition. If the inner relation
-	 * happens to be a base relation, it will require adding a dummy child
-	 * base relation during join processing. Right now, we freeze the base
-	 * relation arrays like PlannerInfo::simple_rte_array after planning for
-	 * base relations. Adding a new (dummy) base relation would require some
-	 * changes to that. So, right now, we do not implement partition-wise join
-	 * in such cases.
-	 */
 	for (i = 0; i < nparts; i++)
 	{
 		int			part1 = matching1[i];
@@ -4376,52 +4306,14 @@ generate_matching_part_pairs(PartitionMap *partmaps1, PartitionMap *partmaps2,
 		/* At least one of the partitions should exist. */
 		Assert(part1 >= 0 || part2 >= 0);
 
-		switch (jointype)
-		{
-			case JOIN_INNER:
-			case JOIN_SEMI:
-
-				/*
-				 * An inner or semi join can not return any row when the
-				 * matching partition on either side is missing. We should
-				 * have eliminated all such cases while merging the bounds.
-				 */
-				Assert(part1 >= 0 && part2 >= 0);
-				break;
-
-			case JOIN_LEFT:
-			case JOIN_ANTI:
-				Assert(part1 >= 0);
-				if (part2 < 0)
-					merged = false;
-				break;
-
-			case JOIN_FULL:
-				if (part1 < 0 || part2 < 0)
-					merged = false;
-				break;
-
-			default:
-				elog(ERROR, "unrecognized join type: %d", (int) jointype);
-		}
-
-		if (!merged)
-			break;
-
-		*matched_parts1 = lappend_int(*matched_parts1, part1);
-		*matched_parts2 = lappend_int(*matched_parts2, part2);
+		*matched_parts1 = lappend(*matched_parts1,
+								  part1 >= 0 ? rel1->part_rels[part1] : NULL);
+		*matched_parts2 = lappend(*matched_parts2,
+								  part2 >= 0 ? rel2->part_rels[part2] : NULL);
 	}
 
 	pfree(matching1);
 	pfree(matching2);
-
-	if (!merged)
-	{
-		list_free(*matched_parts1);
-		list_free(*matched_parts2);
-		*matched_parts1 = NIL;
-		*matched_parts2 = NIL;
-	}
 }
 
 static PartitionBoundInfo
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index 23a06d718e..e06eb9aaae 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -577,8 +577,10 @@ typedef struct PartitionSchemeData *PartitionScheme;
  *		part_scheme - Partitioning scheme of the relation
  *		nparts - Number of partitions
  *		boundinfo - Partition bounds
+ *		merged - true if partition bounds are merged ones
  *		partition_qual - Partition constraint if not the root
  *		part_rels - RelOptInfos for each partition
+ *		all_partrels - Relids set of all partition relids
  *		partexprs, nullable_partexprs - Partition key expressions
  *		partitioned_child_rels - RT indexes of unpruned partitions of
  *								 this relation that are partitioned tables
@@ -718,9 +720,12 @@ typedef struct RelOptInfo
 	PartitionScheme part_scheme;	/* Partitioning scheme. */
 	int			nparts;			/* number of partitions */
 	struct PartitionBoundInfoData *boundinfo;	/* Partition bounds */
+	bool		merged;			/* true if partition bounds were created by
+								 * partition_bounds_merge() */
 	List	   *partition_qual; /* partition constraint */
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of partitions,
 									 * stored in the same order of bounds */
+	Relids		all_partrels;	/* Relids set of all partition relids */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 	List	   *partitioned_child_rels; /* List of RT indexes. */
#91amul sul
sulamul@gmail.com
In reply to: Etsuro Fujita (#90)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Thank you Fujita San for the enhancement, will have a look.

Regards,
Amul

On Wed, Aug 28, 2019 at 3:52 PM Etsuro Fujita <etsuro.fujita@gmail.com>
wrote:

Show quoted text

On Fri, Aug 16, 2019 at 10:25 PM Etsuro Fujita <etsuro.fujita@gmail.com>
wrote:

It seems that I performed the above tests on an assertion-enabled
build. :( So I executed the tests one more time. Here are the
results.

* 2-way self-join of pt: explain analyze select * from pt t0, pt t1
where t0.a = t1.a;
- HEAD:
Planning Time: 0.969 ms
Execution Time: 13.843 ms
- with patch:
Planning Time: 1.720 ms
Execution Time: 14.393 ms
- with patch plus attached:
Planning Time: 1.630 ms
Execution Time: 14.002 ms

* 4-way self-join of pt: explain analyze select * from pt t0, pt t1,
pt t2, pt t3 where t0.a = t1.a

and t1.a = t2.a and t2.a = t3.a;
- HEAD:
Planning Time: 12.203 ms
Execution Time: 31.784 ms
- with patch:
Planning Time: 32.102 ms
Execution Time: 32.504 ms
- with patch plus attached:
Planning Time: 19.471 ms
Execution Time: 32.582 ms

* 8-way self-join of pt: explain analyze select * from pt t0, pt t1,
pt t2, pt t3, pt t4, pt t5, pt t6, pt t7 where t0.a = t1.a and t1.a =
t2.a and t2.a = t3.a and t3.a = t4.a and t4.a = t5.a and t5.a = t6.a
and t6.a = t7.a;
- HEAD:
Planning Time: 948.131 ms
Execution Time: 55.645 ms
- with patch:
Planning Time: 2939.813 ms
Execution Time: 56.760 ms
- with patch plus attached:
Planning Time: 1108.076 ms
Execution Time: 55.750 ms

Note: the attached patch still uses the proposed partition matching
algorithm for these queries. As I said before, these queries don't
need that algorithm, so we could eliminate the planning overhead
compared to HEAD, by planning these queries as before, perhaps, but I
haven't modified the patch as such yet.

I modified the patch further as such. Attached is an updated version
of the patch created on top of the patch in [1]. I did the tests
again using the updated version of the patch. Here are the results:

* 2-way self-join of pt:
Planning Time: 1.043 ms
Execution Time: 13.931 ms

* 4-way self-join of pt:
Planning Time: 12.499 ms
Execution Time: 32.392 ms

* 8-way self-join of pt:
Planning Time: 968.412 ms
Execution Time: 56.328 ms

The planning time for each test case still increased slightly, but IMO
I think that would be acceptable. To see the efficiency of the
attached, I did another testing with test cases that really need the
new partition-matching algorithm:

* explain analyze select * from pt6 t6, pt7 t7 where t6.a = t7.a;
- base patch in [1]
Planning Time: 1.758 ms
Execution Time: 13.977 ms
- with attached
Planning Time: 1.777 ms
Execution Time: 13.959 ms

* explain analyze select * from pt4 t4, pt5 t5, pt6 t6, pt7 t7 where
t4.a = t5.a and t5.a = t6.a and t6.a = t7.a;
- base patch in [1]
Planning Time: 33.201 ms
Execution Time: 32.480 ms
- with attached
Planning Time: 21.019 ms
Execution Time: 32.777 ms

* explain analyze select * from pt0 t0, pt1 t1, pt2 t2, pt3 t3, pt4
t4, pt5 t5, pt6 t6, pt7 t7 where t0.a = t1.a and t1.a = t2.a and t2.a
= t3.a and t3.a = t4.a and t4.a = t5.a and t5.a = t6.a and t6.a =
t7.a;
- base patch in [1]
Planning Time: 3060.000 ms
Execution Time: 55.553 ms
- with attached
Planning Time: 1144.996 ms
Execution Time: 56.233 ms

where pt0, pt1, pt2, pt3, pt4, pt5, pt6 and pt7 are list partitioned
tables that have slighly different list values. (The structure and
list values of ptN are almost the same as that of pt used above, but
ptN's N-th partition ptNpN has an extra list value that pt's N-th
partition ptpN doesn't have.) If anyone is interested in this
testing, I'll send a script file for producing these list partitioned
tables.

About the attached:

* The attached patch modified try_partitionwise_join() so that we call
partition_bounds_equal() then partition_bounds_merge() (if necessary)
to create the partition bounds for the join rel. We don't support for
merging the partition bounds for the hash-partitioning case, so this
makes code to handle the hash-partitioning case in
partition_bounds_merge() completely unnecessary, so I removed that
entirely.

* I removed this assertion in partition_bounds_merge(), because I
think this is covered by two assertions above this.

+   Assert((*outer_parts == NIL || *inner_parts != NIL) &&
+          (*inner_parts == NIL || *outer_parts != NIL));

* (I forgot to mention this in a previous email, but) I removed this
bit of generate_matching_part_pairs(), because we already have the
same check in try_partitionwise_join(), so this bit would be redundant
IIUC.

+       switch (jointype)
+       {
+           case JOIN_INNER:
+           case JOIN_SEMI:
+
+               /*
+                * An inner or semi join can not return any row when the
+                * matching partition on either side is missing. We should
+                * have eliminated all such cases while merging the bounds.
+                */
+               Assert(part1 >= 0 && part2 >= 0);
+               break;
+
+           case JOIN_LEFT:
+           case JOIN_ANTI:
+               Assert(part1 >= 0);
+               if (part2 < 0)
+                   merged = false;
+               break;
+
+           case JOIN_FULL:
+               if (part1 < 0 || part2 < 0)
+                   merged = false;
+               break;
+
+           default:
+               elog(ERROR, "unrecognized join type: %d", (int) jointype);
+       }

* I added more comments.

If there are no objections, I'll merge the attached with the base patch in
[1].

Best regards,
Etsuro Fujita

[1]
/messages/by-id/CAPmGK177W+jNgpM5f_m-vdDKbEBu_=3CyPzFjkT_1nzf1Vqn+A@mail.gmail.com

#92amul sul
sulamul@gmail.com
In reply to: Etsuro Fujita (#90)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Hi Fujita San,

Please find my comments inline below:

On Wed, Aug 28, 2019 at 3:52 PM Etsuro Fujita <etsuro.fujita@gmail.com>
wrote:

On Fri, Aug 16, 2019 at 10:25 PM Etsuro Fujita <etsuro.fujita@gmail.com>
wrote:

[... skipped ..]

About the attached:

* The attached patch modified try_partitionwise_join() so that we call
partition_bounds_equal() then partition_bounds_merge() (if necessary)
to create the partition bounds for the join rel. We don't support for
merging the partition bounds for the hash-partitioning case, so this
makes code to handle the hash-partitioning case in
partition_bounds_merge() completely unnecessary, so I removed that
entirely.

Yes, that make sense.

On thinking further, a thought hits to me why we can't join two hash
partitioned
table which has the same modulus and partition key specification, but some
of
the partitions are missing from either partitioned table.

For e.g. here is a smaller case where foo has two partitions and bar has
only one.

CREATE TABLE foo(a int) PARTITION BY HASH(a);
CREATE TABLE foo_p0 PARTITION OF foo FOR VALUES WITH(modulus 2, remainder
0);
CREATE TABLE foo_p1 PARTITION OF foo FOR VALUES WITH(modulus 2, remainder
1);

CREATE TABLE bar(a int) PARTITION BY HASH(a); <-- missing partitions
for REMAINDER 1
CREATE TABLE bar_p0 PARTITION OF bar FOR VALUES WITH(modulus 2, remainder
0);

Explain:
postgres=# explain select * from foo p1, bar p2 where p1.a = p2.a;
QUERY PLAN

---------------------------------------------------------------------------------
Merge Join (cost=590.35..1578.47 rows=65025 width=8)
Merge Cond: (p2.a = p1.a)
-> Sort (cost=179.78..186.16 rows=2550 width=4)
Sort Key: p2.a
-> Seq Scan on bar_p0 p2 (cost=0.00..35.50 rows=2550 width=4)
-> Sort (cost=410.57..423.32 rows=5100 width=4)
Sort Key: p1.a
-> Append (cost=0.00..96.50 rows=5100 width=4)
-> Seq Scan on foo_p0 p1 (cost=0.00..35.50 rows=2550
width=4)
-> Seq Scan on foo_p1 p1_1 (cost=0.00..35.50 rows=2550
width=4)
(10 rows)

The partitions-wise join will be performed only if we fill the partition
hole that
exists for the joining table i.e. adding partitions to bar table.

postgres=# CREATE TABLE bar_p1 PARTITION OF bar FOR VALUES WITH(modulus 2,
remainder 1);
CREATE TABLE
postgres=# explain select * from foo p1, bar p2 where p1.a = p2.a;
QUERY PLAN

---------------------------------------------------------------------------------
Append (cost=359.57..2045.11 rows=65024 width=8)
-> Merge Join (cost=359.57..860.00 rows=32512 width=8)
Merge Cond: (p1.a = p2.a)
-> Sort (cost=179.78..186.16 rows=2550 width=4)
Sort Key: p1.a
-> Seq Scan on foo_p0 p1 (cost=0.00..35.50 rows=2550
width=4)
-> Sort (cost=179.78..186.16 rows=2550 width=4)
Sort Key: p2.a
-> Seq Scan on bar_p0 p2 (cost=0.00..35.50 rows=2550
width=4)
-> Merge Join (cost=359.57..860.00 rows=32512 width=8)
Merge Cond: (p1_1.a = p2_1.a)
-> Sort (cost=179.78..186.16 rows=2550 width=4)
Sort Key: p1_1.a
-> Seq Scan on foo_p1 p1_1 (cost=0.00..35.50 rows=2550
width=4)
-> Sort (cost=179.78..186.16 rows=2550 width=4)
Sort Key: p2_1.a
-> Seq Scan on bar_p1 p2_1 (cost=0.00..35.50 rows=2550
width=4)
(17 rows)

It would have been nice if we could support this case, as we do allow
partitions
hole for the other partition scheme, but there wouldn't be much objection if
we don't want to add this support for now since there will be a lesser
chance
that hash partitioned table has the hole, IMO.

* I removed this assertion in partition_bounds_merge(), because I
think this is covered by two assertions above this.

+   Assert((*outer_parts == NIL || *inner_parts != NIL) &&
+          (*inner_parts == NIL || *outer_parts != NIL));

* (I forgot to mention this in a previous email, but) I removed this
bit of generate_matching_part_pairs(), because we already have the
same check in try_partitionwise_join(), so this bit would be redundant
IIUC.

Looks good.

* I added more comments.

Thanks.

If there are no objections, I'll merge the attached with the base patch in
[1].

The proposed enhancement in the patch is too good and the patch is pretty
much
reasonable to merge into the main patch.

Here are the few cosmetic fixes for this path I think is needed. Feel free
to
ignore if some of them do not make sense or too obvious.

Note: left side number represents code line number of the patch.

118 + }
119 +
120 + /*
121 + * Try to create the partition bounds along with join pairs.
122 + */
123 + if (boundinfo == NULL)
124 + {

Can we add this block as else part of previous if condition checking equal
partitions bound?

133 + Assert(list_length(parts1) == list_length(parts2));
134 + if (boundinfo == NULL)
135 + {
136 + joinrel->nparts = 0;
137 + return;
138 + }
139 + nparts = list_length(parts1);

Can we move the assert check below at line#139 in the patch i.e. after if
block.

And the question is do we need to do that assert check since
partition_bounds_merge()
does that just before returning, thoughts?

204 + RelOptInfo *child_rel1 = merged ? (RelOptInfo *) lfirst(lcr1) :
rel1->part_rels[cnt_parts];
205 + RelOptInfo *child_rel2 = merged ? (RelOptInfo *) lfirst(lcr2) :
rel2->part_rels[cnt_parts];

How about using lfirst_node instead of lfirst & casting explicitly?

Also, these lines crossing 80 column length which I think we need to fix.
How about
doing the assignment as follow, just after the variable declaration part:

if (merged)
{
child_rel1 = lfirst_node(lcr1, RelOptInfo);
child_rel2 = lfirst_node(lcr2, RelOptInfo);
lcr1 = lnext(parts1, lcr1);
lcr2 = lnext(parts2, lcr2);
}
else
{
child_rel1 = rel1->part_rels[cnt_parts];
child_rel2 = rel2->part_rels[cnt_parts]
}

rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1));
rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2));

266 + * get_matching_part_pairs
267 + * Generate join pairs of partitions for the two inputs for the
join rel

Can we rewrite this description as " Generate join pairs of partitions for
the
join rel from the two inputs." OR "Generate join pairs of partitions for
the
two inputs"

310 + Assert(bms_num_members(child_relids1) ==
bms_num_members(rel1->relids));
311 + /*

335 + Assert(bms_num_members(child_relids2) ==
bms_num_members(rel2->relids));
336 + /*

Need newline after assert statements.

Regards,
Amul

#93Alvaro Herrera
alvherre@2ndquadrant.com
In reply to: Etsuro Fujita (#90)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Fujita-san, amul,

CFbot complains that Fujita-san submitted a patch that doesn't apply,
which makes sense since the necessary previous patch was only referred
to without being resubmitted. I suggest to always post all patches
together with each resubmission so that it can be checked automatically
by the cf bot: http://commitfest.cputube.org/etsuro-fujita.html

I'm not clear on who the author of this patch is, now. Also, I'm not
sure what the *status* is. Are we waiting for Fujita-san to review this
patch?

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

#94amul sul
sulamul@gmail.com
In reply to: Alvaro Herrera (#93)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Wed, Sep 4, 2019 at 2:40 AM Alvaro Herrera <alvherre@2ndquadrant.com>
wrote:

Fujita-san, amul,

CFbot complains that Fujita-san submitted a patch that doesn't apply,
which makes sense since the necessary previous patch was only referred
to without being resubmitted. I suggest to always post all patches
together with each resubmission so that it can be checked automatically
by the cf bot: http://commitfest.cputube.org/etsuro-fujita.html

Understood and sorry about that.

I'm not clear on who the author of this patch is, now. Also, I'm not

sure what the *status* is. Are we waiting for Fujita-san to review this
patch?

Yes, we are waiting for Fujita-san to review. Fujita-san has started a
review
and proposed some enhancement which I reviewed in the last update.

I think soon Fujita-san might post the complete patch including his changes.

Regards,
Amul

#95Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: amul sul (#94)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Hi,

On Thu, Sep 5, 2019 at 1:24 PM amul sul <sulamul@gmail.com> wrote:

On Wed, Sep 4, 2019 at 2:40 AM Alvaro Herrera <alvherre@2ndquadrant.com> wrote:

CFbot complains that Fujita-san submitted a patch that doesn't apply,
which makes sense since the necessary previous patch was only referred
to without being resubmitted. I suggest to always post all patches
together with each resubmission so that it can be checked automatically
by the cf bot: http://commitfest.cputube.org/etsuro-fujita.html

Understood and sorry about that.

Sorry about that.

I'm not clear on who the author of this patch is, now. Also, I'm not
sure what the *status* is. Are we waiting for Fujita-san to review this
patch?

Yes, we are waiting for Fujita-san to review. Fujita-san has started a review
and proposed some enhancement which I reviewed in the last update.

I think soon Fujita-san might post the complete patch including his changes.

I'm a bit busy with something else recently, but I'll do that ASAP.
And I'll continue to review the other part of the patch.

Best regards,
Etsuro Fujita

#96Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: amul sul (#92)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Hi Amul,

On Mon, Sep 2, 2019 at 2:08 PM amul sul <sulamul@gmail.com> wrote:

Please find my comments inline below:

Thank you for the review!

On Wed, Aug 28, 2019 at 3:52 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

On Fri, Aug 16, 2019 at 10:25 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

On thinking further, a thought hits to me why we can't join two hash partitioned
table which has the same modulus and partition key specification, but some of
the partitions are missing from either partitioned table.

For e.g. here is a smaller case where foo has two partitions and bar has only one.

CREATE TABLE foo(a int) PARTITION BY HASH(a);
CREATE TABLE foo_p0 PARTITION OF foo FOR VALUES WITH(modulus 2, remainder 0);
CREATE TABLE foo_p1 PARTITION OF foo FOR VALUES WITH(modulus 2, remainder 1);

CREATE TABLE bar(a int) PARTITION BY HASH(a); <-- missing partitions for REMAINDER 1
CREATE TABLE bar_p0 PARTITION OF bar FOR VALUES WITH(modulus 2, remainder 0);

Explain:
postgres=# explain select * from foo p1, bar p2 where p1.a = p2.a;
QUERY PLAN
---------------------------------------------------------------------------------
Merge Join (cost=590.35..1578.47 rows=65025 width=8)
Merge Cond: (p2.a = p1.a)
-> Sort (cost=179.78..186.16 rows=2550 width=4)
Sort Key: p2.a
-> Seq Scan on bar_p0 p2 (cost=0.00..35.50 rows=2550 width=4)
-> Sort (cost=410.57..423.32 rows=5100 width=4)
Sort Key: p1.a
-> Append (cost=0.00..96.50 rows=5100 width=4)
-> Seq Scan on foo_p0 p1 (cost=0.00..35.50 rows=2550 width=4)
-> Seq Scan on foo_p1 p1_1 (cost=0.00..35.50 rows=2550 width=4)
(10 rows)

The partitions-wise join will be performed only if we fill the partition hole that
exists for the joining table i.e. adding partitions to bar table.

postgres=# CREATE TABLE bar_p1 PARTITION OF bar FOR VALUES WITH(modulus 2, remainder 1);
CREATE TABLE
postgres=# explain select * from foo p1, bar p2 where p1.a = p2.a;
QUERY PLAN
---------------------------------------------------------------------------------
Append (cost=359.57..2045.11 rows=65024 width=8)
-> Merge Join (cost=359.57..860.00 rows=32512 width=8)
Merge Cond: (p1.a = p2.a)
-> Sort (cost=179.78..186.16 rows=2550 width=4)
Sort Key: p1.a
-> Seq Scan on foo_p0 p1 (cost=0.00..35.50 rows=2550 width=4)
-> Sort (cost=179.78..186.16 rows=2550 width=4)
Sort Key: p2.a
-> Seq Scan on bar_p0 p2 (cost=0.00..35.50 rows=2550 width=4)
-> Merge Join (cost=359.57..860.00 rows=32512 width=8)
Merge Cond: (p1_1.a = p2_1.a)
-> Sort (cost=179.78..186.16 rows=2550 width=4)
Sort Key: p1_1.a
-> Seq Scan on foo_p1 p1_1 (cost=0.00..35.50 rows=2550 width=4)
-> Sort (cost=179.78..186.16 rows=2550 width=4)
Sort Key: p2_1.a
-> Seq Scan on bar_p1 p2_1 (cost=0.00..35.50 rows=2550 width=4)
(17 rows)

It would have been nice if we could support this case, as we do allow partitions
hole for the other partition scheme, but there wouldn't be much objection if
we don't want to add this support for now since there will be a lesser chance
that hash partitioned table has the hole, IMO.

I agree with you on that point.

If there are no objections, I'll merge the attached with the base patch in [1].

The proposed enhancement in the patch is too good and the patch is pretty much
reasonable to merge into the main patch.

Done. Attached is a merged version of the patch.

Here are the few cosmetic fixes for this path I think is needed. Feel free to
ignore if some of them do not make sense or too obvious.

Note: left side number represents code line number of the patch.

118 + }
119 +
120 + /*
121 + * Try to create the partition bounds along with join pairs.
122 + */
123 + if (boundinfo == NULL)
124 + {

Can we add this block as else part of previous if condition checking equal partitions bound?

Done.

133 + Assert(list_length(parts1) == list_length(parts2));
134 + if (boundinfo == NULL)
135 + {
136 + joinrel->nparts = 0;
137 + return;
138 + }
139 + nparts = list_length(parts1);

And the question is do we need to do that assert check since partition_bounds_merge()
does that just before returning, thoughts?

You are right; that assertion would be redundant, so I removed that.

204 + RelOptInfo *child_rel1 = merged ? (RelOptInfo *) lfirst(lcr1) : rel1->part_rels[cnt_parts];
205 + RelOptInfo *child_rel2 = merged ? (RelOptInfo *) lfirst(lcr2) : rel2->part_rels[cnt_parts];

How about using lfirst_node instead of lfirst & casting explicitly?

Also, these lines crossing 80 column length which I think we need to fix. How about
doing the assignment as follow, just after the variable declaration part:

if (merged)
{
child_rel1 = lfirst_node(lcr1, RelOptInfo);
child_rel2 = lfirst_node(lcr2, RelOptInfo);
lcr1 = lnext(parts1, lcr1);
lcr2 = lnext(parts2, lcr2);
}
else
{
child_rel1 = rel1->part_rels[cnt_parts];
child_rel2 = rel2->part_rels[cnt_parts]
}

rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1));
rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2));

Done. (I don't think the order of arguments for first_node() above is
correct; the first argument for that should be RelOptInfo.)

266 + * get_matching_part_pairs
267 + * Generate join pairs of partitions for the two inputs for the join rel

Can we rewrite this description as " Generate join pairs of partitions for the
join rel from the two inputs." OR "Generate join pairs of partitions for the
two inputs"

Done.

310 + Assert(bms_num_members(child_relids1) == bms_num_members(rel1->relids));
311 + /*

335 + Assert(bms_num_members(child_relids2) == bms_num_members(rel2->relids));
336 + /*

Need newline after assert statements.

Done.

Other change: I guess this in partbounds.c would be leftovers from an
older version of the patch, so I removed it.

+/*
+ * Allocate and initialize partition maps. We maintain four maps, two maps
+ * for each joining relation. pmap[i] gives the partition from the other
+ * relation which would join with ith partition of the given relation.
+ * Partition i from the given relation will join with partition pmap[i]
+ * from the other relation to produce partition mmap[i] of the join (merged
+ * partition).
+ *
+ * pmap[i] = -1 indicates that ith partition of a given relation does not
+ * have a matching partition from the other relation.
+ *
+ * mmap[i] = -1 indicates that ith partition of a given relation does not
+ * contribute to the join result. That can happen only when the given
+ * relation is the inner relation and it doesn't have a matching partition
+ * from the outer relation, hence pmap[i] should be -1.
+ *
+ * In case of an outer join, every partition of the outer join will appear
+ * in the join result, and thus has mmap[i] set for all i. But it's not
+ * necessary that every partition on the outer side will have a matching
+ * partition on the inner side. In such a case, we end up with pmap[i] = -1
+ * and mmap[i] != -1.
+ */

I will continue to review the rest of the patch.

I am sorry for the long delay.

Best regards,
Etsuro Fujita

Attachments:

Improve-partition-matching-for-partitionwise-joins-v24.patchapplication/octet-stream; name=Improve-partition-matching-for-partitionwise-joins-v24.patchDownload
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index b0dcd02ff6..b815157402 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -2281,6 +2281,8 @@ _outRelOptInfo(StringInfo str, const RelOptInfo *node)
 	WRITE_BOOL_FIELD(has_eclass_joins);
 	WRITE_BOOL_FIELD(consider_partitionwise_join);
 	WRITE_BITMAPSET_FIELD(top_parent_relids);
+	WRITE_BOOL_FIELD(merged);
+	WRITE_BITMAPSET_FIELD(all_partrels);
 	WRITE_NODE_FIELD(partitioned_child_rels);
 }
 
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 6a480ab764..ed7bc23c7b 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -48,6 +48,9 @@ static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 												Relids left_relids, Relids right_relids);
 static int	match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel,
 										 bool strict_op);
+static void get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2);
 
 
 /*
@@ -1357,25 +1360,30 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	bool		merged = false;
+	List	   *parts1 = NIL;
+	List	   *parts2 = NIL;
+	ListCell   *lcr1 = NULL;
+	ListCell   *lcr2 = NULL;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1390,34 +1398,107 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		   joinrel->part_scheme == rel2->part_scheme);
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * If we don't have the partition bounds for the join rel yet, try to
+	 * create it along with pairs of partitions to be joined; else generate
+	 * those using the partitioning info for the join rel we already have.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	if (joinrel->nparts == -1)
+	{
+		PartitionScheme part_scheme = joinrel->part_scheme;
+		PartitionBoundInfo boundinfo = NULL;
+		int			nparts = 0;
+
+		Assert(joinrel->boundinfo == NULL);
+		Assert(joinrel->part_rels == NULL);
+
+		/*
+		 * See if the partition bounds for inputs are exactly the same, in
+		 * which case we don't need to work hard: partitions with the same
+		 * partition indexes will form join pairs, and the join rel will have
+		 * the same partition bounds as inputs; otherwise try to merge the
+		 * partition bounds along with generating join pairs.
+		 *
+		 * Even if one or both inputs have merged partition bounds, it'd be
+		 * possible for the partition bounds to be exactly the same, but it
+		 * seems unlikely to be worth the cycles to check; do this check only
+		 * if both inputs have non-merged partition bounds.
+		 */
+		if (!rel1->merged &&
+			!rel2->merged &&
+			rel1->nparts == rel2->nparts &&
+			partition_bounds_equal(part_scheme->partnatts,
+								   part_scheme->parttyplen,
+								   part_scheme->parttypbyval,
+								   rel1->boundinfo, rel2->boundinfo))
+		{
+			boundinfo = rel1->boundinfo;
+			nparts = rel1->nparts;
+		}
+		else
+		{
+			boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											   part_scheme->parttyplen,
+											   part_scheme->parttypbyval,
+											   part_scheme->partsupfunc,
+											   part_scheme->partcollation,
+											   rel1, rel2,
+											   parent_sjinfo->jointype,
+											   &parts1, &parts2);
+			if (boundinfo == NULL)
+			{
+				joinrel->nparts = 0;
+				return;
+			}
+			nparts = list_length(parts1);
+			merged = true;
+		}
+
+		Assert(nparts > 0);
+		joinrel->boundinfo = boundinfo;
+		joinrel->merged = merged;
+		joinrel->nparts = nparts;
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+	}
+	else
+	{
+		Assert(joinrel->nparts > 0);
+		Assert(joinrel->boundinfo);
+		Assert(joinrel->part_rels);
+
+		/*
+		 * If the partition bounds for the join rel are not merged ones,
+		 * inputs are guaranteed to have the same partition bounds, so
+		 * partitions with the same partition indexes will form join pairs;
+		 * else let get_matching_part_pairs() do the work.
+		 */
+		if (joinrel->merged)
+		{
+			get_matching_part_pairs(root, joinrel, rel1, rel2,
+									&parts1, &parts2);
+			Assert(list_length(parts1) == joinrel->nparts);
+			Assert(list_length(parts2) == joinrel->nparts);
+			merged = true;
+		}
+	}
 
-	nparts = joinrel->nparts;
+	if (merged)
+	{
+		lcr1 = list_head(parts1);
+		lcr2 = list_head(parts2);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
-		bool		rel1_empty = (child_rel1 == NULL ||
-								  IS_DUMMY_REL(child_rel1));
-		bool		rel2_empty = (child_rel2 == NULL ||
-								  IS_DUMMY_REL(child_rel2));
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		bool		rel1_empty;
+		bool		rel2_empty;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1425,6 +1506,22 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		if (merged)
+		{
+			child_rel1 = lfirst_node(RelOptInfo, lcr1);
+			child_rel2 = lfirst_node(RelOptInfo, lcr2);
+			lcr1 = lnext(parts1, lcr1);
+			lcr2 = lnext(parts2, lcr2);
+		}
+		else
+		{
+			child_rel1 = rel1->part_rels[cnt_parts];
+			child_rel2 = rel2->part_rels[cnt_parts];
+		}
+
+		rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1));
+		rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2));
+
 		/*
 		 * Check for cases where we can prove that this segment of the join
 		 * returns no rows, due to one or both inputs being empty (including
@@ -1522,6 +1619,8 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 												 child_sjinfo,
 												 child_sjinfo->jointype);
 			joinrel->part_rels[cnt_parts] = child_joinrel;
+			joinrel->all_partrels = bms_add_members(joinrel->all_partrels,
+													child_joinrel->relids);
 		}
 
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
@@ -1529,6 +1628,7 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
+
 	}
 }
 
@@ -1738,3 +1838,99 @@ match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op)
 
 	return -1;
 }
+
+/*
+ * get_matching_part_pairs
+ *		Generate join pairs of partitions for the two inputs
+ */
+static void
+get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2)
+{
+	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
+	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
+	int 		cnt_parts;
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
+	{
+		RelOptInfo *child_joinrel = joinrel->part_rels[cnt_parts];
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		Relids		child_relids1;
+		Relids		child_relids2;
+
+		/*
+		 * If this segment of the join is empty, it means that this segment
+		 * was ignored when previously creating child-join paths for it in
+		 * try_partitionwise_join() as it would not contribute to the join
+		 * result, due to one or both inputs being empty; add NULL to each of
+		 * the given lists so that this segment will be ignored again in that
+		 * function.
+		 */
+		if (!child_joinrel)
+		{
+			*parts1 = lappend(*parts1, NULL);
+			*parts2 = lappend(*parts2, NULL);
+			continue;
+		}
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel1 side.
+		 */
+		child_relids1 = bms_intersect(child_joinrel->relids,
+									  rel1->all_partrels);
+		Assert(bms_num_members(child_relids1) == bms_num_members(rel1->relids));
+
+		/*
+		 * Get a child rel for rel1 with the relids.  Note that we should have
+		 * the child rel even if rel1 is a join rel, because in that case the
+		 * partitions specified in the relids would have matching/overlapping
+		 * boundaries, so those partitions should be considered as ones to be
+		 * joined even when planning partitionwise joins of rel1, meaning that
+		 * the child rel would have been built by the time we get here.
+		 */
+		if (rel1_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids1);
+
+			child_rel1 = find_base_rel(root, varno);
+		}
+		else
+			child_rel1 = find_join_rel(root, child_relids1);
+		Assert(child_rel1);
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel2 side.
+		 */
+		child_relids2 = bms_intersect(child_joinrel->relids,
+									  rel2->all_partrels);
+		Assert(bms_num_members(child_relids2) == bms_num_members(rel2->relids));
+
+		/*
+		 * Get a child rel for rel2 with the relids.  See above comments.
+		 */
+		if (rel2_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids2);
+
+			child_rel2 = find_base_rel(root, varno);
+		}
+		else
+			child_rel2 = find_join_rel(root, child_relids2);
+		Assert(child_rel2);
+
+		/*
+		 * The join of rel1 and rel2 is legal, so is the join of the child
+		 * rels obtained above; add them to the given lists as a join pair
+		 * producing this join segment.
+		 */
+		*parts1 = lappend(*parts1, child_rel1);
+		*parts2 = lappend(*parts2, child_rel2);
+	}
+}
diff --git a/src/backend/optimizer/util/inherit.c b/src/backend/optimizer/util/inherit.c
index 38bc61e687..caf6039c10 100644
--- a/src/backend/optimizer/util/inherit.c
+++ b/src/backend/optimizer/util/inherit.c
@@ -376,6 +376,8 @@ expand_partitioned_rtentry(PlannerInfo *root, RelOptInfo *relinfo,
 		/* Create the otherrel RelOptInfo too. */
 		childrelinfo = build_simple_rel(root, childRTindex, relinfo);
 		relinfo->part_rels[i] = childrelinfo;
+		relinfo->all_partrels = bms_add_members(relinfo->all_partrels,
+												childrelinfo->relids);
 
 		/* If this child is itself partitioned, recurse */
 		if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 85415381fb..248a9e9093 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -240,10 +240,12 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->has_eclass_joins = false;
 	rel->consider_partitionwise_join = false;	/* might get changed later */
 	rel->part_scheme = NULL;
-	rel->nparts = 0;
+	rel->nparts = -1;
 	rel->boundinfo = NULL;
+	rel->merged = false;
 	rel->partition_qual = NIL;
 	rel->part_rels = NULL;
+	rel->all_partrels = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->partitioned_child_rels = NIL;
@@ -653,10 +655,12 @@ build_join_rel(PlannerInfo *root,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -829,10 +833,12 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -1626,7 +1632,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1639,24 +1645,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1668,17 +1656,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 318d8ecae9..1238999c22 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -25,6 +25,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/pathnodes.h"
 #include "parser/parse_coerce.h"
 #include "partitioning/partbounds.h"
 #include "partitioning/partdesc.h"
@@ -69,6 +70,12 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+typedef struct PartitionMap
+{
+	int from;
+	int to;
+} PartitionMap;
+
 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);
@@ -108,6 +115,57 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 									 Expr **keyCol,
 									 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static PartitionBoundInfo partition_range_bounds_merge(
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 List **outer_parts, List **inner_parts,
+							 JoinType jointype, int partnatts,
+							 FmgrInfo *supfuncs, Oid *collations);
+static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype);
+static void generate_matching_part_pairs(RelOptInfo *rel1,
+							 RelOptInfo *rel2,
+							 PartitionMap *partmaps1,
+							 PartitionMap *partmaps2,
+							 int nparts1, int nparts2,
+							 int nparts,
+							 List **matched_parts1,
+							 List **matched_parts2);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int map_and_merge_partitions(PartitionMap *outer_maps,
+										PartitionMap *inner_maps,
+										int index1, int index2, int *next_index);
+static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *collations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2);
+static bool partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
+						   Oid *collations, PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, int *ub_cmpval,
+						   int *lb_cmpval);
+static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes);
+static bool merge_default_partitions(PartitionBoundInfo outer_bi,
+						 PartitionBoundInfo inner_bi,
+						 PartitionMap *outer_maps,
+						 PartitionMap *inner_maps,
+						 JoinType jointype,
+						 int *next_index, int *default_index);
+static bool merge_null_partitions(PartitionBoundInfo outer_bi,
+								  PartitionBoundInfo inner_bi,
+								  PartitionMap *outer_maps,
+								  PartitionMap *inner_maps,
+								  JoinType jointype,
+								  int *next_index, int *null_index,
+								  int *default_index);
 
 /*
  * get_qual_from_partbound
@@ -2996,3 +3054,1491 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * The function produces the partition bounds for a join between two relations
+ * whose partition bounds are given. The function also returns two lists of
+ * partition indexes one for each of the joining relations. Both the lists
+ * contain the same number of elements. The partition indexes at the same
+ * positions in the lists indicate the pair partitions, one from each side, to
+ * be joined and the position itself corresponds to the index of partition
+ * produced by that child-join in the partitioned join.
+ *
+ * The function returns NULL if we can not find the matching pair of
+ * partitions. This happens if 1. multiple partitions on one side match with
+ * one partition on the other side. 2. a given partition on the outer side
+ * doesn't have a matching partition on the inner side. We can not support the
+ * first case since we don't have a way to represent multiple partitions as a
+ * single relation (RelOptInfo) and then perform join using the ganged
+ * relation. We can not support the second case since the missing inner
+ * partition needs to be represented as an empty relation and we don't have a
+ * way to introduce empty relation during join planning after creating paths
+ * for all the base relations.
+ */
+PartitionBoundInfo
+partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype,
+					   List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo 	merged_bounds;
+	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
+						inner_binfo = inner_rel->boundinfo;
+	char				strategy = outer_binfo->strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = NULL;
+
+			break;
+
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partsupfunc,
+														partcollation,
+														outer_rel, inner_rel,
+														outer_parts, inner_parts,
+														jointype);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(outer_rel, inner_rel,
+														 outer_parts, inner_parts,
+														 jointype, partnatts,
+														 partsupfunc,
+														 partcollation);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	return merged_bounds;
+}
+
+/*
+ * partition_get_range_bounds
+ *
+ * Given the index of lower bound in datums array, return lower and upper
+ * bounds and the index of the partition with that lower bound.
+ */
+static int
+partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
+						   PartitionRangeBound *lower,
+						   PartitionRangeBound *upper)
+{
+	int			part_index;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The lower bound should correspond to a valid partition. */
+	part_index = bi->indexes[lb_index + 1];
+	Assert(part_index >= 0);
+
+	lower->kind = bi->kind[lb_index];
+	lower->datums = bi->datums[lb_index];
+	lower->lower = true;
+	upper->kind = bi->kind[lb_index + 1];
+	upper->datums = bi->datums[lb_index + 1];
+	upper->lower = false;
+
+	return part_index;
+}
+
+/*
+ * partition_range_get_next_lb_index
+ *
+ * Given the index of lower bound in datums array return the
+ * index of lower bound of the next partition. When the given index corresponds
+ * to the last partition, return number of datums (ndatums).
+ */
+static int
+partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
+{
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The partition index corresponding to the upper bound should be valid. */
+	Assert(bi->indexes[lb_index + 1] >= 0);
+
+	/*
+	 * If there are no bounds left beyond the upper bound, we have reached the
+	 * last partition.
+	 */
+	if (lb_index + 2 < bi->ndatums)
+	{
+		/*
+		 * If the bound next to the upper bound corresponds to no partition,
+		 * that's the next lower bound of the next partition. Otherwise, the
+		 * current upper bound is the lower bound of the next partition.
+		 */
+		if (bi->indexes[lb_index + 2] < 0)
+			return lb_index + 2;
+		else
+			return lb_index + 1;
+	}
+	else
+		return bi->ndatums;
+}
+
+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *partcollations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2)
+{
+	return partition_rbound_cmp(partnatts, partsupfunc, partcollations,
+								bound1->datums, bound1->kind, bound1->lower,
+								bound2);
+}
+
+/*
+ * partition_range_cmp
+ *
+ * Compare the bounds of two range partitions. Set ub_cmpval <, = or > 0, if the
+ * first partition's upper bound is lower than, equal to or higher than the
+ * second partition's upper bound resp. Similarly set lb_cmpval <, =  or > 0,
+ * if the first partition's lower bound is lower than, equal to or higher than
+ * the second partition's lower bound resp.
+ *
+ * Return true, if the ranges overlap, otherwise return false.
+ */
+static bool
+partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
+					PartitionRangeBound *lower_bound1,
+					PartitionRangeBound *upper_bound1,
+					PartitionRangeBound *lower_bound2,
+					PartitionRangeBound *upper_bound2, int *ub_cmpval,
+					int *lb_cmpval)
+{
+	bool		overlap;
+
+	/*
+	 * Compare upper bound of the first partition with the lower bound of the
+	 * second and vice-versa. If lower bound is higher than the upper bound,
+	 * the partitions are not overlapping. All other cases indicate overlapping
+	 * partitions.
+	 */
+	if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+								  lower_bound1, upper_bound2) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = 1;
+		*lb_cmpval = 1;
+	}
+	else if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+									   lower_bound2, upper_bound1) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = -1;
+		*lb_cmpval = -1;
+	}
+	else
+	{
+		overlap = true;
+		*ub_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, upper_bound1,
+											   upper_bound2);
+		*lb_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, lower_bound1,
+											   lower_bound2);
+	}
+
+	return overlap;
+}
+
+/*
+ * partition_range_merge
+ *
+ * Merge the partition bounds of given two partitions such that the join
+ * between the given two partitions fits merged bounds.
+ *
+ * "merged_upper" will be set to one of the given upper bounds and
+ * "merged_lower" will be set to one of the given lower bounds.
+ */
+static void
+partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
+					  Oid *partcollations, JoinType jointype,
+					  PartitionRangeBound *left_lb,
+					  PartitionRangeBound *left_ub,
+					  PartitionRangeBound *right_lb,
+					  PartitionRangeBound *right_ub,
+					  PartitionRangeBound **merged_lb,
+					  PartitionRangeBound **merged_ub)
+{
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = left_ub;
+			*merged_lb = left_lb;
+			break;
+
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) < 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) > 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		case JOIN_FULL:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) > 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) < 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+
+	return;
+}
+
+/*
+ * Add the lower bound of the next range to the list of bounds, if the lower
+ * bound is higher or equal to the previous upper bound. If successful return
+ * true, otherwise false.
+ */
+static bool
+partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
+							  Oid *partcollations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		prev_ub.datums = llast(*merged_datums);
+		prev_ub.kind = llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  next_lb_datums, next_lb_kind, false,
+									  &prev_ub);
+	}
+
+	/*
+	 * The lower bound is lower than the last upper bound, thus does not fit
+	 * the bounds created so far and hence can not be merged with the existing
+	 * bounds.
+	 */
+	if (cmpval < 0)
+		return false;
+
+	/*
+	 * Add bounds of the new merged partition. If the next lower bound is
+	 * higher than the last upper bound, add new range with index
+	 * corresponding to the lower bound as -1. If the merged lower bound
+	 * is same as the last merged upper bound, the last upper bound will be
+	 * reused as the lower bound of the next range.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, next_lb_datums);
+		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	return true;
+}
+
+/*
+ * handle_missing_partition
+ *
+ * If a range appears in one of the joining relations but not the other, a row
+ * in the corresponding partition will not have any join partner in the other
+ * relation, unless the other relation has a default partition. If a given list
+ * value is present in one joining relation but not the other, the default
+ * partition on the other side may contain that value.
+ *
+ * In both these cases, such an extra partition forms a joining pair with the
+ * default partition, if any,  on the other side.
+ *
+ * If the default partition happens to be on the outer side of the join, the
+ * resultant partition will act as the default partition of the join relation.
+ * Otherwise the resultant partition will be associated with the range.
+ *
+ * When the default partition is not present in the other relation, the rows in
+ * the extra partition will be included in the bounds of the join result, if it
+ * appears on the outer side of the join, since all rows from the outer side
+ * are included in the join result.
+ *
+ * This function handles all these cases.
+ *
+ * maps_with_missing and missing_side_default are the partition maps (See
+ * partition_range/list_bounds_merge() for details) and the index of default
+ * partition respectively corresponding the side with missing partition.
+ *
+ * maps_with_extra and extra_part are the partition maps (See
+ * partition_range/list_bounds_merge() for details) and the index of extra
+ * partition respectively corresponding to the side with the extra partition.
+ *
+ * It returns true if the matching succeeds, otherwise returns false.
+ */
+static bool
+handle_missing_partition(PartitionMap *maps_with_missing,
+						 PartitionMap *maps_with_extra,
+						 int missing_side_default,
+						 int extra_part,
+						 bool missing_side_outer,
+						 bool missing_side_inner,
+						 int *next_index, int *default_index,
+						 int *merged_index)
+{
+	bool missing_has_default = (missing_side_default != -1);
+
+	if (missing_has_default)
+	{
+		*merged_index = map_and_merge_partitions(maps_with_missing,
+												 maps_with_extra,
+												 missing_side_default,
+												 extra_part,
+												 next_index);
+		if (*merged_index < 0)
+			return false;
+
+		if (missing_side_outer)
+		{
+			/*
+			 * Default partition on the outer side forms the default
+			 * partition of the join result.
+			 */
+			if (*default_index < 0)
+				*default_index = *merged_index;
+			else if(*default_index != *merged_index)
+			{
+				/*
+				 * Ended up with default partition on the outer side
+				 * being joined with multiple partitions on the inner
+				 * side. We don't support this case.
+				 */
+				return false;
+			}
+
+			/*
+			 * Since the merged partition acts as a default partition, it
+			 * doesn't need a separate index.
+			 */
+			*merged_index = -1;
+		}
+	}
+	else if (missing_side_inner)
+	{
+		/*
+		 * If this partition has already been mapped (say because we
+		 * found an overlapping range earlier), we know where does it
+		 * fit in the join result. Nothing to do in that case. Else
+		 * create a new merged partition.
+		 */
+		PartitionMap *extra_map = &maps_with_extra[extra_part];
+		if (extra_map->to < 0)
+		{
+			extra_map->to = *next_index;
+			*next_index = *next_index + 1;
+			*merged_index = extra_map->to;
+		}
+	}
+	else
+		*merged_index = -1;
+
+	return true;
+}
+
+static PartitionMap*
+init_partition_map(RelOptInfo *rel)
+{
+	int i, nparts = rel->nparts;
+	PartitionMap *map;
+
+	map = (PartitionMap *) palloc(sizeof(PartitionMap) * nparts);
+
+	for (i = 0; i < nparts; i++)
+	{
+		map[i].from = -1;
+		map[i].to = -1;
+	}
+
+	return map;
+}
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 List **outer_parts, List **inner_parts,
+							 JoinType jointype, int partnatts,
+							 FmgrInfo *partsupfuncs, Oid *partcollations)
+
+{
+	PartitionMap *outer_maps = NULL;
+	PartitionMap *inner_maps = NULL;
+	int			outer_part = 0;
+	int			inner_part = 0;
+	PartitionBoundInfo merged_bounds = NULL;
+	int			outer_lb_index;
+	int			inner_lb_index;
+	int			next_index;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	List	   *merged_kinds = NIL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			inner_default = inner_bi->default_index;
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	outer_maps = init_partition_map(outer_rel);
+	inner_maps = init_partition_map(inner_rel);
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = 0;
+	inner_lb_index = 0;
+	next_index = 0;
+	while (outer_lb_index < outer_bi->ndatums ||
+		   inner_lb_index < inner_bi->ndatums)
+	{
+		PartitionRangeBound outer_lb, outer_ub,
+							inner_lb, inner_ub,
+							*merged_lb = NULL,
+							*merged_ub = NULL;
+
+		int			merged_index = -1;
+		bool		overlap;
+		bool		finished_outer = false;
+		bool		finished_inner = false;
+
+		/* Result of bounds comparison per partition_rbound_cmp(). */
+		int			ub_cmpval;	/* Upper bounds comparison result. */
+		int			lb_cmpval;	/* Lower bounds comparison result. */
+
+		/* Get the range bounds of the next pair of partitions. */
+		if (outer_lb_index < outer_bi->ndatums)
+			outer_part = partition_get_range_bounds(outer_bi, outer_lb_index,
+												&outer_lb, &outer_ub);
+		else
+			finished_outer = true;
+
+		if (inner_lb_index < inner_bi->ndatums)
+			inner_part = partition_get_range_bounds(inner_bi, inner_lb_index,
+												&inner_lb, &inner_ub);
+		else
+			finished_inner = true;
+
+		Assert(!finished_outer || !finished_inner);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (finished_outer)
+		{
+			overlap = false;
+			ub_cmpval = 1;
+			lb_cmpval = 1;
+		}
+		else if (finished_inner)
+		{
+			overlap = false;
+			ub_cmpval = -1;
+			lb_cmpval = -1;
+		}
+		else
+			overlap = partition_range_cmp(partnatts, partsupfuncs, partcollations,
+										  &outer_lb, &outer_ub, &inner_lb,
+										  &inner_ub, &ub_cmpval, &lb_cmpval);
+
+		if (overlap)
+		{
+			/*
+			 * The rows from overlapping portion of ranges on both sides may
+			 * join, hence the corresponding pair of partitions form a joining
+			 * pair. Match them and produce the bounds of the joint partition
+			 * and its index by merging the bounds according to the type of
+			 * join.
+			 */
+			partition_range_merge(partnatts, partsupfuncs, partcollations,
+								  jointype, &outer_lb, &outer_ub, &inner_lb,
+								  &inner_ub, &merged_lb, &merged_ub);
+
+			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+													outer_part, inner_part,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				/* Failed to match the partitions. */
+				return NULL;
+			}
+
+			/*
+			 * If the ranges overlap but don't exactly match, a row from
+			 * non-overlapping portion of the range from one side of join may
+			 * find its join partner in the previous or next overlapping
+			 * partition or default partition on the other side , if such a
+			 * partition exists. All those cases, if true, will cause one
+			 * partition from that side to match at least two partitions on the
+			 * other side; a case that we do not support now. Previous
+			 * partition has been delt with in the previous iteration of this
+			 * loop, next partition will be delt in the next iteration. We will
+			 * deal with the default partition here.
+			 */
+			if ((lb_cmpval < 0 && inner_has_default) ||
+				/* Non-overlapping range on the lower side of outer range. */
+				(lb_cmpval > 0 && outer_has_default) ||
+				/* Non-overlapping range on the lower side of inner range. */
+				(ub_cmpval < 0 && outer_has_default) ||
+				/* Non-overlapping range on the upper side of inner range. */
+				(ub_cmpval > 0 && inner_has_default))
+				/* Non-overlapping range on the upper side of outer range. */
+				return NULL;
+		}
+
+		if (ub_cmpval == 0)
+		{
+			/* Upper bounds of both the ranges match. */
+			Assert(overlap);
+
+			/* Move to the next pair of partitions. */
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the inner side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &outer_lb;
+				merged_ub = &outer_ub;
+
+				/*
+				 * For a FULL join, inner relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the inner relation acts as
+				 * INNER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_inner = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_outer = (jointype == JOIN_FULL);
+				if (!handle_missing_partition(inner_maps,
+											  outer_maps,
+											  inner_default,
+											  outer_part,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the outer side. */
+			Assert(!finished_outer);
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+		}
+		else
+		{
+			Assert(ub_cmpval > 0);
+
+			/* Upper bound of outer range higher than that of the inner. */
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				/* A range missing from the outer side. */
+				bool		missing_side_outer;
+				bool		missing_side_inner;
+
+				merged_lb = &inner_lb;
+				merged_ub = &inner_ub;
+
+				/*
+				 * For a FULL join, outer relation acts as both OUTER and INNER
+				 * relation.  For LEFT and ANTI join the outer relation acts as
+				 * OUTER relation. For INNER and SEMI join OUTER and INNER
+				 * differentiation is immaterial.
+				 */
+				missing_side_outer = (jointype == JOIN_FULL ||
+									  jointype == JOIN_LEFT ||
+									  jointype == JOIN_ANTI);
+				missing_side_inner = (jointype == JOIN_FULL);
+
+				if (!handle_missing_partition(outer_maps,
+											  inner_maps,
+											  outer_default,
+											  inner_part,
+											  missing_side_outer,
+											  missing_side_inner,
+											  &next_index,
+											  &default_index,
+											  &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the inner side. */
+			Assert (!finished_inner);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+
+		if (merged_index < 0)
+		{
+			/* We didn't find a new merged partition. */
+			continue;
+		}
+
+		/*
+		 * We have a valid partition index for the next partition of join. The
+		 * partition should have valid range.
+		 */
+		Assert(merged_lb && merged_ub);
+
+		/* Try merging new lower bound with the last upper bound. */
+		if (!partition_range_merge_next_lb(partnatts, partsupfuncs,
+										   partcollations,
+										   merged_lb->datums,
+										   merged_lb->kind, &merged_datums,
+										   &merged_kinds, &merged_indexes))
+			return NULL;
+
+		/* Add upper bound with the merged partition index. */
+		merged_datums = lappend(merged_datums, merged_ub->datums);
+		merged_kinds = lappend(merged_kinds, merged_ub->kind);
+		merged_indexes = lappend_int(merged_indexes, merged_index);
+	}
+
+	if (!merge_default_partitions(outer_bi, inner_bi,
+										  outer_maps, inner_maps,
+										  jointype, &next_index,
+										  &default_index))
+		return NULL;
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 outer_maps, inner_maps,
+								 outer_nparts, inner_nparts,
+								 next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes,
+													  merged_kinds,
+													  -1, default_index);
+	}
+
+	/* Free any memory we used in this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							List **outer_parts, List **inner_parts,
+							JoinType jointype)
+{
+	PartitionMap *outer_maps = NULL;
+	PartitionMap *inner_maps = NULL;
+	int			cnto;
+	int			cnti;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			next_index = 0;
+	int			null_index = -1;
+	int			default_index = -1;
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
+					   inner_bi = inner_rel->boundinfo;
+	int			      *outer_indexes = outer_bi->indexes;
+	int			      *inner_indexes = inner_bi->indexes;
+	int				   outer_default = outer_bi->default_index;
+	int				   inner_default = inner_bi->default_index;
+	int 			   outer_nparts = outer_rel->nparts,
+					   inner_nparts = inner_rel->nparts;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	outer_maps = init_partition_map(outer_rel);
+	inner_maps = init_partition_map(inner_rel);
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			o_index;
+		int			i_index;
+		int			cmpval;
+		int			merged_index = -1;
+		Datum	   *merged_datum;
+		bool		finished_inner;
+		bool		finished_outer;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+		{
+			finished_outer = true;
+			odatums = NULL;
+			o_index = -1;
+		}
+		else
+		{
+			finished_outer = false;
+			odatums = outer_bi->datums[cnto];
+			o_index = outer_indexes[cnto];
+		}
+
+		if (cnti >= inner_bi->ndatums)
+		{
+			finished_inner = true;
+			idatums = NULL;
+			i_index = -1;
+		}
+		else
+		{
+			finished_inner = false;
+			idatums = inner_bi->datums[cnti];
+			i_index = inner_indexes[cnti];
+		}
+
+		/* If we exhausted both the sides, we won't enter the loop. */
+		Assert(!finished_inner || !finished_outer);
+
+		if (finished_outer)
+			cmpval = 1;
+		else if (finished_inner)
+			cmpval = -1;
+		else
+		{
+			/* Every list datum should map to a valid partition index. */
+			Assert(o_index >= 0 && i_index >= 0 &&
+				   odatums != NULL && idatums != NULL);
+
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0], idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+													o_index, i_index,
+													&next_index);
+
+			if (merged_index < 0)
+				return NULL;
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			/* A datum missing from the inner side. */
+			merged_index = -1;
+			merged_datum = odatums;
+
+			/*
+			 * For a FULL join, inner relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the inner relation acts as
+			 * INNER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_inner = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_outer = (jointype == JOIN_FULL);
+
+			if (!handle_missing_partition(inner_maps,
+										  outer_maps,
+										  inner_default,
+										  o_index,
+										  missing_side_outer,
+										  missing_side_inner,
+										  &next_index,
+										  &default_index,
+										  &merged_index))
+				return NULL;
+
+			/* Move to the next datum on the outer side. */
+			Assert(!finished_outer);
+			cnto++;
+		}
+		else
+		{
+			bool		missing_side_outer;
+			bool		missing_side_inner;
+
+			Assert(cmpval > 0);
+
+			/* A datum missing from the outer side. */
+			merged_index = -1;
+			merged_datum = idatums;
+
+			/*
+			 * For a FULL join, outer relation acts as both OUTER and INNER
+			 * relation.  For LEFT and ANTI join the outer relation acts as
+			 * OUTER relation. For INNER and SEMI join OUTER and INNER
+			 * differentiation is immaterial.
+			 */
+			missing_side_outer = (jointype == JOIN_FULL ||
+								  jointype == JOIN_LEFT ||
+								  jointype == JOIN_ANTI);
+			missing_side_inner = (jointype == JOIN_FULL);
+
+			if (!handle_missing_partition(outer_maps,
+										  inner_maps,
+										  outer_default,
+										  i_index,
+										  missing_side_outer,
+										  missing_side_inner,
+										  &next_index,
+										  &default_index,
+										  &merged_index))
+				return NULL;
+
+			/* Move to the next datum on the right side. */
+			Assert(!finished_inner);
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	if (!merge_null_partitions(outer_bi, inner_bi,
+							   outer_maps, inner_maps,
+							   jointype, &next_index, &null_index,
+							   &default_index))
+		return NULL;
+
+	if (!merge_default_partitions(outer_bi, inner_bi,
+								  outer_maps, inner_maps,
+								  jointype, &next_index,
+								  &default_index))
+		return NULL;
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 outer_maps, inner_maps,
+								 outer_nparts, inner_nparts,
+								 next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes, NIL,
+													  null_index, default_index);
+	}
+
+	/* Free up all extra memory before returning from this function. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+
+	return merged_bounds;
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * If the two given partitions (given by index1 and index2 resp.) are
+ * already mapped to each other return the index of corresponding partition in
+ * the merged set of partitions.  If they do not have a merged partition
+ * associated with them, assign a new merged partition index.  If the
+ * partitions are already mapped and their mapped partitions are different from
+ * each other, they can not be merged, so return -1.
+ *
+ * partmaps1[i] gives the mapping of partitions for both relations. It
+ * describes which partition of relation 2 matches ith partition of relation 1,
+ * and which partition in the merged set matches ith partition of relation 1
+ * maps to. Similarly for partmap2.
+ *
+ * index1 and index2 are the indexes of matching partition from respective
+ * relations.
+ *
+ * *next_index is used and incremented when the given partitions require a new
+ * merged partition.
+ */
+
+static int
+map_and_merge_partitions(PartitionMap *partmaps1, PartitionMap *partmaps2,
+						 int index1, int index2, int *next_index)
+{
+	PartitionMap 	*partmap1 = &partmaps1[index1];
+	PartitionMap 	*partmap2 = &partmaps2[index2];
+	int				merged_index;
+
+	/*
+	 * If both the partitions are not mapped to each other, update the
+	 * maps.
+	 */
+	if (partmap1->from < 0 && partmap2->from < 0)
+	{
+		partmap1->from = index2;
+		partmap2->from = index1;
+	}
+
+	/*
+	 * If the given to partitions map to each other, find the corresponding
+	 * merged partition index .
+	 */
+	if (partmap1->from == index2 && partmap2->from == index1)
+	{
+		/*
+		 * If both the partitions are mapped to the same merged partition, get
+		 * the index of merged partition.
+		 */
+		if (partmap1->to == partmap2->to)
+		{
+			merged_index = partmap1->to;
+
+			/*
+			 * If the given two partitions do not have a merged partition
+			 * associated with them, allocate a new merged partition.
+			 */
+			if (merged_index < 0)
+			{
+				merged_index = *next_index;
+				*next_index = *next_index + 1;
+				partmap1->to = merged_index;
+				partmap2->to = merged_index;
+			}
+		}
+
+		/*
+		 * If partition from one relation was mapped to a merged partition but
+		 * not the partition from the other relation, map the same merged
+		 * partition to the partition from other relation, since matching
+		 * partitions map to the same merged partition.
+		 */
+		else if (partmap1->to >= 0 && partmap2->to < 0)
+		{
+			partmap2->to = partmap1->to;
+			merged_index = partmap1->to;
+		}
+		else if (partmap1->to < 0 && partmap2->to >= 0)
+		{
+			partmap1->to = partmap2->to;
+			merged_index = partmap2->to;
+		}
+		else
+		{
+			Assert(partmap1->to != partmap2->to &&
+				   partmap1->to >= 0 && partmap2->to >= 0);
+
+			/*
+			 * Both the partitions map to different merged partitions. This
+			 * means that multiple partitions from one relation matches to one
+			 * partition from the other relation. Partition-wise join does not
+			 * handle this case right now, since it requires ganging multiple
+			 * partitions together (into one RelOptInfo).
+			 */
+			merged_index = -1;
+		}
+	}
+	else
+	{
+		/*
+		 * Multiple partitions from one relation map to one partition from the
+		 * other relation. Partition-wise join does not handle this case right
+		 * now, since it requires ganging multiple partitions together (into
+		 * one RelOptInfo).
+		 */
+		merged_index = -1;
+	}
+
+	return merged_index;
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * partmaps1 map each partition from either side of the join to a merged
+ * partition resp. E.g. partmaps1[i].to gives the merged partition to which ith
+ * partition of first relation maps. Similarly for partmap2. If
+ * partmaps1[i].to == partmaps2[j].to, i and j form the matching pair of
+ * partitions.
+ *
+ * Given these maps this function produces the list pairs of partitions which
+ * when joined produce the merged partitions in the order of merged partition
+ * indexes.
+ *
+ * nparts1 and nparts2 are the number of partitions of the joining relations
+ * resp.
+ *
+ * nparts is the number of merged partitions.
+ *
+ * If successful, the pairs of partitions are returned as two separate lists,
+ * parts1 and parts2 resp., one for each side. Otherwise, those lists will be
+ * set to NIL.
+ */
+static void
+generate_matching_part_pairs(RelOptInfo *rel1, RelOptInfo *rel2,
+							 PartitionMap *partmaps1, PartitionMap *partmaps2,
+							 int nparts1, int nparts2, int nparts,
+							 List **matched_parts1, List **matched_parts2)
+{
+	int		   *matching1,
+			   *matching2;
+	int 		i;
+	int			max_nparts;
+
+	*matched_parts1 = NIL;
+	*matched_parts2 = NIL;
+
+	matching1 = (int *) palloc(sizeof(int) * nparts),
+	matching2 = (int *) palloc(sizeof(int) * nparts);
+	for (i = 0; i < nparts; i++)
+		matching1[i] = matching2[i] = -1;
+
+	/* Set pairs of matching partitions. */
+	max_nparts = Max(nparts1, nparts2);
+	for (i = 0; i < max_nparts; i++)
+	{
+		if (i < nparts1)
+		{
+			PartitionMap outer_map = partmaps1[i];
+
+			if (outer_map.to >= 0)
+			{
+				Assert(outer_map.to < nparts);
+				matching1[outer_map.to] = i;
+			}
+		}
+
+		if (i < nparts2)
+		{
+			PartitionMap inner_map = partmaps2[i];
+
+			if (inner_map.to >= 0)
+			{
+				Assert(inner_map.to < nparts);
+				matching2[inner_map.to] = i;
+			}
+		}
+	}
+
+	for (i = 0; i < nparts; i++)
+	{
+		int			part1 = matching1[i];
+		int			part2 = matching2[i];
+
+		/* At least one of the partitions should exist. */
+		Assert(part1 >= 0 || part2 >= 0);
+
+		*matched_parts1 = lappend(*matched_parts1,
+								  part1 >= 0 ? rel1->part_rels[part1] : NULL);
+		*matched_parts2 = lappend(*matched_parts2,
+								  part2 >= 0 ? rel2->part_rels[part2] : NULL);
+	}
+
+	pfree(matching1);
+	pfree(matching2);
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * Merge default partitions from both sides, if any, and assign the default
+ * partition for the join result, if necessary.
+ *
+ * If both the relations have default partitions, try mapping those to each
+ * other. If the mapping succeeds corresponding merged partition will act as
+ * the default partition of the join result.
+ *
+ * If inner side of the join has default but not the outer side, rows in it
+ * won't appear in the join result. So don't create a default partition. If
+ * outer side of the join has default but not the inner side, rows in it will
+ * appear in the join result, so create a default merged partition.
+ */
+static bool
+merge_default_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+						 PartitionMap *outer_maps, PartitionMap *inner_maps,
+						 JoinType jointype, int *next_index, int *default_index)
+{
+	int				outer_default = outer_bi->default_index;
+	int				inner_default = inner_bi->default_index;
+	bool			outer_has_default = partition_bound_has_default(outer_bi);
+	bool			inner_has_default = partition_bound_has_default(inner_bi);
+	bool			merged = true;
+	PartitionMap 	*outer_default_map = NULL;
+	PartitionMap 	*inner_default_map = NULL;
+
+	if (outer_has_default)
+		outer_default_map = &outer_maps[outer_default];
+
+	if (inner_has_default)
+		inner_default_map = &inner_maps[inner_default];
+
+	if (!outer_has_default && !inner_has_default)
+		Assert(*default_index < 0);
+	else if (outer_default_map != NULL && inner_default_map == NULL)
+	{
+		if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+			jointype == JOIN_ANTI)
+		{
+			if (outer_default_map->to < 0)
+			{
+				outer_default_map->to = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = outer_default_map->to;
+			}
+			else
+				Assert(*default_index == outer_default_map->to);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else if (outer_default_map == NULL && inner_default_map != NULL)
+	{
+		if (jointype == JOIN_FULL)
+		{
+			if (inner_default_map->to < 0)
+			{
+				inner_default_map->to = *next_index;
+				*next_index = *next_index + 1;
+				Assert(*default_index < 0);
+				*default_index = inner_default_map->to;
+			}
+			else
+				Assert(*default_index == inner_default_map->to);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		*default_index = map_and_merge_partitions(outer_maps, inner_maps,
+												  outer_default, inner_default,
+												  next_index);
+
+		if (*default_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
+
+/*
+ * merge_null_partitions
+ *
+ * Merge NULL partitions, i.e. a partition that can hold NULL values for a list
+ * partitioned table, if any. Find the index of merged partition to which the
+ * NULL values would belong in the join result. If one joining relation has a
+ * NULL partition but not the other, try matching it with the default partition
+ * from the other relation since the default partition may have rows with NULL
+ * partition key. We can eliminate a NULL partition when it appears only on the
+ * inner side of the join and the outer side doesn't have a default partition.
+ *
+ * When the equality operator used for join is strict, two NULL values will not
+ * be considered as equal, and thus a NULL partition can be eliminated for an
+ * inner join. But we don't check the strictness operator here.
+ */
+static bool
+merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_maps, PartitionMap *inner_maps,
+					  JoinType jointype, int *next_index,
+					  int *null_index, int *default_index)
+{
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	int			outer_ni = outer_bi->null_index;
+	int			inner_ni = inner_bi->null_index;
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	bool		merged = true;
+
+	if (!outer_has_null && !inner_has_null)
+		Assert(*null_index < 0);
+	else if (outer_has_null && !inner_has_null)
+	{
+		int			merged_index = -1;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, inner relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the inner relation acts as
+		 * INNER relation. For INNER and SEMI join OUTER and INNER
+		 * differentiation is immaterial.
+		 */
+		missing_side_inner = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_outer = (jointype == JOIN_FULL);
+
+		merged = handle_missing_partition(inner_maps,
+										  outer_maps,
+										  inner_default,
+										  outer_ni,
+										  missing_side_outer,
+										  missing_side_inner, next_index,
+										  default_index, &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join to which the outer null partition maps
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = outer_maps[outer_ni].to;
+	}
+	else if (!outer_has_null && inner_has_null)
+	{
+		int			merged_index = -1;
+		bool		missing_side_outer;
+		bool		missing_side_inner;
+
+		/*
+		 * For a FULL join, outer relation acts as both OUTER and INNER
+		 * relation.  For LEFT and ANTI join the outer relation acts as OUTER
+		 * relation. For INNER and SEMI join OUTER and INNER differentiation is
+		 * immaterial.
+		 */
+		missing_side_outer = (jointype == JOIN_FULL ||
+							  jointype == JOIN_LEFT ||
+							  jointype == JOIN_ANTI);
+		missing_side_inner = (jointype == JOIN_FULL);
+		merged = handle_missing_partition(outer_maps,
+										  inner_maps,
+										  outer_default,
+										  inner_ni,
+										  missing_side_outer,
+										  missing_side_inner,
+										  next_index, default_index,
+										  &merged_index);
+		*null_index = merged_index;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join, to which the outer side null partition maps,
+		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the join.
+		 */
+		if (missing_side_inner)
+			*null_index = inner_maps[inner_ni].to;
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		*null_index = map_and_merge_partitions(outer_maps,
+											   inner_maps,
+											   outer_ni,
+											   inner_ni,
+											   next_index);
+		if (*null_index < 0)
+			merged = false;
+	}
+
+	return merged;
+}
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index 23a06d718e..e06eb9aaae 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -577,8 +577,10 @@ typedef struct PartitionSchemeData *PartitionScheme;
  *		part_scheme - Partitioning scheme of the relation
  *		nparts - Number of partitions
  *		boundinfo - Partition bounds
+ *		merged - true if partition bounds are merged ones
  *		partition_qual - Partition constraint if not the root
  *		part_rels - RelOptInfos for each partition
+ *		all_partrels - Relids set of all partition relids
  *		partexprs, nullable_partexprs - Partition key expressions
  *		partitioned_child_rels - RT indexes of unpruned partitions of
  *								 this relation that are partitioned tables
@@ -718,9 +720,12 @@ typedef struct RelOptInfo
 	PartitionScheme part_scheme;	/* Partitioning scheme. */
 	int			nparts;			/* number of partitions */
 	struct PartitionBoundInfoData *boundinfo;	/* Partition bounds */
+	bool		merged;			/* true if partition bounds were created by
+								 * partition_bounds_merge() */
 	List	   *partition_qual; /* partition constraint */
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of partitions,
 									 * stored in the same order of bounds */
+	Relids		all_partrels;	/* Relids set of all partition relids */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 	List	   *partitioned_child_rels; /* List of RT indexes. */
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index 0d0fd42b18..9292aa11e5 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -16,6 +16,7 @@
 #include "nodes/pg_list.h"
 #include "partitioning/partdefs.h"
 #include "utils/relcache.h"
+struct RelOptInfo;				/* avoid including pathnodes.h here */
 
 
 /*
@@ -108,5 +109,11 @@ extern int	partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 										  int nvalues, Datum *values, bool *is_equal);
 extern int	partition_hash_bsearch(PartitionBoundInfo boundinfo,
 								   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   struct RelOptInfo *outer_rel, struct RelOptInfo *inner_rel,
+					   JoinType jointype,
+					   List **outer_parts, List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index cad8dd591a..b5b5c8a260 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -8,59 +8,86 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-(4 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+(7 rows)
 
 -- left outer join, with whole-row reference; partitionwise join does not apply
 EXPLAIN (COSTS OFF)
@@ -72,35 +99,50 @@ SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER
    ->  Hash Right Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(22 rows)
 
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-      t1      |      t2      
---------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
- (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
- (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
- (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
- (550,0,0550) | 
-(12 rows)
+       t1       |       t2       
+----------------+----------------
+ (-250,0,-0250) | 
+ (-200,0,-0200) | 
+ (-150,0,-0150) | (0,-150,-0150)
+ (-100,0,-0100) | 
+ (-50,0,-0050)  | 
+ (0,0,0000)     | (0,0,0000)
+ (50,0,0050)    | 
+ (100,0,0100)   | 
+ (150,0,0150)   | (0,150,0150)
+ (200,0,0200)   | 
+ (250,0,0250)   | 
+ (300,0,0300)   | (0,300,0300)
+ (350,0,0350)   | 
+ (400,0,0400)   | 
+ (450,0,0450)   | (0,450,0450)
+ (500,0,0500)   | 
+ (550,0,0550)   | 
+ (600,0,0600)   | (0,600,0600)
+ (650,0,0650)   | 
+ (700,0,0700)   | 
+ (750,0,0750)   | (0,750,0750)
+(21 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
@@ -112,35 +154,53 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
    ->  Append
          ->  Hash Right Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Right Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t2_2.b)
-(20 rows)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+      |       | -225 | -0225
+      |       |  -75 | -0075
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(14 rows)
 
 -- full outer join, with placeholder vars
 EXPLAIN (COSTS OFF)
@@ -148,8 +208,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                             QUERY PLAN                            
 ------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               Filter: (((50) = prt1_p0.a) OR ((75) = prt2_p0.b))
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
@@ -174,7 +242,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               Filter: (((50) = prt1_p4.a) OR ((75) = prt2_p4.b))
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
@@ -211,35 +287,44 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Hash Right Join
-         Hash Cond: (prt2_p2.b = prt1_p1.a)
+         Hash Cond: (prt2_p2.b = prt1_p0.a)
          ->  Append
                ->  Seq Scan on prt2_p2
                      Filter: (b > 250)
                ->  Seq Scan on prt2_p3
                      Filter: (b > 250)
+               ->  Seq Scan on prt2_p4
+                     Filter: (b > 250)
          ->  Hash
                ->  Append
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p1
                            Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p2
                            Filter: ((a < 450) AND (b = 0))
-(15 rows)
+(19 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-(9 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+(14 rows)
 
 -- Currently we can't do partitioned join if nullable-side partitions are pruned
 EXPLAIN (COSTS OFF)
@@ -247,11 +332,13 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                       QUERY PLAN                      
 ------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Hash Full Join
-         Hash Cond: (prt1_p1.a = prt2_p2.b)
-         Filter: ((prt1_p1.b = 0) OR (prt2_p2.a = 0))
+         Hash Cond: (prt1_p0.a = prt2_p2.b)
+         Filter: ((prt1_p0.b = 0) OR (prt2_p2.a = 0))
          ->  Append
+               ->  Seq Scan on prt1_p0
+                     Filter: (a < 450)
                ->  Seq Scan on prt1_p1
                      Filter: (a < 450)
                ->  Seq Scan on prt1_p2
@@ -262,64 +349,147 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                            Filter: (b > 250)
                      ->  Seq Scan on prt2_p3
                            Filter: (b > 250)
-(16 rows)
+                     ->  Seq Scan on prt2_p4
+                           Filter: (b > 250)
+(20 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
-(12 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+      |       | 375 | 0375
+      |       | 450 | 0450
+      |       | 525 | 0525
+      |       | 600 | 0600
+      |       | 675 | 0675
+      |       | 750 | 0750
+(20 rows)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Semi Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Semi Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                            Filter: (a = 0)
-         ->  Nested Loop Semi Join
-               Join Filter: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
-               ->  Materialize
-                     ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
-(24 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt1_p3 t2_3
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+                           Filter: (b = 0)
+(37 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
 
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
@@ -330,27 +500,82 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
    ->  Append
          ->  Hash Anti Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Hash Anti Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Hash Anti Join
                Hash Cond: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
-(17 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(27 rows)
 
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
-  sum  |         avg          | sum  |         avg         
--------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+  sum  |         avg          | sum  |        avg         
+-------+----------------------+------+--------------------
+ 95550 | 273.0000000000000000 | 2200 | 6.2857142857142857
 (1 row)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Append
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p0 t1
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+               Index Cond: (a = t1.b)
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.b = t2_1.a)
+         ->  Seq Scan on prt2_p1 t1_1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p1 t2_1
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p2 t1_2
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+               Index Cond: (a = t1_2.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p3 t1_3
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+               Index Cond: (a = t1_3.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p4 t1_4
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+               Index Cond: (a = t1_4.b)
+(27 rows)
+
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+  b   |   c   
+------+-------
+ -225 | -0225
+  -75 | -0075
+   75 | 0075
+  225 | 0225
+  375 | 0375
+  525 | 0525
+  675 | 0675
+(7 rows)
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -362,49 +587,74 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
+                     ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
                            Index Cond: (a = t1.a)
-                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t3
                            Index Cond: (b = t2.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_1
+                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
                            Index Cond: (a = t1_1.a)
-                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3_1
                            Index Cond: (b = t2_1.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_2
+                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
                            Index Cond: (a = t1_2.a)
-                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_2
                            Index Cond: (b = t2_2.a)
-(27 rows)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                           Index Cond: (a = t1_3.a)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_3
+                           Index Cond: (b = t2_3.a)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                           Index Cond: (a = t1_4.a)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t3_4
+                           Index Cond: (b = t2_4.a)
+(43 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   | t2a | t3a | least 
------+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
- 100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
- 250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
- 400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
- 550 | 0 | 0550 |     |     |      
-(12 rows)
+  a   | b |   c   | t2a  | t3a | least 
+------+---+-------+------+-----+-------
+ -250 | 0 | -0250 |      |     |      
+ -200 | 0 | -0200 |      |     |      
+ -150 | 0 | -0150 | -150 |   0 |  -150
+ -100 | 0 | -0100 |      |     |      
+  -50 | 0 | -0050 |      |     |      
+    0 | 0 | 0000  |    0 |   0 |     0
+   50 | 0 | 0050  |      |     |      
+  100 | 0 | 0100  |      |     |      
+  150 | 0 | 0150  |  150 |   0 |   150
+  200 | 0 | 0200  |      |     |      
+  250 | 0 | 0250  |      |     |      
+  300 | 0 | 0300  |  300 |   0 |   300
+  350 | 0 | 0350  |      |     |      
+  400 | 0 | 0400  |      |     |      
+  450 | 0 | 0450  |  450 |   0 |   450
+  500 | 0 | 0500  |      |     |      
+  550 | 0 | 0550  |      |     |      
+  600 | 0 | 0600  |  600 |   0 |   600
+  650 | 0 | 0650  |      |     |      
+  700 | 0 | 0700  |      |     |      
+  750 | 0 | 0750  |  750 |   0 |   750
+(21 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
@@ -418,64 +668,95 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
          Hash Cond: ((t1.c)::text = (t2.c)::text)
          Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Seq Scan on prt1_p1 t1
-               ->  Seq Scan on prt1_p2 t1_1
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
          ->  Hash
                ->  Append
                      ->  Hash Join
                            Hash Cond: (t2.a = t3.b)
-                           ->  Seq Scan on prt1_p1 t2
+                           ->  Seq Scan on prt1_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t3
+                                 ->  Seq Scan on prt2_p0 t3
                      ->  Hash Join
                            Hash Cond: (t2_1.a = t3_1.b)
-                           ->  Seq Scan on prt1_p2 t2_1
+                           ->  Seq Scan on prt1_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t3_1
+                                 ->  Seq Scan on prt2_p1 t3_1
                      ->  Hash Join
                            Hash Cond: (t2_2.a = t3_2.b)
-                           ->  Seq Scan on prt1_p3 t2_2
+                           ->  Seq Scan on prt1_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p3 t3_2
-(26 rows)
+                                 ->  Seq Scan on prt2_p2 t3_2
+                     ->  Hash Join
+                           Hash Cond: (t2_3.a = t3_3.b)
+                           ->  Seq Scan on prt1_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t3_3
+                     ->  Hash Join
+                           Hash Cond: (t2_4.a = t3_4.b)
+                           ->  Seq Scan on prt1_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t3_4
+(38 rows)
 
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, t2.b t2b, t2.c t2c, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
-  a  | t2a | t2c  
------+-----+------
-   0 |   0 | 0000
-  50 |     | 
- 100 |     | 
- 150 | 150 | 0150
- 200 |     | 
- 250 |     | 
- 300 | 300 | 0300
- 350 |     | 
- 400 |     | 
- 450 | 450 | 0450
- 500 |     | 
- 550 |     | 
-(12 rows)
+  a   | t2a  |  t2c  
+------+------+-------
+ -250 |      | 
+ -200 |      | 
+ -150 | -150 | -0150
+ -100 |      | 
+  -50 |      | 
+    0 |    0 | 0000
+   50 |      | 
+  100 |      | 
+  150 |  150 | 0150
+  200 |      | 
+  250 |      | 
+  300 |  300 | 0300
+  350 |      | 
+  400 |      | 
+  450 |  450 | 0450
+  500 |      | 
+  550 |      | 
+  600 |  600 | 0600
+  650 |      | 
+  700 |      | 
+  750 |  750 | 0750
+(21 rows)
 
 --
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
@@ -486,32 +767,49 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
    ->  Append
          ->  Hash Join
                Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
-               ->  Seq Scan on prt2_e_p1 t2
+               ->  Seq Scan on prt2_e_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 t1
+                     ->  Seq Scan on prt1_e_p0 t1
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
-               ->  Seq Scan on prt2_e_p2 t2_1
+               Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+               ->  Seq Scan on prt1_e_p1 t1_1
+                     Filter: (c = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 t1_1
-                           Filter: (c = 0)
+                     ->  Seq Scan on prt2_e_p1 t2_1
          ->  Hash Join
                Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
-               ->  Seq Scan on prt2_e_p3 t2_2
+               ->  Seq Scan on prt2_e_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 t1_2
+                     ->  Seq Scan on prt1_e_p2 t1_2
                            Filter: (c = 0)
-(21 rows)
+         ->  Hash Join
+               Hash Cond: (((t2_3.b + t2_3.a) / 2) = ((t1_3.a + t1_3.b) / 2))
+               ->  Seq Scan on prt2_e_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p3 t1_3
+                           Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (((t2_4.b + t2_4.a) / 2) = ((t1_4.a + t1_4.b) / 2))
+               ->  Seq Scan on prt2_e_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4 t1_4
+                           Filter: (c = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
- 150 | 0 | 150 | 0
- 300 | 0 | 300 | 0
- 450 | 0 | 450 | 0
-(4 rows)
+  a   | c |  b   | c 
+------+---+------+---
+ -250 | 0 | -250 | 0
+ -100 | 0 | -100 | 0
+    0 | 0 |    0 | 0
+    0 | 0 |    0 | 0
+  150 | 0 |  150 | 0
+  300 | 0 |  300 | 0
+  450 | 0 |  450 | 0
+  600 | 0 |  600 | 0
+  750 | 0 |  750 | 0
+(9 rows)
 
 --
 -- N-way join
@@ -524,154 +822,232 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = ((t3.a + t3.b) / 2))
+               Join Filter: (t1.a = t2.b)
                ->  Hash Join
-                     Hash Cond: (t2.b = t1.a)
-                     ->  Seq Scan on prt2_p1 t2
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on prt1_e_p0 t3
                      ->  Hash
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                     Index Cond: (((a + b) / 2) = t2.b)
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
+                     Index Cond: (b = ((t3.a + t3.b) / 2))
          ->  Nested Loop
                Join Filter: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_1.b = t1_1.a)
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
+               ->  Index Scan using iprt1_e_p4_ab2 on prt1_e_p1 t3_1
                      Index Cond: (((a + b) / 2) = t2_1.b)
          ->  Nested Loop
                Join Filter: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_2.b = t1_2.a)
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
+               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_2
                      Index Cond: (((a + b) / 2) = t2_2.b)
-(33 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = ((t3_3.a + t3_3.b) / 2))
+               ->  Nested Loop
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t2_3.b)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t2_4.b)
+               ->  Hash Join
+                     Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                     ->  Seq Scan on prt1_e_p4 t3_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_p4 t1_4
+                                 Filter: (b = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = ((t3_4.a + t3_4.b) / 2))
+(52 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
-(4 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(8 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                          QUERY PLAN                          
---------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Hash Right Join
                Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
-               ->  Seq Scan on prt1_e_p1 t3
+               ->  Seq Scan on prt1_e_p0 t3
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2.b = t1.a)
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_1.a)
-               ->  Seq Scan on prt1_e_p2 t3_1
+               ->  Seq Scan on prt1_e_p1 t3_1
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_1.b = t1_1.a)
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_2.a)
-               ->  Seq Scan on prt1_e_p3 t3_2
+               ->  Seq Scan on prt1_e_p2 t3_2
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_2.b = t1_2.a)
-                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                                        Filter: (b = 0)
-(33 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+               ->  Seq Scan on prt1_e_p4 t3_4
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_4.b = t1_4.a)
+                           ->  Seq Scan on prt2_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_p4 t1_4
+                                       Filter: (b = 0)
+(51 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                            QUERY PLAN                             
--------------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p1 t3
+                           ->  Seq Scan on prt1_e_p0 t3
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p1_b on prt2_p1 t2
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
                      Index Cond: (b = t1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p2 t3_1
+                           ->  Seq Scan on prt1_e_p1 t3_1
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
+               ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
                      Index Cond: (b = t1_1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p3 t3_2
+                           ->  Seq Scan on prt1_e_p2 t3_2
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
                      Index Cond: (b = t1_2.a)
-(30 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_e_p3 t3_3
+                           Filter: (c = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                           Index Cond: (a = ((t3_3.a + t3_3.b) / 2))
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Nested Loop Left Join
+               ->  Hash Right Join
+                     Hash Cond: (t1_4.a = ((t3_4.a + t3_4.b) / 2))
+                     ->  Seq Scan on prt1_p4 t1_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p4 t3_4
+                                 Filter: (c = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = t1_4.a)
+(47 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -680,21 +1056,34 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                                                    QUERY PLAN                                                   
 ----------------------------------------------------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   Sort Key: prt1_p0.a, prt2_p0.b, ((prt1_e_p0.a + prt1_e_p0.b))
    ->  Append
          ->  Hash Full Join
-               Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
-               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               Hash Cond: (prt1_p0.a = ((prt1_e_p0.a + prt1_e_p0.b) / 2))
+               Filter: ((prt1_p0.a = (50)) OR (prt2_p0.b = (75)) OR (((prt1_e_p0.a + prt1_e_p0.b) / 2) = (50)))
                ->  Hash Full Join
-                     Hash Cond: (prt1_p1.a = prt2_p1.b)
-                     ->  Seq Scan on prt1_p1
+                     Hash Cond: (prt1_p0.a = prt2_p0.b)
+                     ->  Seq Scan on prt1_p0
                            Filter: (b = 0)
                      ->  Hash
-                           ->  Seq Scan on prt2_p1
+                           ->  Seq Scan on prt2_p0
                                  Filter: (a = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1
+                     ->  Seq Scan on prt1_e_p0
                            Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = prt1_p1.a)
+               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               ->  Seq Scan on prt1_e_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on prt1_p1
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1
+                                       Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
                Filter: ((prt1_p2.a = (50)) OR (prt2_p2.b = (75)) OR (((prt1_e_p2.a + prt1_e_p2.b) / 2) = (50)))
@@ -721,7 +1110,20 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                ->  Hash
                      ->  Seq Scan on prt1_e_p3
                            Filter: (c = 0)
-(42 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = ((prt1_e_p4.a + prt1_e_p4.b) / 2))
+               Filter: ((prt1_p4.a = (50)) OR (prt2_p4.b = (75)) OR (((prt1_e_p4.a + prt1_e_p4.b) / 2) = (50)))
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = prt2_p4.b)
+                     ->  Seq Scan on prt1_p4
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4
+                                 Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4
+                           Filter: (c = 0)
+(68 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
@@ -739,172 +1141,260 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = t1_3.b)
+               Join Filter: (t1.a = t1_5.b)
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Join
-                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
-                           ->  Seq Scan on prt1_e_p1 t2
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t1_3
+                                 ->  Seq Scan on prt2_p0 t1_5
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
                      Index Cond: (a = ((t2.a + t2.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_1.a = t1_4.b)
+               Join Filter: (t1_1.a = t1_6.b)
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Join
-                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
-                           ->  Seq Scan on prt1_e_p2 t2_1
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_6.b)
+                           ->  Seq Scan on prt1_e_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t1_4
+                                 ->  Seq Scan on prt2_p1 t1_6
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
                      Index Cond: (a = ((t2_1.a + t2_1.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_2.a = t1_5.b)
+               Join Filter: (t1_2.a = t1_7.b)
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
+                           ->  Seq Scan on prt2_p2 t1_7
                                  Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
+                           ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t2_2
+                                 Index Cond: (((a + b) / 2) = t1_7.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t1_8.b)
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Nested Loop
+                           ->  Seq Scan on prt2_p3 t1_8
+                                 Filter: (a = 0)
+                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_3
+                                 Index Cond: (((a + b) / 2) = t1_8.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = ((t2_3.a + t2_3.b) / 2))
+                     Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t1_9.b)
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Join
+                           Hash Cond: (((t2_4.a + t2_4.b) / 2) = t1_9.b)
+                           ->  Seq Scan on prt1_e_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t1_9
+                                       Filter: (a = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = ((t2_4.a + t2_4.b) / 2))
+                     Filter: (b = 0)
+(66 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                               QUERY PLAN                                
--------------------------------------------------------------------------
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
-                           ->  Seq Scan on prt2_p1 t1_3
+                           Hash Cond: (t1_5.b = ((t1_10.a + t1_10.b) / 2))
+                           ->  Seq Scan on prt2_p0 t1_5
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t1_6
+                                 ->  Seq Scan on prt1_e_p0 t1_10
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
-                     Index Cond: (a = t1_3.b)
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t1_5.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
-                           ->  Seq Scan on prt2_p2 t1_4
+                           Hash Cond: (t1_6.b = ((t1_11.a + t1_11.b) / 2))
+                           ->  Seq Scan on prt2_p1 t1_6
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t1_7
+                                 ->  Seq Scan on prt1_e_p1 t1_11
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
-                     Index Cond: (a = t1_4.b)
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
+                     Index Cond: (a = t1_6.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
-                           ->  Seq Scan on prt2_p3 t1_5
+                           Hash Cond: (t1_7.b = ((t1_12.a + t1_12.b) / 2))
+                           ->  Seq Scan on prt2_p2 t1_7
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p3 t1_8
+                                 ->  Seq Scan on prt1_e_p2 t1_12
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t1_5.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t1_7.b)
                      Filter: (b = 0)
-(39 rows)
-
-SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_8.b = ((t1_13.a + t1_13.b) / 2))
+                           ->  Seq Scan on prt2_p3 t1_8
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p3 t1_13
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t1_8.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_9.b = ((t1_14.a + t1_14.b) / 2))
+                           ->  Seq Scan on prt2_p4 t1_9
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t1_14
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = t1_9.b)
+                     Filter: (b = 0)
+(63 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 -- test merge joins
 SET enable_hashjoin TO off;
 SET enable_nestloop TO off;
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                           QUERY PLAN                           
-----------------------------------------------------------------
+                            QUERY PLAN                            
+------------------------------------------------------------------
  Merge Append
    Sort Key: t1.a
    ->  Merge Semi Join
-         Merge Cond: (t1.a = t1_3.b)
+         Merge Cond: (t1.a = t1_5.b)
          ->  Sort
                Sort Key: t1.a
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+               Merge Cond: (t1_5.b = (((t1_10.a + t1_10.b) / 2)))
                ->  Sort
-                     Sort Key: t1_3.b
-                     ->  Seq Scan on prt2_p1 t1_3
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on prt2_p0 t1_5
                ->  Sort
-                     Sort Key: (((t1_6.a + t1_6.b) / 2))
-                     ->  Seq Scan on prt1_e_p1 t1_6
+                     Sort Key: (((t1_10.a + t1_10.b) / 2))
+                     ->  Seq Scan on prt1_e_p0 t1_10
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_1.a = t1_4.b)
+         Merge Cond: (t1_1.a = t1_6.b)
          ->  Sort
                Sort Key: t1_1.a
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+               Merge Cond: (t1_6.b = (((t1_11.a + t1_11.b) / 2)))
                ->  Sort
-                     Sort Key: t1_4.b
-                     ->  Seq Scan on prt2_p2 t1_4
+                     Sort Key: t1_6.b
+                     ->  Seq Scan on prt2_p1 t1_6
                ->  Sort
-                     Sort Key: (((t1_7.a + t1_7.b) / 2))
-                     ->  Seq Scan on prt1_e_p2 t1_7
+                     Sort Key: (((t1_11.a + t1_11.b) / 2))
+                     ->  Seq Scan on prt1_e_p1 t1_11
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_2.a = t1_5.b)
+         Merge Cond: (t1_2.a = t1_7.b)
          ->  Sort
                Sort Key: t1_2.a
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+               Merge Cond: (t1_7.b = (((t1_12.a + t1_12.b) / 2)))
                ->  Sort
-                     Sort Key: t1_5.b
-                     ->  Seq Scan on prt2_p3 t1_5
+                     Sort Key: t1_7.b
+                     ->  Seq Scan on prt2_p2 t1_7
                ->  Sort
-                     Sort Key: (((t1_8.a + t1_8.b) / 2))
-                     ->  Seq Scan on prt1_e_p3 t1_8
+                     Sort Key: (((t1_12.a + t1_12.b) / 2))
+                     ->  Seq Scan on prt1_e_p2 t1_12
                            Filter: (c = 0)
-(47 rows)
+   ->  Merge Semi Join
+         Merge Cond: (t1_3.a = t1_8.b)
+         ->  Sort
+               Sort Key: t1_3.a
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_8.b = (((t1_13.a + t1_13.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_8.b
+                     ->  Seq Scan on prt2_p3 t1_8
+               ->  Sort
+                     Sort Key: (((t1_13.a + t1_13.b) / 2))
+                     ->  Seq Scan on prt1_e_p3 t1_13
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_4.a = t1_9.b)
+         ->  Sort
+               Sort Key: t1_4.a
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_9.b = (((t1_14.a + t1_14.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_9.b
+                     ->  Seq Scan on prt2_p4 t1_9
+               ->  Sort
+                     Sort Key: (((t1_14.a + t1_14.b) / 2))
+                     ->  Seq Scan on prt1_e_p4 t1_14
+                           Filter: (c = 0)
+(77 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
@@ -921,14 +1411,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
                            ->  Sort
                                  Sort Key: (((t3.a + t3.b) / 2))
-                                 ->  Seq Scan on prt1_e_p1 t3
+                                 ->  Seq Scan on prt1_e_p0 t3
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1.a
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                ->  Sort
                      Sort Key: t2.b
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Merge Left Join
                Merge Cond: (t1_1.a = t2_1.b)
                ->  Sort
@@ -937,14 +1427,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
                            ->  Sort
                                  Sort Key: (((t3_1.a + t3_1.b) / 2))
-                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                 ->  Seq Scan on prt1_e_p1 t3_1
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_1.a
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                ->  Sort
                      Sort Key: t2_1.b
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Merge Left Join
                Merge Cond: (t1_2.a = t2_2.b)
                ->  Sort
@@ -953,32 +1443,74 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
                            ->  Sort
                                  Sort Key: (((t3_2.a + t3_2.b) / 2))
-                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                 ->  Seq Scan on prt1_e_p2 t3_2
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_2.a
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                ->  Sort
                      Sort Key: t2_2.b
-                     ->  Seq Scan on prt2_p3 t2_2
-(51 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Merge Left Join
+               Merge Cond: (t1_3.a = t2_3.b)
+               ->  Sort
+                     Sort Key: t1_3.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_3.a + t3_3.b) / 2)) = t1_3.a)
+                           ->  Sort
+                                 Sort Key: (((t3_3.a + t3_3.b) / 2))
+                                 ->  Seq Scan on prt1_e_p3 t3_3
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_3.a
+                                 ->  Seq Scan on prt1_p3 t1_3
+               ->  Sort
+                     Sort Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Merge Left Join
+               Merge Cond: (t1_4.a = t2_4.b)
+               ->  Sort
+                     Sort Key: t1_4.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_4.a + t3_4.b) / 2)) = t1_4.a)
+                           ->  Sort
+                                 Sort Key: (((t3_4.a + t3_4.b) / 2))
+                                 ->  Seq Scan on prt1_e_p4 t3_4
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_4.a
+                                 ->  Seq Scan on prt1_p4 t1_4
+               ->  Sort
+                     Sort Key: t2_4.b
+                     ->  Seq Scan on prt2_p4 t2_4
+(83 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- MergeAppend on nullable column
 -- This should generate a partitionwise join, but currently fails to
@@ -987,12 +1519,14 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Merge Left Join
-         Merge Cond: (prt1_p1.a = prt2_p2.b)
+         Merge Cond: (prt1_p0.a = prt2_p2.b)
          ->  Sort
-               Sort Key: prt1_p1.a
+               Sort Key: prt1_p0.a
                ->  Append
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p1
                            Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p2
@@ -1004,21 +1538,28 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                            Filter: (b > 250)
                      ->  Seq Scan on prt2_p3
                            Filter: (b > 250)
-(18 rows)
+                     ->  Seq Scan on prt2_p4
+                           Filter: (b > 250)
+(22 rows)
 
 SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  b  
------+-----
-   0 |    
-  50 |    
- 100 |    
- 150 |    
- 200 |    
- 250 |    
- 300 | 300
- 350 |    
- 400 |    
-(9 rows)
+  a   |  b  
+------+-----
+ -250 |    
+ -200 |    
+ -150 |    
+ -100 |    
+  -50 |    
+    0 |    
+   50 |    
+  100 |    
+  150 |    
+  200 |    
+  250 |    
+  300 | 300
+  350 |    
+  400 |    
+(14 rows)
 
 -- merge join when expression with whole-row reference needs to be sorted;
 -- partitionwise join does not apply
@@ -1032,175 +1573,2412 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
          Sort Key: t1.a, ((((t1.*)::prt1))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
    ->  Sort
          Sort Key: t2.b, ((((t2.*)::prt2))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt2_p1 t2
-                     ->  Seq Scan on prt2_p2 t2_1
-                     ->  Seq Scan on prt2_p3 t2_2
-(16 rows)
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+(20 rows)
 
 SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.b ORDER BY t1.a;
- a  | b  
-----+----
-  0 |  0
-  6 |  6
- 12 | 12
- 18 | 18
- 24 | 24
-(5 rows)
+  a  |  b  
+-----+-----
+ -24 | -24
+ -18 | -18
+ -12 | -12
+  -6 |  -6
+   0 |   0
+   6 |   6
+  12 |  12
+  18 |  18
+  24 |  24
+(9 rows)
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p4 t2_3
+               ->  Seq Scan on prt2_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_4
+                           Filter: (b = 0)
+(22 rows)
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p3 t1_2
+                           Filter: (b = 0)
+(21 rows)
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -150 | -0150 | 0 | -0150
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+(7 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 |   | 
+ -200 | -0200 |   | 
+ -150 | -0150 | 0 | -0150
+ -100 | -0100 |   | 
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+(21 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -200 | 0 | -0200
+ -100 | 0 | -0100
+  -50 | 0 | -0050
+   50 | 0 | 0050
+  100 | 0 | 0100
+  200 | 0 | 0200
+  250 | 0 | 0250
+  350 | 0 | 0350
+  400 | 0 | 0400
+  500 | 0 | 0500
+  550 | 0 | 0550
+  650 | 0 | 0650
+  700 | 0 | 0700
+(14 rows)
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+                             QUERY PLAN                              
+---------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t3.c
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p0_a on prt1_p0 t3
+                           Index Cond: (a = t2.b)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t2.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_1.a = t2_1.b)
+                           ->  Seq Scan on prt1_p1 t3_1
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1 t2_1
+                                       Filter: (a = 0)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p2_a on prt1_p2 t3_2
+                           Index Cond: (a = t2_2.b)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t3_3
+                           Index Cond: (a = t2_3.b)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_4.a = t2_4.b)
+                           ->  Seq Scan on prt1_p4 t3_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t2_4
+                                       Filter: (a = 0)
+(47 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+  a   | a |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.b
+   ->  Hash Right Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p5 t2_5
+                           Filter: (a = 0)
+(24 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: (t1.a = t2.b)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.a, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
+                     Filter: (a = 0)
+(22 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
+               ->  Seq Scan on prt1_m_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p1
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
+               ->  Seq Scan on prt1_m_p2
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p2
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
+               ->  Seq Scan on prt1_m_p3
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p3
+                           Filter: (c = 0)
+(24 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+  a  | c |  b  | c 
+-----+---+-----+---
+   0 | 0 |   0 | 0
+  50 | 0 |     |  
+ 100 | 0 |     |  
+ 150 | 0 | 150 | 0
+ 200 | 0 |     |  
+ 250 | 0 |     |  
+ 300 | 0 | 300 | 0
+ 350 | 0 |     |  
+ 400 | 0 |     |  
+ 450 | 0 | 450 | 0
+ 500 | 0 |     |  
+ 550 | 0 |     |  
+     |   |  75 | 0
+     |   | 225 | 0
+     |   | 375 | 0
+     |   | 525 | 0
+(16 rows)
+
+--
+-- tests for list partitioned tables.
+--
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
+--
+-- list partitioned by expression
+--
+CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
+ANALYZE plt1_e;
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+                                           QUERY PLAN                                           
+------------------------------------------------------------------------------------------------
+ GroupAggregate
+   Group Key: t1.c, t2.c, t3.c
+   ->  Sort
+         Sort Key: t1.c, t3.c
+         ->  Append
+               ->  Hash Join
+                     Hash Cond: ((t1.c)::text = ltrim(t3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2.b = t1.b) AND ((t2.c)::text = (t1.c)::text))
+                           ->  Seq Scan on plt2_p4 t2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p4 t1
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p4 t3
+               ->  Hash Join
+                     Hash Cond: ((t1_1.c)::text = ltrim(t3_1.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t1_1.b = t2_1.b) AND ((t1_1.c)::text = (t2_1.c)::text))
+                           ->  Seq Scan on plt1_p1 t1_1
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p1 t2_1
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p1 t3_1
+               ->  Hash Join
+                     Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t1_2.b = t2_2.b) AND ((t1_2.c)::text = (t2_2.c)::text))
+                           ->  Seq Scan on plt1_p2 t1_2
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p2 t2_2
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p2 t3_2
+               ->  Hash Join
+                     Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2_3.b = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text))
+                           ->  Seq Scan on plt2_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p3 t3_3
+(41 rows)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p5 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_3
+                                 ->  Seq Scan on plt2_p3 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
 
-RESET enable_hashjoin;
-RESET enable_nestloop;
---
--- partitioned by multiple columns
---
-CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
-CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
-ANALYZE prt1_m;
-CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
-CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
-ANALYZE prt2_m;
+-- semi join
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-                                                             QUERY PLAN                                                             
-------------------------------------------------------------------------------------------------------------------------------------
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
  Sort
-   Sort Key: prt1_m_p1.a, prt2_m_p1.b
-   ->  Append
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
-               ->  Seq Scan on prt1_m_p1
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p1
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
-               ->  Seq Scan on prt1_m_p2
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p2
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
-               ->  Seq Scan on prt1_m_p3
-                     Filter: (c = 0)
-               ->  Hash
-                     ->  Seq Scan on prt2_m_p3
-                           Filter: (c = 0)
-(24 rows)
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Seq Scan on plt2_p2 t2_2
+                                 ->  Seq Scan on plt2_p3 t2_3
+(22 rows)
 
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
-  50 | 0 |     |  
- 100 | 0 |     |  
- 150 | 0 | 150 | 0
- 200 | 0 |     |  
- 250 | 0 |     |  
- 300 | 0 | 300 | 0
- 350 | 0 |     |  
- 400 | 0 |     |  
- 450 | 0 | 450 | 0
- 500 | 0 |     |  
- 550 | 0 |     |  
-     |   |  75 | 0
-     |   | 225 | 0
-     |   | 375 | 0
-     |   | 525 | 0
-(16 rows)
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
 
---
--- tests for list partitioned tables.
---
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
---
--- list partitioned by expression
---
-CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1_e;
--- test partition matching with N-way join
+-- anti join
 EXPLAIN (COSTS OFF)
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                   QUERY PLAN                                   
---------------------------------------------------------------------------------
- GroupAggregate
-   Group Key: t1.c, t2.c, t3.c
-   ->  Sort
-         Sort Key: t1.c, t3.c
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
          ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.c = ltrim(t3.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1.b = t2.b) AND (t1.c = t2.c))
-                           ->  Seq Scan on plt1_p1 t1
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.c = ltrim(t3_1.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
-                           ->  Seq Scan on plt1_p2 t1_1
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.c = ltrim(t3_2.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
-                           ->  Seq Scan on plt1_p3 t1_2
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p3 t3_2
-(32 rows)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+(19 rows)
 
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  75.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 123.0000000000000000 | 123.0000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 225.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 273.0000000000000000 | 273.0000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 375.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 423.0000000000000000 | 423.0000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 525.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 573.0000000000000000 | 573.0000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1225,22 +4003,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
 --------------------------------------------------
  Hash Left Join
    Hash Cond: (t2.b = a)
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t3.a = t2.b)
-               ->  Seq Scan on prt1_p1 t3
-               ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
-         ->  Hash Join
-               Hash Cond: (t3_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t3_1
-               ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
-         ->  Hash Join
-               Hash Cond: (t3_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t3_2
-               ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
+   ->  Hash Join
+         Hash Cond: (t3.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t3
+               ->  Seq Scan on prt1_p1 t3_1
+               ->  Seq Scan on prt1_p2 t3_2
+               ->  Seq Scan on prt1_p3 t3_3
+               ->  Seq Scan on prt1_p4 t3_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
    ->  Hash
          ->  Result
                One-Time Filter: false
@@ -1255,16 +4033,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 --
 -- tests for hash partitioned tables.
@@ -1340,41 +4124,9 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  273.0000000000000000 | 273.0000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
 (6 rows)
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
- Sort
-   Sort Key: t1.a
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
-               ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
-               ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
-                           Filter: (b = 0)
-(21 rows)
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
@@ -1389,26 +4141,24 @@ SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c
    Sort Key: t1.c
    ->  HashAggregate
          Group Key: t1.c, t2.c
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t2.c = t1.c)
-                     ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p1 t1
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on plt1_p4 t1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_1.c = t1_1.c)
-                     ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_p2 t1_1
+                           ->  Seq Scan on plt1_p1 t1_1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_2.c = t1_2.c)
-                     ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Seq Scan on plt1_p2 t1_2
                                  Filter: ((a % 25) = 0)
-(23 rows)
+                           ->  Seq Scan on plt1_p3 t1_3
+                                 Filter: ((a % 25) = 0)
+(21 rows)
 
 --
 -- multiple levels of partitioning
@@ -1804,64 +4554,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
  Hash Join
    Hash Cond: (t1.a = t2.a)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt4_n_p1 t2
                ->  Seq Scan on prt4_n_p2 t2_1
                ->  Seq Scan on prt4_n_p3 t2_2
-(11 rows)
+(13 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a and t1.a = t3.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                        QUERY PLAN                        
+----------------------------------------------------------
  Hash Join
-   Hash Cond: (t2.a = t1.a)
+   Hash Cond: (t3.b = t1.a)
    ->  Append
-         ->  Seq Scan on prt4_n_p1 t2
-         ->  Seq Scan on prt4_n_p2 t2_1
-         ->  Seq Scan on prt4_n_p3 t2_2
+         ->  Seq Scan on prt2_p0 t3
+         ->  Seq Scan on prt2_p1 t3_1
+         ->  Seq Scan on prt2_p2 t3_2
+         ->  Seq Scan on prt2_p3 t3_3
+         ->  Seq Scan on prt2_p4 t3_4
+         ->  Seq Scan on prt2_p5 t3_5
    ->  Hash
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.a = t3_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_2.b)
-                     ->  Seq Scan on prt1_p3 t1_2
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3 t3_2
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt4_n_p1 t2
+                           ->  Seq Scan on prt4_n_p2 t2_1
+                           ->  Seq Scan on prt4_n_p3 t2_2
 (23 rows)
 
 -- partitionwise join can not be applied if there are no equi-join conditions
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (b > t1.a)
-(12 rows)
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+(16 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partitionwise join
@@ -1947,16 +4703,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
-                     ->  Seq Scan on plt1_p1 t3
-                     ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3
+                     ->  Seq Scan on plt1_p1 t3_1
+                     ->  Seq Scan on plt1_p2 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partitionwise join can not be applied for a join between list and range
 -- partitioned tables
@@ -1967,14 +4724,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
  Hash Full Join
    Hash Cond: ((t2.c)::text = (t1.c)::text)
    ->  Append
-         ->  Seq Scan on prt1_p1 t2
-         ->  Seq Scan on prt1_p2 t2_1
-         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p0 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p2 t2_2
+         ->  Seq Scan on prt1_p3 t2_3
+         ->  Seq Scan on prt1_p4 t2_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt1_n_p1 t1
                ->  Seq Scan on prt1_n_p2 t1_1
-(10 rows)
+(12 rows)
 
 -- partitionwise join can not be applied if only one of joining tables has
 -- default partition
@@ -1990,16 +4749,23 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
    ->  Hash Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index fb3ba18a26..34ae92135f 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -10,25 +10,39 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -69,11 +83,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -95,20 +117,30 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 
 EXPLAIN (COSTS OFF)
@@ -172,6 +204,128 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -196,28 +350,79 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
 -- test partition matching with N-way join
@@ -225,6 +430,175 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
@@ -270,27 +644,18 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
 ALTER TABLE plt2 DETACH PARTITION plt2_p3;
 ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
 ANALYZE plt2;
-
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.a % 25 = 0 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
 --
 -- multiple levels of partitioning
 --
#97Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Etsuro Fujita (#96)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Wed, Sep 25, 2019 at 12:59 AM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

I will continue to review the rest of the patch.

I've been reviewing the rest of the patch. Here are my review comments:

* map_and_merge_partitions() checks whether the two partitions from
the outer and inner sides can be merged in two steps: 1) see if the
partitions are mapped to each other (ie, partmap1->from == index2 &&
partmap2->from == index1), and 2) see if the merged partition indexes
assigned are the same (ie, partmap1->to == partmap2->to) (or satisfy
some other conditions), but the first step seems redundant to me
because I think that if the merged partition indexes are the same,
then the partitions would be guaranteed to be mapped to each other.
Also, I noticed that that function can't handle some list-partitioning
cases properly. Here is an example:

CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0002');
CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM
generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
ANALYZE plt1;
CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0001', '0002');
CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM
generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
ANALYZE plt2;
EXPLAIN (COSTS OFF)
SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c
= t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 !=
4 ORDER BY t1.c, t1.a, t2.a;
QUERY PLAN

-------------------------------------------------------------------------------
------
Sort
Sort Key: t1.c, t1.a, t2.a
-> Hash Full Join
Hash Cond: (t1.c = t2.c)
Filter: (((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a,
0) % 5) <> 4))
-> Append
-> Seq Scan on plt1_p1 t1
-> Seq Scan on plt1_p2 t1_1
-> Hash
-> Append
-> Seq Scan on plt2_p1 t2
-> Seq Scan on plt2_p2 t2_1
(12 rows)

This should use partitionwise join by the new partition-matching
algorithm but doesn't. The reason for that is because plt1_p1 and
plt2_p1 are mapped to different merged partitions and thus considered
not merge-able by map_and_merge_partitions() as-is. I might be
missing something, but I don't think this is intentional, so I rewrote
that function completely in the attached, which is a WIP patch created
on top of the patch [1]/messages/by-id/CAPmGK14WHKckT1P6UJV2B63TZAxPyMn8iZJ99XF=AZuNhG6vow@mail.gmail.com.

* In handle_missing_partition(), I noticed this:

+   else if (missing_side_inner)
+   {
+       /*
+        * If this partition has already been mapped (say because we
+        * found an overlapping range earlier), we know where does it
+        * fit in the join result. Nothing to do in that case. Else
+        * create a new merged partition.
+        */
+       PartitionMap *extra_map = &maps_with_extra[extra_part];
+       if (extra_map->to < 0)
+       {
+           extra_map->to = *next_index;
+           *next_index = *next_index + 1;
+           *merged_index = extra_map->to;
+       }
+   }

As commented, that function skips setting *merged_index when the
"extra_part" partition is already mapped to a merged partition. This
would be correct for range partitioning, but not for list
partitioning, I think. Here is an example:

CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0001', '0002');
CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM
generate_series(0, 24) i;
ANALYZE plt1;
CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002');
CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM
generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
ANALYZE plt2;
CREATE TABLE plt3 (a int, b int, c text) PARTITION BY LIST(c);
CREATE TABLE plt3_p1 PARTITION OF plt3 FOR VALUES IN ('0001');
CREATE TABLE plt3_p2 PARTITION OF plt3 FOR VALUES IN ('0003', '0004');
INSERT INTO plt3 SELECT i, i, to_char(i % 5, 'FM0000') FROM
generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
ANALYZE plt3;
EXPLAIN (COSTS OFF)
SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2
t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE
COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY
t1.c, t1.a, t2.a, t3.a;
QUERY PLAN

-------------------------------------------------------------------------------
------
Sort
Sort Key: t1.c, t1.a, t2.a, t3.a
-> Hash Full Join
Hash Cond: (t1.c = t3.c)
Filter: (((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a,
0) % 5) <> 4))
-> Hash Left Join
Hash Cond: (t1.c = t2.c)
-> Append
-> Seq Scan on plt1_p1 t1
-> Seq Scan on plt1_p2 t1_1
-> Hash
-> Append
-> Seq Scan on plt2_p1 t2
-> Seq Scan on plt2_p2 t2_1
-> Hash
-> Append
-> Seq Scan on plt3_p1 t3
-> Seq Scan on plt3_p2 t3_1
(18 rows)

I think this should use 3-way partitionwise join by the new algorithm
but doesn't. The reason for that is because when considering
partitionwise join for plt1 and plt2, partition_list_bounds_merge()
would incorrectly produce {'0000', '0002'} as the merged values for
the join segment of plt1_p1 and plt2_p1, not {'0000', '0001', '0002'},
because that function would call handle_missing_partition() for the
values '0000' and '0001' of plt1_p1, and it would set *merged_index
correctly for '0000' but not for '0001' (keeping *merged_index=-1),
due to the behavior mentioned above, resulting in the merged values
for the join segment {'0000', '0002'}, not {'0000', '0001', '0002'}.
This would cause to fail to merge the values for the join segment
{'0000', '0002'} and the values for plt3_p1 {'0001'} when considering
partitionwise join for the 2-way join and plt3. I fixed this as well
in the attached. handle_missing_partition() can handle both cases
where a datum is missing from the inner side and where a datum is
missing from the outer side in a unified way, but I think that that
makes the code pretty hard to read. Also, I think
handle_missing_partition() is not caller-friendly because the caller
needs to write something like this:

+               /* A range missing from the inner side. */
+               bool        missing_side_outer;
+               bool        missing_side_inner;
+               /*
+                * For a FULL join, inner relation acts as both OUTER and INNER
+                * relation.  For LEFT and ANTI join the inner relation acts as
+                * INNER relation. For INNER and SEMI join OUTER and INNER
+                * differentiation is immaterial.
+                */
+               missing_side_inner = (jointype == JOIN_FULL ||
+                                     jointype == JOIN_LEFT ||
+                                     jointype == JOIN_ANTI);
+               missing_side_outer = (jointype == JOIN_FULL);
+               if (!handle_missing_partition(inner_maps,
+                                             outer_maps,
+                                             inner_default,
+                                             outer_part,
+                                             missing_side_outer,
+                                             missing_side_inner,
+                                             &next_index,
+                                             &default_index,
+                                             &merged_index))
+                   return NULL;

So I'd like to propose to introduce separate functions like
process_outer_partition() and process_inner_partition() in the
attached, instead of handle_missing_partition(). (I added a fast path
to these functions that if both outer/inner sides have the default
partitions, give up on merging partitions. Also, I modified the
caller sites of proposed functions so that they call these if
necessary.)

Other minor changes in the attached:

* I modified partition_range_bounds_merge() and
partition_list_bounds_merge() so that the order of arguments for them
matches partition_bounds_merge(). Also, I did some cleanup for these
functions, and removed this in these function because this merely
re-checks the while condition above.

+       /* If we exhausted both the sides, we won't enter the loop. */
+       Assert(!finished_inner || !finished_outer);

* I modified merge_null_partitions() and merge_default_partitions()
accordingly to the changes described above. Also, I did some cleanup
and modified the caller sites of these functions so that they call
these functions if necessary.

* I also modified generate_matching_part_pairs() accordingly.

* I added a creanup to free memory allocated by init_partition_map().

* We have this in a few places such as merge_default_partitions():

+           extra_map->to = *next_index;
+           *next_index = *next_index + 1;
+           *merged_index = extra_map->to;

To reduce code duplication, I made this into a separate function.

* I think the return statement in the end of partition_range_merge()
is useless, so I removed it.

That is all for now. Will continue to review.

Best regards,
Etsuro Fujita

[1]: /messages/by-id/CAPmGK14WHKckT1P6UJV2B63TZAxPyMn8iZJ99XF=AZuNhG6vow@mail.gmail.com

Attachments:

modify-partbounds-changes-1.patchapplication/octet-stream; name=modify-partbounds-changes-1.patchDownload
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 1238999c22..4c3a31c82c 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -72,8 +72,13 @@ typedef struct PartitionRangeBound
 
 typedef struct PartitionMap
 {
-	int from;
-	int to;
+	int			nparts;			/* number of partitions */
+	int		   *merged_indexes;	/* indexes of merged partitions */
+	bool	   *merged;			/* flags to indicate whether partitions are
+								 * merged with non-dummy partitions */
+	bool		did_remapping;	/* did we remap partitions? */
+	int		   *old_indexes;	/* old indexes of merged partitions if
+								 * did_remapping */
 } PartitionMap;
 
 static int32 qsort_partition_hbound_cmp(const void *a, const void *b);
@@ -115,20 +120,49 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 									 Expr **keyCol,
 									 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
-static PartitionBoundInfo partition_range_bounds_merge(
+static PartitionBoundInfo partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
 							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
-							 List **outer_parts, List **inner_parts,
-							 JoinType jointype, int partnatts,
-							 FmgrInfo *supfuncs, Oid *collations);
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts);
 static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
 							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
-							List **outer_parts, List **inner_parts,
-							JoinType jointype);
+							JoinType jointype,
+							List **outer_parts, List **inner_parts);
+static void init_partition_map(RelOptInfo *rel, PartitionMap *map);
+static void free_partition_map(PartitionMap *map);
+static int map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_part, int inner_part, int *next_index);
+static int merge_partition_with_dummy(PartitionMap *map, int index,
+						   int *next_index);
+static bool process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index);
+static bool process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index);
+static void fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes);
 static void generate_matching_part_pairs(RelOptInfo *rel1,
 							 RelOptInfo *rel2,
-							 PartitionMap *partmaps1,
-							 PartitionMap *partmaps2,
-							 int nparts1, int nparts2,
+							 PartitionMap *map1,
+							 PartitionMap *map2,
 							 int nparts,
 							 List **matched_parts1,
 							 List **matched_parts2);
@@ -136,9 +170,6 @@ static PartitionBoundInfo build_merged_partition_bounds(char strategy,
 							  List *merged_datums, List *merged_indexes,
 							  List *merged_contents, int null_index,
 							  int default_index);
-static int map_and_merge_partitions(PartitionMap *outer_maps,
-										PartitionMap *inner_maps,
-										int index1, int index2, int *next_index);
 static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
 						  Oid *collations, PartitionRangeBound *bound1,
 						  PartitionRangeBound *bound2);
@@ -153,19 +184,21 @@ static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
 							  PartitionRangeDatumKind *next_lb_kind,
 							  List **merged_datums, List **merged_kinds,
 							  List **merged_indexes);
-static bool merge_default_partitions(PartitionBoundInfo outer_bi,
-						 PartitionBoundInfo inner_bi,
-						 PartitionMap *outer_maps,
-						 PartitionMap *inner_maps,
+static bool merge_default_partitions(PartitionMap *outer_map,
+						 PartitionMap *inner_map,
+						 bool outer_has_default,
+						 int outer_default,
+						 bool inner_has_default,
+						 int inner_default,
 						 JoinType jointype,
-						 int *next_index, int *default_index);
-static bool merge_null_partitions(PartitionBoundInfo outer_bi,
-								  PartitionBoundInfo inner_bi,
-								  PartitionMap *outer_maps,
-								  PartitionMap *inner_maps,
-								  JoinType jointype,
-								  int *next_index, int *null_index,
-								  int *default_index);
+						 int *next_index,
+						 int *default_index);
+static bool merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_map, PartitionMap *inner_map,
+					  bool outer_has_default, bool inner_has_default,
+					  bool outer_has_null, bool inner_has_null,
+					  JoinType jointype, int *next_index,
+					  int *default_index, int *null_index);
 
 /*
  * get_qual_from_partbound
@@ -3082,8 +3115,8 @@ partition_bounds_merge(int partnatts,
 					   int16 *parttyplen, bool *parttypbyval,
 					   FmgrInfo *partsupfunc, Oid *partcollation,
 					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
-					   JoinType jointype,
-					   List **outer_parts, List **inner_parts)
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
 {
 	PartitionBoundInfo 	merged_bounds;
 	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
@@ -3111,17 +3144,22 @@ partition_bounds_merge(int partnatts,
 		case PARTITION_STRATEGY_LIST:
 			merged_bounds = partition_list_bounds_merge(partsupfunc,
 														partcollation,
-														outer_rel, inner_rel,
-														outer_parts, inner_parts,
-														jointype);
+														outer_rel,
+														inner_rel,
+														jointype,
+														outer_parts,
+														inner_parts);
 			break;
 
 		case PARTITION_STRATEGY_RANGE:
-			merged_bounds = partition_range_bounds_merge(outer_rel, inner_rel,
-														 outer_parts, inner_parts,
-														 jointype, partnatts,
+			merged_bounds = partition_range_bounds_merge(partnatts,
 														 partsupfunc,
-														 partcollation);
+														 partcollation,
+														 outer_rel,
+														 inner_rel,
+														 jointype,
+														 outer_parts,
+														 inner_parts);
 			break;
 
 		default:
@@ -3336,8 +3374,6 @@ partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
 		default:
 			elog(ERROR, "unexpected join type %d", jointype);
 	}
-
-	return;
 }
 
 /*
@@ -3397,157 +3433,34 @@ partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
 	return true;
 }
 
-/*
- * handle_missing_partition
- *
- * If a range appears in one of the joining relations but not the other, a row
- * in the corresponding partition will not have any join partner in the other
- * relation, unless the other relation has a default partition. If a given list
- * value is present in one joining relation but not the other, the default
- * partition on the other side may contain that value.
- *
- * In both these cases, such an extra partition forms a joining pair with the
- * default partition, if any,  on the other side.
- *
- * If the default partition happens to be on the outer side of the join, the
- * resultant partition will act as the default partition of the join relation.
- * Otherwise the resultant partition will be associated with the range.
- *
- * When the default partition is not present in the other relation, the rows in
- * the extra partition will be included in the bounds of the join result, if it
- * appears on the outer side of the join, since all rows from the outer side
- * are included in the join result.
- *
- * This function handles all these cases.
- *
- * maps_with_missing and missing_side_default are the partition maps (See
- * partition_range/list_bounds_merge() for details) and the index of default
- * partition respectively corresponding the side with missing partition.
- *
- * maps_with_extra and extra_part are the partition maps (See
- * partition_range/list_bounds_merge() for details) and the index of extra
- * partition respectively corresponding to the side with the extra partition.
- *
- * It returns true if the matching succeeds, otherwise returns false.
- */
-static bool
-handle_missing_partition(PartitionMap *maps_with_missing,
-						 PartitionMap *maps_with_extra,
-						 int missing_side_default,
-						 int extra_part,
-						 bool missing_side_outer,
-						 bool missing_side_inner,
-						 int *next_index, int *default_index,
-						 int *merged_index)
-{
-	bool missing_has_default = (missing_side_default != -1);
-
-	if (missing_has_default)
-	{
-		*merged_index = map_and_merge_partitions(maps_with_missing,
-												 maps_with_extra,
-												 missing_side_default,
-												 extra_part,
-												 next_index);
-		if (*merged_index < 0)
-			return false;
-
-		if (missing_side_outer)
-		{
-			/*
-			 * Default partition on the outer side forms the default
-			 * partition of the join result.
-			 */
-			if (*default_index < 0)
-				*default_index = *merged_index;
-			else if(*default_index != *merged_index)
-			{
-				/*
-				 * Ended up with default partition on the outer side
-				 * being joined with multiple partitions on the inner
-				 * side. We don't support this case.
-				 */
-				return false;
-			}
-
-			/*
-			 * Since the merged partition acts as a default partition, it
-			 * doesn't need a separate index.
-			 */
-			*merged_index = -1;
-		}
-	}
-	else if (missing_side_inner)
-	{
-		/*
-		 * If this partition has already been mapped (say because we
-		 * found an overlapping range earlier), we know where does it
-		 * fit in the join result. Nothing to do in that case. Else
-		 * create a new merged partition.
-		 */
-		PartitionMap *extra_map = &maps_with_extra[extra_part];
-		if (extra_map->to < 0)
-		{
-			extra_map->to = *next_index;
-			*next_index = *next_index + 1;
-			*merged_index = extra_map->to;
-		}
-	}
-	else
-		*merged_index = -1;
-
-	return true;
-}
-
-static PartitionMap*
-init_partition_map(RelOptInfo *rel)
-{
-	int i, nparts = rel->nparts;
-	PartitionMap *map;
-
-	map = (PartitionMap *) palloc(sizeof(PartitionMap) * nparts);
-
-	for (i = 0; i < nparts; i++)
-	{
-		map[i].from = -1;
-		map[i].to = -1;
-	}
-
-	return map;
-}
-
 /*
  * partition_range_bounds_merge
  *
  * partition_bounds_merge()'s arm for range partitioned tables.
  */
 static PartitionBoundInfo
-partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
-							 List **outer_parts, List **inner_parts,
-							 JoinType jointype, int partnatts,
-							 FmgrInfo *partsupfuncs, Oid *partcollations)
-
+partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts)
 {
-	PartitionMap *outer_maps = NULL;
-	PartitionMap *inner_maps = NULL;
-	int			outer_part = 0;
-	int			inner_part = 0;
 	PartitionBoundInfo merged_bounds = NULL;
-	int			outer_lb_index;
-	int			inner_lb_index;
-	int			next_index;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			next_index = 0;
 	int			default_index = -1;
 	List	   *merged_datums = NIL;
-	List	   *merged_indexes = NIL;
 	List	   *merged_kinds = NIL;
-	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
-					   inner_bi = inner_rel->boundinfo;
-	int			inner_default = inner_bi->default_index;
-	int			outer_default = outer_bi->default_index;
-	bool		inner_has_default = partition_bound_has_default(inner_bi);
-	bool		outer_has_default = partition_bound_has_default(outer_bi);
-	int 			   outer_nparts = outer_rel->nparts,
-					   inner_nparts = inner_rel->nparts;
+	List	   *merged_indexes = NIL;
+	int			outer_lb_index;
+	int			inner_lb_index;
 
 	Assert(outer_bi->strategy == inner_bi->strategy &&
 		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
@@ -3555,8 +3468,8 @@ partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 	Assert(*outer_parts == NIL);
 	Assert(*inner_parts == NIL);
 
-	outer_maps = init_partition_map(outer_rel);
-	inner_maps = init_partition_map(inner_rel);
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
 
 	/*
 	 * Merge the ranges (partitions) from both sides. Every iteration compares
@@ -3567,40 +3480,30 @@ partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 	 * lb_index, for inner or outer side, keeps track of the index of lower bound
 	 * datum in PartitionBoundInfo::datums of that side.
 	 */
-	outer_lb_index = 0;
-	inner_lb_index = 0;
-	next_index = 0;
+	outer_lb_index = inner_lb_index = 0;
 	while (outer_lb_index < outer_bi->ndatums ||
 		   inner_lb_index < inner_bi->ndatums)
 	{
-		PartitionRangeBound outer_lb, outer_ub,
-							inner_lb, inner_ub,
-							*merged_lb = NULL,
-							*merged_ub = NULL;
-
+		PartitionRangeBound *merged_lb = NULL;
+		PartitionRangeBound *merged_ub = NULL;
 		int			merged_index = -1;
+		PartitionRangeBound outer_lb;
+		PartitionRangeBound outer_ub;
+		PartitionRangeBound inner_lb;
+		PartitionRangeBound inner_ub;
+		int			outer_part = -1;
+		int			inner_part = -1;
 		bool		overlap;
-		bool		finished_outer = false;
-		bool		finished_inner = false;
-
-		/* Result of bounds comparison per partition_rbound_cmp(). */
-		int			ub_cmpval;	/* Upper bounds comparison result. */
-		int			lb_cmpval;	/* Lower bounds comparison result. */
+		int			ub_cmpval;
+		int			lb_cmpval;
 
 		/* Get the range bounds of the next pair of partitions. */
 		if (outer_lb_index < outer_bi->ndatums)
 			outer_part = partition_get_range_bounds(outer_bi, outer_lb_index,
-												&outer_lb, &outer_ub);
-		else
-			finished_outer = true;
-
+													&outer_lb, &outer_ub);
 		if (inner_lb_index < inner_bi->ndatums)
 			inner_part = partition_get_range_bounds(inner_bi, inner_lb_index,
-												&inner_lb, &inner_ub);
-		else
-			finished_inner = true;
-
-		Assert(!finished_outer || !finished_inner);
+													&inner_lb, &inner_ub);
 
 		/*
 		 * We run this loop till both the sides finish. This allows to avoid
@@ -3612,13 +3515,13 @@ partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 		 * side. That way we advance the partitions on that side till all of
 		 * them are  exhausted.
 		 */
-		if (finished_outer)
+		if (outer_lb_index >= outer_bi->ndatums)
 		{
 			overlap = false;
 			ub_cmpval = 1;
 			lb_cmpval = 1;
 		}
-		else if (finished_inner)
+		else if (inner_lb_index >= inner_bi->ndatums)
 		{
 			overlap = false;
 			ub_cmpval = -1;
@@ -3642,7 +3545,7 @@ partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 								  jointype, &outer_lb, &outer_ub, &inner_lb,
 								  &inner_ub, &merged_lb, &merged_ub);
 
-			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
 													outer_part, inner_part,
 													&next_index);
 
@@ -3681,6 +3584,8 @@ partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 			Assert(overlap);
 
 			/* Move to the next pair of partitions. */
+			Assert(outer_lb_index < outer_bi->ndatums);
+			Assert(inner_lb_index < inner_bi->ndatums);
 			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
 															   outer_lb_index);
 			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
@@ -3696,37 +3601,31 @@ partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 			}
 			else
 			{
-				/* A range missing from the inner side. */
-				bool		missing_side_outer;
-				bool		missing_side_inner;
+				if (inner_has_default ||
+					jointype == JOIN_LEFT ||
+					jointype == JOIN_ANTI ||
+					jointype == JOIN_FULL)
+				{
+					if (!process_outer_partition(&outer_map,
+												 &inner_map,
+												 outer_has_default,
+												 inner_has_default,
+												 outer_part,
+												 inner_default,
+												 jointype,
+												 outer_bi->strategy,
+												 &next_index,
+												 &default_index,
+												 &merged_index))
+						return NULL;
+				}
 
 				merged_lb = &outer_lb;
 				merged_ub = &outer_ub;
-
-				/*
-				 * For a FULL join, inner relation acts as both OUTER and INNER
-				 * relation.  For LEFT and ANTI join the inner relation acts as
-				 * INNER relation. For INNER and SEMI join OUTER and INNER
-				 * differentiation is immaterial.
-				 */
-				missing_side_inner = (jointype == JOIN_FULL ||
-									  jointype == JOIN_LEFT ||
-									  jointype == JOIN_ANTI);
-				missing_side_outer = (jointype == JOIN_FULL);
-				if (!handle_missing_partition(inner_maps,
-											  outer_maps,
-											  inner_default,
-											  outer_part,
-											  missing_side_outer,
-											  missing_side_inner,
-											  &next_index,
-											  &default_index,
-											  &merged_index))
-					return NULL;
 			}
 
 			/* Move to the next partition on the outer side. */
-			Assert(!finished_outer);
+			Assert(outer_lb_index < outer_bi->ndatums);
 			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
 															   outer_lb_index);
 		}
@@ -3741,38 +3640,28 @@ partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 			}
 			else
 			{
-				/* A range missing from the outer side. */
-				bool		missing_side_outer;
-				bool		missing_side_inner;
+				if (outer_has_default || jointype == JOIN_FULL)
+				{
+					if (!process_inner_partition(&outer_map,
+												 &inner_map,
+												 outer_has_default,
+												 inner_has_default,
+												 inner_part,
+												 outer_default,
+												 jointype,
+												 outer_bi->strategy,
+												 &next_index,
+												 &default_index,
+												 &merged_index))
+						return NULL;
+				}
 
 				merged_lb = &inner_lb;
 				merged_ub = &inner_ub;
-
-				/*
-				 * For a FULL join, outer relation acts as both OUTER and INNER
-				 * relation.  For LEFT and ANTI join the outer relation acts as
-				 * OUTER relation. For INNER and SEMI join OUTER and INNER
-				 * differentiation is immaterial.
-				 */
-				missing_side_outer = (jointype == JOIN_FULL ||
-									  jointype == JOIN_LEFT ||
-									  jointype == JOIN_ANTI);
-				missing_side_inner = (jointype == JOIN_FULL);
-
-				if (!handle_missing_partition(outer_maps,
-											  inner_maps,
-											  outer_default,
-											  inner_part,
-											  missing_side_outer,
-											  missing_side_inner,
-											  &next_index,
-											  &default_index,
-											  &merged_index))
-					return NULL;
 			}
 
 			/* Move to the next partition on the inner side. */
-			Assert (!finished_inner);
+			Assert(inner_lb_index < inner_bi->ndatums);
 			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
 															   inner_lb_index);
 		}
@@ -3803,16 +3692,28 @@ partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 		merged_indexes = lappend_int(merged_indexes, merged_index);
 	}
 
-	if (!merge_default_partitions(outer_bi, inner_bi,
-										  outer_maps, inner_maps,
-										  jointype, &next_index,
-										  &default_index))
-		return NULL;
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+	{
+		if (!merge_default_partitions(&outer_map,
+									  &inner_map,
+									  outer_has_default,
+									  outer_default,
+									  inner_has_default,
+									  inner_default,
+									  jointype,
+									  &next_index,
+									  &default_index))
+			return NULL;
+	}
+	else
+		Assert(default_index == -1);
+
+	Assert(!outer_map.did_remapping && !inner_map.did_remapping);
 
 	/* Use maps to match partition from the joining relations. */
 	generate_matching_part_pairs(outer_rel, inner_rel,
-								 outer_maps, inner_maps,
-								 outer_nparts, inner_nparts,
+								 &outer_map, &inner_map,
 								 next_index,
 								 outer_parts, inner_parts);
 
@@ -3829,6 +3730,8 @@ partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 	}
 
 	/* Free any memory we used in this function. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
 	list_free(merged_datums);
 	list_free(merged_indexes);
 	list_free(merged_kinds);
@@ -3845,27 +3748,27 @@ partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 static PartitionBoundInfo
 partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
-							List **outer_parts, List **inner_parts,
-							JoinType jointype)
+							JoinType jointype,
+							List **outer_parts, List **inner_parts)
 {
-	PartitionMap *outer_maps = NULL;
-	PartitionMap *inner_maps = NULL;
-	int			cnto;
-	int			cnti;
-	List	   *merged_datums = NIL;
-	List	   *merged_indexes = NIL;
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	PartitionMap outer_map;
+	PartitionMap inner_map;
 	int			next_index = 0;
 	int			null_index = -1;
 	int			default_index = -1;
-	PartitionBoundInfo merged_bounds = NULL;
-	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
-					   inner_bi = inner_rel->boundinfo;
-	int			      *outer_indexes = outer_bi->indexes;
-	int			      *inner_indexes = inner_bi->indexes;
-	int				   outer_default = outer_bi->default_index;
-	int				   inner_default = inner_bi->default_index;
-	int 			   outer_nparts = outer_rel->nparts,
-					   inner_nparts = inner_rel->nparts;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			cnto;
+	int			cnti;
 
 	Assert(*outer_parts == NIL);
 	Assert(*inner_parts == NIL);
@@ -3876,8 +3779,8 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 	/* List partitions do not require unbounded ranges. */
 	Assert(!outer_bi->kind && !inner_bi->kind);
 
-	outer_maps = init_partition_map(outer_rel);
-	inner_maps = init_partition_map(inner_rel);
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
 
 	/*
 	 * Merge the list value datums from both sides. Every iteration compares a
@@ -3889,15 +3792,15 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 	cnto = cnti = 0;
 	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
 	{
+		Datum	   *merged_datum = NULL;
+		int			merged_index = -1;
 		Datum	   *odatums;
 		Datum	   *idatums;
-		int			o_index;
-		int			i_index;
 		int			cmpval;
-		int			merged_index = -1;
-		Datum	   *merged_datum;
-		bool		finished_inner;
-		bool		finished_outer;
+
+		/* Get the list datums of the next pair of partitions. */
+		odatums = cnto < outer_bi->ndatums ? outer_bi->datums[cnto] : NULL;
+		idatums = cnti < inner_bi->ndatums ? inner_bi->datums[cnti] : NULL;
 
 		/*
 		 * We run this loop till both the sides finish. This allows to avoid
@@ -3909,51 +3812,23 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 		 * its datums are exhausted.
 		 */
 		if (cnto >= outer_bi->ndatums)
-		{
-			finished_outer = true;
-			odatums = NULL;
-			o_index = -1;
-		}
-		else
-		{
-			finished_outer = false;
-			odatums = outer_bi->datums[cnto];
-			o_index = outer_indexes[cnto];
-		}
-
-		if (cnti >= inner_bi->ndatums)
-		{
-			finished_inner = true;
-			idatums = NULL;
-			i_index = -1;
-		}
-		else
-		{
-			finished_inner = false;
-			idatums = inner_bi->datums[cnti];
-			i_index = inner_indexes[cnti];
-		}
-
-		/* If we exhausted both the sides, we won't enter the loop. */
-		Assert(!finished_inner || !finished_outer);
-
-		if (finished_outer)
 			cmpval = 1;
-		else if (finished_inner)
+		else if (cnti >= inner_bi->ndatums)
 			cmpval = -1;
 		else
 		{
-			/* Every list datum should map to a valid partition index. */
-			Assert(o_index >= 0 && i_index >= 0 &&
-				   odatums != NULL && idatums != NULL);
-
+			Assert(odatums != NULL && idatums != NULL);
 			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
 													 partcollation[0],
-													 odatums[0], idatums[0]));
+													 odatums[0],
+													 idatums[0]));
 		}
 
 		if (cmpval == 0)
 		{
+			int			o_index = outer_bi->indexes[cnto];
+			int			i_index = inner_bi->indexes[cnti];
+
 			/*
 			 * Datums match. Rows on either side with these datums as partition
 			 * key value will join and will be part of the partition of the
@@ -3963,7 +3838,8 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 			 * partition containing it.
 			 */
 			merged_datum = odatums;
-			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+			Assert(o_index >= 0 && i_index >= 0);
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
 													o_index, i_index,
 													&next_index);
 
@@ -3976,74 +3852,64 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 		}
 		else if (cmpval < 0)
 		{
-			bool		missing_side_outer;
-			bool		missing_side_inner;
+			Assert(cnto < outer_bi->ndatums);
 
 			/* A datum missing from the inner side. */
-			merged_index = -1;
 			merged_datum = odatums;
 
-			/*
-			 * For a FULL join, inner relation acts as both OUTER and INNER
-			 * relation.  For LEFT and ANTI join the inner relation acts as
-			 * INNER relation. For INNER and SEMI join OUTER and INNER
-			 * differentiation is immaterial.
-			 */
-			missing_side_inner = (jointype == JOIN_FULL ||
-								  jointype == JOIN_LEFT ||
-								  jointype == JOIN_ANTI);
-			missing_side_outer = (jointype == JOIN_FULL);
-
-			if (!handle_missing_partition(inner_maps,
-										  outer_maps,
-										  inner_default,
-										  o_index,
-										  missing_side_outer,
-										  missing_side_inner,
-										  &next_index,
-										  &default_index,
-										  &merged_index))
-				return NULL;
+			if (inner_has_default ||
+				jointype == JOIN_LEFT ||
+				jointype == JOIN_ANTI ||
+				jointype == JOIN_FULL)
+			{
+				int			o_index = outer_bi->indexes[cnto];
+
+				Assert(o_index >= 0);
+				if (!process_outer_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 o_index,
+											 inner_default,
+											 jointype,
+											 outer_bi->strategy,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
+			}
 
 			/* Move to the next datum on the outer side. */
-			Assert(!finished_outer);
 			cnto++;
 		}
 		else
 		{
-			bool		missing_side_outer;
-			bool		missing_side_inner;
-
 			Assert(cmpval > 0);
+			Assert(cnti < inner_bi->ndatums);
 
 			/* A datum missing from the outer side. */
-			merged_index = -1;
 			merged_datum = idatums;
 
-			/*
-			 * For a FULL join, outer relation acts as both OUTER and INNER
-			 * relation.  For LEFT and ANTI join the outer relation acts as
-			 * OUTER relation. For INNER and SEMI join OUTER and INNER
-			 * differentiation is immaterial.
-			 */
-			missing_side_outer = (jointype == JOIN_FULL ||
-								  jointype == JOIN_LEFT ||
-								  jointype == JOIN_ANTI);
-			missing_side_inner = (jointype == JOIN_FULL);
-
-			if (!handle_missing_partition(outer_maps,
-										  inner_maps,
-										  outer_default,
-										  i_index,
-										  missing_side_outer,
-										  missing_side_inner,
-										  &next_index,
-										  &default_index,
-										  &merged_index))
-				return NULL;
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				int			i_index = inner_bi->indexes[cnti];
+
+				Assert(i_index >= 0);
+				if (!process_inner_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 i_index,
+											 outer_default,
+											 jointype,
+											 outer_bi->strategy,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
+			}
 
-			/* Move to the next datum on the right side. */
-			Assert(!finished_inner);
+			/* Move to the next datum on the inner side. */
 			cnti++;
 		}
 
@@ -4058,22 +3924,43 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 		}
 	}
 
-	if (!merge_null_partitions(outer_bi, inner_bi,
-							   outer_maps, inner_maps,
-							   jointype, &next_index, &null_index,
-							   &default_index))
-		return NULL;
+	/* Merge null partitions if any. */
+	if (outer_has_null || inner_has_null)
+	{
+		if (!merge_null_partitions(outer_bi, inner_bi,
+								   &outer_map, &inner_map,
+								   outer_has_default, inner_has_default,
+								   outer_has_null, inner_has_null,
+								   jointype, &next_index, &default_index,
+								   &null_index))
+			return NULL;
+	}
+	else
+		Assert(null_index == -1);
 
-	if (!merge_default_partitions(outer_bi, inner_bi,
-								  outer_maps, inner_maps,
-								  jointype, &next_index,
-								  &default_index))
-		return NULL;
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+	{
+		if (!merge_default_partitions(&outer_map, &inner_map,
+									  outer_has_default, outer_default,
+									  inner_has_default, inner_default,
+									  jointype, &next_index, &default_index))
+			return NULL;
+	}
+	else
+		Assert(default_index == -1);
+
+	/* Fix the merged_indexes list if necessary. */
+	if (outer_map.did_remapping || inner_map.did_remapping)
+	{
+		Assert(jointype == JOIN_FULL);
+		fix_merged_indexes(&outer_map, &inner_map,
+						   next_index, merged_indexes);
+	}
 
 	/* Use maps to match partition from the joining relations. */
 	generate_matching_part_pairs(outer_rel, inner_rel,
-								 outer_maps, inner_maps,
-								 outer_nparts, inner_nparts,
+								 &outer_map, &inner_map,
 								 next_index,
 								 outer_parts, inner_parts);
 
@@ -4081,7 +3968,7 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 	if (*outer_parts && *inner_parts)
 	{
 		Assert(list_length(*outer_parts) == list_length(*inner_parts));
-		Assert(list_length(*outer_parts) == next_index);
+		Assert(list_length(*outer_parts) <= next_index);
 		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
 													  merged_datums,
 													  merged_indexes, NIL,
@@ -4089,6 +3976,8 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 	}
 
 	/* Free up all extra memory before returning from this function. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
 	list_free(merged_datums);
 	list_free(merged_indexes);
 
@@ -4096,132 +3985,400 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 }
 
 /*
- * map_and_merge_partitions
- *
- * If the two given partitions (given by index1 and index2 resp.) are
- * already mapped to each other return the index of corresponding partition in
- * the merged set of partitions.  If they do not have a merged partition
- * associated with them, assign a new merged partition index.  If the
- * partitions are already mapped and their mapped partitions are different from
- * each other, they can not be merged, so return -1.
- *
- * partmaps1[i] gives the mapping of partitions for both relations. It
- * describes which partition of relation 2 matches ith partition of relation 1,
- * and which partition in the merged set matches ith partition of relation 1
- * maps to. Similarly for partmap2.
+ * init_partition_map
  *
- * index1 and index2 are the indexes of matching partition from respective
- * relations.
- *
- * *next_index is used and incremented when the given partitions require a new
- * merged partition.
+ * Initialize a PartitionMap struct for given relation.
  */
-
-static int
-map_and_merge_partitions(PartitionMap *partmaps1, PartitionMap *partmaps2,
-						 int index1, int index2, int *next_index)
+static void
+init_partition_map(RelOptInfo *rel, PartitionMap *map)
 {
-	PartitionMap 	*partmap1 = &partmaps1[index1];
-	PartitionMap 	*partmap2 = &partmaps2[index2];
-	int				merged_index;
+	int			nparts = rel->nparts;
+	int			i;
 
-	/*
-	 * If both the partitions are not mapped to each other, update the
-	 * maps.
-	 */
-	if (partmap1->from < 0 && partmap2->from < 0)
+	map->nparts = nparts;
+	map->merged_indexes = (int *) palloc(sizeof(int) * nparts);
+	map->merged = (bool *) palloc(sizeof(bool) * nparts);
+	map->did_remapping = false;
+	map->old_indexes = (int *) palloc(sizeof(int) * nparts);
+	for (i = 0; i < nparts; i++)
 	{
-		partmap1->from = index2;
-		partmap2->from = index1;
+		map->merged_indexes[i] = map->old_indexes[i] = -1;
+		map->merged[i] = false;
 	}
+}
+
+/*
+ * free_partition_map
+ */
+static void
+free_partition_map(PartitionMap *map)
+{
+	pfree(map->merged_indexes);
+	pfree(map->merged);
+	pfree(map->old_indexes);
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given partitions.
+ */
+static int
+map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_index, int inner_index, int *next_index)
+{
+	int 		outer_merged_index;
+	bool 		outer_merged;
+	int 		inner_merged_index;
+	bool 		inner_merged;
+
+	Assert(outer_index >= 0 && outer_index < outer_map->nparts);
+	outer_merged_index = outer_map->merged_indexes[outer_index];
+	outer_merged = outer_map->merged[outer_index];
+	Assert(inner_index >= 0 && inner_index < inner_map->nparts);
+	inner_merged_index = inner_map->merged_indexes[inner_index];
+	inner_merged = inner_map->merged[inner_index];
 
 	/*
-	 * If the given to partitions map to each other, find the corresponding
-	 * merged partition index .
+	 * Handle cases where both partitions are mapped to merged partitions.
 	 */
-	if (partmap1->from == index2 && partmap2->from == index1)
+	if (outer_merged_index >= 0 && inner_merged_index >= 0)
 	{
 		/*
-		 * If both the partitions are mapped to the same merged partition, get
-		 * the index of merged partition.
+		 * If the mereged partitions are the same, no need to do anything;
+		 * return the index of the merged partition.  Otherwise, if both
+		 * partitions are merged with dummy partitions, re-merge them; map
+		 * them to the merged partition with the smaller of the two merged
+		 * indexes and return the smaller index.  Otherwise they can't be
+		 * merged, so return -1.
 		 */
-		if (partmap1->to == partmap2->to)
+		if (outer_merged_index == inner_merged_index)
+		{
+			Assert(outer_merged);
+			Assert(inner_merged);
+			return outer_merged_index;
+		}
+		if (!outer_merged && !inner_merged)
 		{
-			merged_index = partmap1->to;
-
 			/*
-			 * If the given two partitions do not have a merged partition
-			 * associated with them, allocate a new merged partition.
+			 * Note that we will fix the larger index that have been added to
+			 * the merged_indexes list so far in fix_merged_indexes().
 			 */
-			if (merged_index < 0)
+			if (outer_merged_index < inner_merged_index)
 			{
-				merged_index = *next_index;
-				*next_index = *next_index + 1;
-				partmap1->to = merged_index;
-				partmap2->to = merged_index;
+				outer_map->merged[outer_index] = true;
+				inner_map->merged_indexes[inner_index] = outer_merged_index;
+				inner_map->merged[inner_index] = true;
+				inner_map->did_remapping = true;
+				inner_map->old_indexes[inner_index] = inner_merged_index;
+				return outer_merged_index;
+			}
+			else
+			{
+				inner_map->merged[inner_index] = true;
+				outer_map->merged_indexes[outer_index] = inner_merged_index;
+				outer_map->merged[outer_index] = true;
+				outer_map->did_remapping = true;
+				outer_map->old_indexes[outer_index] = outer_merged_index;
+				return inner_merged_index;
 			}
 		}
+		return -1;
+	}
+
+	/* At least one partition isn't mapped to a merged partition. */
+	Assert(outer_merged_index == -1 || inner_merged_index == -1);
 
+	/*
+	 * If neither of partitions isn't mapped, assign them a new merged
+	 * partition and return the index of the merged partition.  Otherwise, if
+	 * one of partitions is merged with a dummy relation (and the other isn't
+	 * merged), re-merge it with the other, with the same index, and return
+	 * the index.  Otherwise they can't be merged, so return -1.
+	 */
+	if (outer_merged_index == -1 && inner_merged_index == -1)
+	{
+		int 		merged_index = *next_index;
+
+		Assert(!outer_merged);
+		Assert(!inner_merged);
+		outer_map->merged_indexes[outer_index] = merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged_indexes[inner_index] = merged_index;
+		inner_map->merged[inner_index] = true;
+		*next_index = *next_index + 1;
+		return merged_index;
+	}
+	if (outer_merged_index >= 0 && !outer_map->merged[outer_index])
+	{
+		Assert(inner_merged_index == -1);
+		Assert(!inner_merged);
+		inner_map->merged_indexes[inner_index] = outer_merged_index;
+		inner_map->merged[inner_index] = true;
+		outer_map->merged[outer_index] = true;
+		return outer_merged_index;
+	}
+	if (inner_merged_index >= 0 && !inner_map->merged[inner_index])
+	{
+		Assert(outer_merged_index == -1);
+		Assert(!outer_merged);
+		outer_map->merged_indexes[outer_index] = inner_merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged[inner_index] = true;
+		return inner_merged_index;
+	}
+	return -1;
+}
+
+/*
+ * merge_partition_with_dummy
+ *
+ * *next_index is incremented.
+ */
+static int
+merge_partition_with_dummy(PartitionMap *map, int index, int *next_index)
+{
+	int 		merged_index = *next_index;
+
+	Assert(index >= 0 && index < map->nparts);
+	Assert(map->merged_indexes[index] == -1);
+	Assert(!map->merged[index]);
+	map->merged_indexes[index] = merged_index;
+	/* Leave the merged flag alone! */
+	*next_index = *next_index + 1;
+	return merged_index;
+}
+
+/*
+ * process_outer_partition
+ *
+ * Determine the merged partition associated with the given outer partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given outer partition.
+ */
+static bool
+process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index)
+{
+	/*
+	 * If the inner side has the default partition, the outer partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an outer join, in which case the outer partition has to be
+	 * scanned all the way anyway; if the outer partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the outer partition with a dummy partition.
+	 */
+	if (inner_has_default)
+	{
 		/*
-		 * If partition from one relation was mapped to a merged partition but
-		 * not the partition from the other relation, map the same merged
-		 * partition to the partition from other relation, since matching
-		 * partitions map to the same merged partition.
+		 * If the outer side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
 		 */
-		else if (partmap1->to >= 0 && partmap2->to < 0)
+		if (outer_has_default)
+			return false;
+
+		*merged_index = map_and_merge_partitions(outer_map, inner_map,
+												 outer_index, inner_default,
+												 next_index);
+		if (*merged_index == -1)
+			return false;
+
+		/*
+		 * If this is a FULL join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (jointype == JOIN_FULL)
 		{
-			partmap2->to = partmap1->to;
-			merged_index = partmap1->to;
+			if (*default_index == -1)
+				*default_index = *merged_index;
+			else
+				Assert(*merged_index == *default_index);
+			/* Don't add this index to the list of merged indexes. */
+			*merged_index = -1;
 		}
-		else if (partmap1->to < 0 && partmap2->to >= 0)
+	}
+	else
+	{
+		Assert(jointype == JOIN_LEFT || jointype == JOIN_ANTI ||
+			   jointype == JOIN_FULL);
+
+		/*
+		 * In range partitioning, if the given outer partition is already
+		 * merged (eg, because we found an overlapping range earlier), we know
+		 * where it fits in the join result; nothing to do in that case.  Else
+		 * create a new merged partition.
+		 */
+		if (outer_map->merged_indexes[outer_index] >= 0)
 		{
-			partmap1->to = partmap2->to;
-			merged_index = partmap2->to;
+			if (strategy == PARTITION_STRATEGY_LIST)
+				*merged_index = outer_map->merged_indexes[outer_index];
+			else
+			{
+				Assert(strategy == PARTITION_STRATEGY_RANGE);
+				*merged_index = -1;
+			}
 		}
 		else
-		{
-			Assert(partmap1->to != partmap2->to &&
-				   partmap1->to >= 0 && partmap2->to >= 0);
+			*merged_index = merge_partition_with_dummy(outer_map, outer_index,
+													   next_index);
+	}
+	return true;
+}
 
-			/*
-			 * Both the partitions map to different merged partitions. This
-			 * means that multiple partitions from one relation matches to one
-			 * partition from the other relation. Partition-wise join does not
-			 * handle this case right now, since it requires ganging multiple
-			 * partitions together (into one RelOptInfo).
-			 */
-			merged_index = -1;
+/*
+ * process_inner_partition
+ *
+ * Determine the merged partition associated with the given inner partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given outer partition.
+ */
+static bool
+process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index)
+{
+	/*
+	 * If the outer side has the default partition, the inner partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an FULL join, in which case the inner partition has to be
+	 * scanned all the way anyway; if the inner partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the inner partition with a dummy partition.
+	 */
+	if (outer_has_default)
+	{
+		/*
+		 * If the inner side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (inner_has_default)
+			return false;
+
+		*merged_index = map_and_merge_partitions(outer_map, inner_map,
+												 outer_default, inner_index,
+												 next_index);
+		if (*merged_index == -1)
+			return false;
+
+		/*
+		 * If this is an outer join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (jointype == JOIN_LEFT || jointype == JOIN_ANTI ||
+			jointype == JOIN_FULL)
+		{
+			if (*default_index == -1)
+				*default_index = *merged_index;
+			else
+				Assert(*merged_index == *default_index);
+			/* Don't add this index to the list of merged indexes. */
+			*merged_index = -1;
 		}
 	}
 	else
 	{
+		Assert(jointype == JOIN_FULL);
+
 		/*
-		 * Multiple partitions from one relation map to one partition from the
-		 * other relation. Partition-wise join does not handle this case right
-		 * now, since it requires ganging multiple partitions together (into
-		 * one RelOptInfo).
+		 * In range partitioning, if the given outer partition is already
+		 * merged (eg, because we found an overlapping range earlier), we know
+		 * where it fits in the join result; nothing to do in that case.  Else
+		 * create a new merged partition.
 		 */
-		merged_index = -1;
+		if (inner_map->merged_indexes[inner_index] >= 0)
+		{
+			if (strategy == PARTITION_STRATEGY_LIST)
+				*merged_index = inner_map->merged_indexes[inner_index];
+			else
+			{
+				Assert(strategy == PARTITION_STRATEGY_RANGE);
+				*merged_index = -1;
+			}
+		}
+		else
+			*merged_index = merge_partition_with_dummy(inner_map, inner_index,
+													   next_index);
 	}
+	return true;
+}
 
-	return merged_index;
+/*
+ * fix_merged_indexes
+ */
+static void
+fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes)
+{
+	int		   *new_indexes;
+	int			merged_index;
+	int			i;
+	ListCell   *lc;
+
+	new_indexes = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		new_indexes[i] = -1;
+
+	/* Build the mapping of old merged indexes to new merged indexes. */
+	if (outer_map->did_remapping)
+	{
+		for (i = 0; i < outer_map->nparts; i++)
+		{
+			merged_index = outer_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = outer_map->merged_indexes[i];
+		}
+	}
+	if (inner_map->did_remapping)
+	{
+		for (i = 0; i < inner_map->nparts; i++)
+		{
+			merged_index = inner_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = inner_map->merged_indexes[i];
+		}
+	}
+
+	/* Fix the merged_indexes list using the mapping. */
+	foreach(lc, merged_indexes)
+	{
+		merged_index = lfirst_int(lc);
+		Assert(merged_index >= 0);
+		if (new_indexes[merged_index] >= 0)
+			lfirst_int(lc) = new_indexes[merged_index];
+	}
+
+	pfree(new_indexes);
 }
 
 /*
  * generate_matching_part_pairs
  *
- * partmaps1 map each partition from either side of the join to a merged
- * partition resp. E.g. partmaps1[i].to gives the merged partition to which ith
- * partition of first relation maps. Similarly for partmap2. If
- * partmaps1[i].to == partmaps2[j].to, i and j form the matching pair of
- * partitions.
- *
- * Given these maps this function produces the list pairs of partitions which
- * when joined produce the merged partitions in the order of merged partition
- * indexes.
- *
- * nparts1 and nparts2 are the number of partitions of the joining relations
- * resp.
+ * This function produces the list pairs of partitions which produce merged
+ * partitions in the order of merged partition indexes.
  *
  * nparts is the number of merged partitions.
  *
@@ -4231,15 +4388,20 @@ map_and_merge_partitions(PartitionMap *partmaps1, PartitionMap *partmaps2,
  */
 static void
 generate_matching_part_pairs(RelOptInfo *rel1, RelOptInfo *rel2,
-							 PartitionMap *partmaps1, PartitionMap *partmaps2,
-							 int nparts1, int nparts2, int nparts,
+							 PartitionMap *map1, PartitionMap *map2,
+							 int nparts,
 							 List **matched_parts1, List **matched_parts2)
 {
+	int			nparts1 = map1->nparts;
+	int			nparts2 = map2->nparts;
 	int		   *matching1,
 			   *matching2;
 	int 		i;
 	int			max_nparts;
 
+	Assert(map1->nparts == rel1->nparts);
+	Assert(map2->nparts == rel2->nparts);
+
 	*matched_parts1 = NIL;
 	*matched_parts2 = NIL;
 
@@ -4254,23 +4416,22 @@ generate_matching_part_pairs(RelOptInfo *rel1, RelOptInfo *rel2,
 	{
 		if (i < nparts1)
 		{
-			PartitionMap outer_map = partmaps1[i];
+			int 		merged_index = map1->merged_indexes[i];
 
-			if (outer_map.to >= 0)
+			if (merged_index >= 0)
 			{
-				Assert(outer_map.to < nparts);
-				matching1[outer_map.to] = i;
+				Assert(merged_index < nparts);
+				matching1[merged_index] = i;
 			}
 		}
-
 		if (i < nparts2)
 		{
-			PartitionMap inner_map = partmaps2[i];
+			int 		merged_index = map2->merged_indexes[i];
 
-			if (inner_map.to >= 0)
+			if (merged_index >= 0)
 			{
-				Assert(inner_map.to < nparts);
-				matching2[inner_map.to] = i;
+				Assert(merged_index < nparts);
+				matching2[merged_index] = i;
 			}
 		}
 	}
@@ -4280,8 +4441,8 @@ generate_matching_part_pairs(RelOptInfo *rel1, RelOptInfo *rel2,
 		int			part1 = matching1[i];
 		int			part2 = matching2[i];
 
-		/* At least one of the partitions should exist. */
-		Assert(part1 >= 0 || part2 >= 0);
+		if (part1 == -1 && part2 == -1)
+			continue;
 
 		*matched_parts1 = lappend(*matched_parts1,
 								  part1 >= 0 ? rel1->part_rels[part1] : NULL);
@@ -4357,57 +4518,53 @@ build_merged_partition_bounds(char strategy, List *merged_datums,
  * appear in the join result, so create a default merged partition.
  */
 static bool
-merge_default_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
-						 PartitionMap *outer_maps, PartitionMap *inner_maps,
-						 JoinType jointype, int *next_index, int *default_index)
+merge_default_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 bool outer_has_default, int outer_default,
+						 bool inner_has_default, int inner_default,
+						 JoinType jointype, int *next_index,
+						 int *default_index)
 {
-	int				outer_default = outer_bi->default_index;
-	int				inner_default = inner_bi->default_index;
-	bool			outer_has_default = partition_bound_has_default(outer_bi);
-	bool			inner_has_default = partition_bound_has_default(inner_bi);
-	bool			merged = true;
-	PartitionMap 	*outer_default_map = NULL;
-	PartitionMap 	*inner_default_map = NULL;
+	Assert(outer_has_default || inner_has_default);
 
-	if (outer_has_default)
-		outer_default_map = &outer_maps[outer_default];
-
-	if (inner_has_default)
-		inner_default_map = &inner_maps[inner_default];
-
-	if (!outer_has_default && !inner_has_default)
-		Assert(*default_index < 0);
-	else if (outer_default_map != NULL && inner_default_map == NULL)
+	if (outer_has_default && !inner_has_default)
 	{
 		if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
 			jointype == JOIN_ANTI)
 		{
-			if (outer_default_map->to < 0)
+			int			merged_index;
+
+			Assert(outer_default >= 0 && outer_default < outer_map->nparts);
+			merged_index = outer_map->merged_indexes[outer_default];
+			if (merged_index == -1)
 			{
-				outer_default_map->to = *next_index;
-				*next_index = *next_index + 1;
-				Assert(*default_index < 0);
-				*default_index = outer_default_map->to;
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(outer_map,
+															outer_default,
+															next_index);
 			}
 			else
-				Assert(*default_index == outer_default_map->to);
+				Assert(*default_index == merged_index);
 		}
 		else
 			Assert(*default_index < 0);
 	}
-	else if (outer_default_map == NULL && inner_default_map != NULL)
+	else if (!outer_has_default && inner_has_default)
 	{
 		if (jointype == JOIN_FULL)
 		{
-			if (inner_default_map->to < 0)
+			int			merged_index;
+
+			Assert(inner_default >= 0 && inner_default < inner_map->nparts);
+			merged_index = inner_map->merged_indexes[inner_default];
+			if (merged_index == -1)
 			{
-				inner_default_map->to = *next_index;
-				*next_index = *next_index + 1;
-				Assert(*default_index < 0);
-				*default_index = inner_default_map->to;
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(inner_map,
+															inner_default,
+															next_index);
 			}
 			else
-				Assert(*default_index == inner_default_map->to);
+				Assert(*default_index == merged_index);
 		}
 		else
 			Assert(*default_index < 0);
@@ -4416,15 +4573,16 @@ merge_default_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_b
 	{
 		Assert(outer_has_default && inner_has_default);
 
-		*default_index = map_and_merge_partitions(outer_maps, inner_maps,
-												  outer_default, inner_default,
+		*default_index = map_and_merge_partitions(outer_map,
+												  inner_map,
+												  outer_default,
+												  inner_default,
 												  next_index);
-
-		if (*default_index < 0)
-			merged = false;
+		if (*default_index == -1)
+			return false;
 	}
 
-	return merged;
+	return true;
 }
 
 /*
@@ -4444,101 +4602,83 @@ merge_default_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_b
  */
 static bool
 merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
-					  PartitionMap *outer_maps, PartitionMap *inner_maps,
+					  PartitionMap *outer_map, PartitionMap *inner_map,
+					  bool outer_has_default, bool inner_has_default,
+					  bool outer_has_null, bool inner_has_null,
 					  JoinType jointype, int *next_index,
-					  int *null_index, int *default_index)
+					  int *default_index, int *null_index)
 {
-	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
-	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
-	int			outer_ni = outer_bi->null_index;
-	int			inner_ni = inner_bi->null_index;
-	int			outer_default = outer_bi->default_index;
-	int			inner_default = inner_bi->default_index;
-	bool		merged = true;
+	Assert(outer_has_null || inner_has_null);
+	Assert(*null_index == -1);
 
-	if (!outer_has_null && !inner_has_null)
-		Assert(*null_index < 0);
-	else if (outer_has_null && !inner_has_null)
+	if (outer_has_null && !inner_has_null)
 	{
 		int			merged_index = -1;
-		bool		missing_side_outer;
-		bool		missing_side_inner;
-
-		/*
-		 * For a FULL join, inner relation acts as both OUTER and INNER
-		 * relation.  For LEFT and ANTI join the inner relation acts as
-		 * INNER relation. For INNER and SEMI join OUTER and INNER
-		 * differentiation is immaterial.
-		 */
-		missing_side_inner = (jointype == JOIN_FULL ||
-							  jointype == JOIN_LEFT ||
-							  jointype == JOIN_ANTI);
-		missing_side_outer = (jointype == JOIN_FULL);
-
-		merged = handle_missing_partition(inner_maps,
-										  outer_maps,
-										  inner_default,
-										  outer_ni,
-										  missing_side_outer,
-										  missing_side_inner, next_index,
-										  default_index, &merged_index);
-		*null_index = merged_index;
 
 		/*
 		 * If the NULL partition was missing from the inner side of the join,
-		 * the partition of the join to which the outer null partition maps
-		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the partition of the join to which the NULL partition matches will
+		 * contain the NULL values and thus become the NULL partition of the
 		 * the join.
 		 */
-		if (missing_side_inner)
-			*null_index = outer_maps[outer_ni].to;
+		if (inner_has_default ||
+			jointype == JOIN_LEFT ||
+			jointype == JOIN_ANTI ||
+			jointype == JOIN_FULL)
+		{
+			if (!process_outer_partition(outer_map,
+										 inner_map,
+										 outer_has_default,
+										 inner_has_default,
+										 outer_bi->null_index,
+										 inner_bi->default_index,
+										 jointype,
+										 outer_bi->strategy,
+										 next_index,
+										 default_index,
+										 &merged_index))
+				return false;
+		}
+		*null_index = merged_index;
 	}
 	else if (!outer_has_null && inner_has_null)
 	{
 		int			merged_index = -1;
-		bool		missing_side_outer;
-		bool		missing_side_inner;
-
-		/*
-		 * For a FULL join, outer relation acts as both OUTER and INNER
-		 * relation.  For LEFT and ANTI join the outer relation acts as OUTER
-		 * relation. For INNER and SEMI join OUTER and INNER differentiation is
-		 * immaterial.
-		 */
-		missing_side_outer = (jointype == JOIN_FULL ||
-							  jointype == JOIN_LEFT ||
-							  jointype == JOIN_ANTI);
-		missing_side_inner = (jointype == JOIN_FULL);
-		merged = handle_missing_partition(outer_maps,
-										  inner_maps,
-										  outer_default,
-										  inner_ni,
-										  missing_side_outer,
-										  missing_side_inner,
-										  next_index, default_index,
-										  &merged_index);
-		*null_index = merged_index;
 
 		/*
 		 * If the NULL partition was missing from the inner side of the join,
-		 * the partition of the join, to which the outer side null partition maps,
-		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the partition of the join to which the NULL partition matches will
+		 * contain the NULL values and thus become the NULL partition of the
 		 * the join.
 		 */
-		if (missing_side_inner)
-			*null_index = inner_maps[inner_ni].to;
+		if (outer_has_default || jointype == JOIN_FULL)
+		{
+			if (!process_inner_partition(outer_map,
+										 inner_map,
+										 outer_has_default,
+										 inner_has_default,
+										 inner_bi->null_index,
+										 outer_bi->default_index,
+										 jointype,
+										 outer_bi->strategy,
+										 next_index,
+										 default_index,
+										 &merged_index))
+				return false;
+		}
+		*null_index = merged_index;
 	}
 	else
 	{
 		/* Both the relations have NULL partitions, try merging them. */
-		*null_index = map_and_merge_partitions(outer_maps,
-											   inner_maps,
-											   outer_ni,
-											   inner_ni,
+		*null_index = map_and_merge_partitions(outer_map,
+											   inner_map,
+											   outer_bi->null_index,
+											   inner_bi->null_index,
 											   next_index);
-		if (*null_index < 0)
-			merged = false;
+		if (*null_index == -1)
+			return false;
 	}
 
-	return merged;
+	return true;
 }
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index b5b5c8a260..89f0c6a9ee 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -4769,3 +4769,182 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
                            Filter: (b = 0)
 (23 rows)
 
+DROP TABLE plt1;
+DROP TABLE plt2;
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
+ANALYZE plt1;
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+ANALYZE plt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.c = t2.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt1_p1 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t2_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_1
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+  0 | 0000 |    | 
+  5 | 0000 |    | 
+ 10 | 0000 |    | 
+ 15 | 0000 |    | 
+ 20 | 0000 |    | 
+  2 | 0002 |  2 | 0002
+  2 | 0002 |  7 | 0002
+  2 | 0002 | 12 | 0002
+  2 | 0002 | 17 | 0002
+  2 | 0002 | 22 | 0002
+  7 | 0002 |  2 | 0002
+  7 | 0002 |  7 | 0002
+  7 | 0002 | 12 | 0002
+  7 | 0002 | 17 | 0002
+  7 | 0002 | 22 | 0002
+ 12 | 0002 |  2 | 0002
+ 12 | 0002 |  7 | 0002
+ 12 | 0002 | 12 | 0002
+ 12 | 0002 | 17 | 0002
+ 12 | 0002 | 22 | 0002
+ 17 | 0002 |  2 | 0002
+ 17 | 0002 |  7 | 0002
+ 17 | 0002 | 12 | 0002
+ 17 | 0002 | 17 | 0002
+ 17 | 0002 | 22 | 0002
+ 22 | 0002 |  2 | 0002
+ 22 | 0002 |  7 | 0002
+ 22 | 0002 | 12 | 0002
+ 22 | 0002 | 17 | 0002
+ 22 | 0002 | 22 | 0002
+    |      |  1 | 0001
+    |      |  6 | 0001
+    |      | 11 | 0001
+    |      | 16 | 0001
+    |      | 21 | 0001
+(35 rows)
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1;
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2;
+CREATE TABLE plt3 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt3_p1 PARTITION OF plt3 FOR VALUES IN ('0001');
+CREATE TABLE plt3_p2 PARTITION OF plt3 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2.a, t3.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.c = t3.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1.c = t2.c)
+                     ->  Seq Scan on plt1_p1 t1
+                     ->  Hash
+                           ->  Seq Scan on plt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on plt3_p1 t3
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t3_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_1.c = t2_1.c)
+                     ->  Seq Scan on plt1_p2 t1_1
+                     ->  Hash
+                           ->  Seq Scan on plt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt3_p2 t3_1
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+ a  |  c   | a  |  c   | a  |  c   
+----+------+----+------+----+------
+  0 | 0000 |    |      |    | 
+  5 | 0000 |    |      |    | 
+ 10 | 0000 |    |      |    | 
+ 15 | 0000 |    |      |    | 
+ 20 | 0000 |    |      |    | 
+  1 | 0001 |    |      |  1 | 0001
+  1 | 0001 |    |      |  6 | 0001
+  1 | 0001 |    |      | 11 | 0001
+  1 | 0001 |    |      | 16 | 0001
+  1 | 0001 |    |      | 21 | 0001
+  6 | 0001 |    |      |  1 | 0001
+  6 | 0001 |    |      |  6 | 0001
+  6 | 0001 |    |      | 11 | 0001
+  6 | 0001 |    |      | 16 | 0001
+  6 | 0001 |    |      | 21 | 0001
+ 11 | 0001 |    |      |  1 | 0001
+ 11 | 0001 |    |      |  6 | 0001
+ 11 | 0001 |    |      | 11 | 0001
+ 11 | 0001 |    |      | 16 | 0001
+ 11 | 0001 |    |      | 21 | 0001
+ 16 | 0001 |    |      |  1 | 0001
+ 16 | 0001 |    |      |  6 | 0001
+ 16 | 0001 |    |      | 11 | 0001
+ 16 | 0001 |    |      | 16 | 0001
+ 16 | 0001 |    |      | 21 | 0001
+ 21 | 0001 |    |      |  1 | 0001
+ 21 | 0001 |    |      |  6 | 0001
+ 21 | 0001 |    |      | 11 | 0001
+ 21 | 0001 |    |      | 16 | 0001
+ 21 | 0001 |    |      | 21 | 0001
+  2 | 0002 |  2 | 0002 |    | 
+  2 | 0002 |  7 | 0002 |    | 
+  2 | 0002 | 12 | 0002 |    | 
+  2 | 0002 | 17 | 0002 |    | 
+  2 | 0002 | 22 | 0002 |    | 
+  7 | 0002 |  2 | 0002 |    | 
+  7 | 0002 |  7 | 0002 |    | 
+  7 | 0002 | 12 | 0002 |    | 
+  7 | 0002 | 17 | 0002 |    | 
+  7 | 0002 | 22 | 0002 |    | 
+ 12 | 0002 |  2 | 0002 |    | 
+ 12 | 0002 |  7 | 0002 |    | 
+ 12 | 0002 | 12 | 0002 |    | 
+ 12 | 0002 | 17 | 0002 |    | 
+ 12 | 0002 | 22 | 0002 |    | 
+ 17 | 0002 |  2 | 0002 |    | 
+ 17 | 0002 |  7 | 0002 |    | 
+ 17 | 0002 | 12 | 0002 |    | 
+ 17 | 0002 | 17 | 0002 |    | 
+ 17 | 0002 | 22 | 0002 |    | 
+ 22 | 0002 |  2 | 0002 |    | 
+ 22 | 0002 |  7 | 0002 |    | 
+ 22 | 0002 | 12 | 0002 |    | 
+ 22 | 0002 | 17 | 0002 |    | 
+ 22 | 0002 | 22 | 0002 |    | 
+(55 rows)
+
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 34ae92135f..b0ab002eee 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -800,3 +800,47 @@ ANALYZE prt2;
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
+ANALYZE plt1;
+
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+ANALYZE plt2;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1;
+
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2;
+
+CREATE TABLE plt3 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt3_p1 PARTITION OF plt3 FOR VALUES IN ('0001');
+CREATE TABLE plt3_p2 PARTITION OF plt3 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
#98amul sul
sulamul@gmail.com
In reply to: Etsuro Fujita (#97)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Wed, Oct 16, 2019 at 6:20 PM Etsuro Fujita <etsuro.fujita@gmail.com>
wrote:

On Wed, Sep 25, 2019 at 12:59 AM Etsuro Fujita <etsuro.fujita@gmail.com>
wrote:

I will continue to review the rest of the patch.

I've been reviewing the rest of the patch. Here are my review comments:

[....]

So I'd like to propose to introduce separate functions like
process_outer_partition() and process_inner_partition() in the
attached, instead of handle_missing_partition(). (I added a fast path
to these functions that if both outer/inner sides have the default
partitions, give up on merging partitions. Also, I modified the
caller sites of proposed functions so that they call these if
necessary.)

Agree -- process_outer_partition() & process_inner_partition() approach
looks
much cleaner than before.

Here are the few comments:

Note that LHS numbers are the line numbers in your previously posted
patch[1].

455 + if (inner_has_default ||
456 + jointype == JOIN_LEFT ||
457 + jointype == JOIN_ANTI ||
458 + jointype == JOIN_FULL)
459 + {
460 + if (!process_outer_partition(&outer_map,

512 + if (outer_has_default || jointype == JOIN_FULL)
513 + {
514 + if (!process_inner_partition(&outer_map,

How about adding these conditions to the else block of
process_outer_partition()
& process_inner_partition() function respectively so that these functions
can be
called unconditionally? Thoughts/Comments?
---

456 + jointype == JOIN_LEFT ||
457 + jointype == JOIN_ANTI ||
458 + jointype == JOIN_FULL)

Also, how about using IS_OUTER_JOIN() instead. But we need an assertion to
restrict JOIN_RIGHT or something.

For the convenience, I did both aforesaid changes in the attached delta
patch that
can be applied atop of your previously posted patch[1]. Kindly have a look
& share
your thoughts, thanks.
--

1273 + * *next_index is incremented when creating a new merged partition
associated
1274 + * with the given outer partition.
1275 + */

Correction: s/outer/inner
---

1338 + * In range partitioning, if the given outer partition is
already
1339 + * merged (eg, because we found an overlapping range earlier),
we know
1340 + * where it fits in the join result; nothing to do in that
case. Else
1341 + * create a new merged partition.

Correction: s/outer/inner.
---

1712 /*
1713 * If the NULL partition was missing from the inner side of
the join,

s/inner side/outer side
--

Regards,
Amul

1]
/messages/by-id/CAPmGK145V8DNCNQ2gQBgNE3QqoJGjsmK5WMwaA3FMirNM723KQ@mail.gmail.com

Attachments:

delta.patchapplication/x-patch; name=delta.patchDownload
From c7f165b575fd984ca3053ce7162bdd8e4bf56be8 Mon Sep 17 00:00:00 2001
From: Amul Sul <amul.sul@enterprisedb.com>
Date: Thu, 24 Oct 2019 05:38:11 -0400
Subject: [PATCH] delta

Changes :
1.  Call process_outer_partition & process_inner_partition
unconditionally.
2. Used IS_OUTER_JOIN() instead of listing individual join type.
---
 src/backend/partitioning/partbounds.c | 209 ++++++++++++--------------
 1 file changed, 93 insertions(+), 116 deletions(-)

diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 4c3a31c82c0..b29e44e2f28 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -3601,24 +3601,18 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 			}
 			else
 			{
-				if (inner_has_default ||
-					jointype == JOIN_LEFT ||
-					jointype == JOIN_ANTI ||
-					jointype == JOIN_FULL)
-				{
-					if (!process_outer_partition(&outer_map,
-												 &inner_map,
-												 outer_has_default,
-												 inner_has_default,
-												 outer_part,
-												 inner_default,
-												 jointype,
-												 outer_bi->strategy,
-												 &next_index,
-												 &default_index,
-												 &merged_index))
-						return NULL;
-				}
+				if (!process_outer_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 outer_part,
+											 inner_default,
+											 jointype,
+											 outer_bi->strategy,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
 
 				merged_lb = &outer_lb;
 				merged_ub = &outer_ub;
@@ -3640,21 +3634,18 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 			}
 			else
 			{
-				if (outer_has_default || jointype == JOIN_FULL)
-				{
-					if (!process_inner_partition(&outer_map,
-												 &inner_map,
-												 outer_has_default,
-												 inner_has_default,
-												 inner_part,
-												 outer_default,
-												 jointype,
-												 outer_bi->strategy,
-												 &next_index,
-												 &default_index,
-												 &merged_index))
-						return NULL;
-				}
+				if (!process_inner_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 inner_part,
+											 outer_default,
+											 jointype,
+											 outer_bi->strategy,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
 
 				merged_lb = &inner_lb;
 				merged_ub = &inner_ub;
@@ -3857,27 +3848,18 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 			/* A datum missing from the inner side. */
 			merged_datum = odatums;
 
-			if (inner_has_default ||
-				jointype == JOIN_LEFT ||
-				jointype == JOIN_ANTI ||
-				jointype == JOIN_FULL)
-			{
-				int			o_index = outer_bi->indexes[cnto];
-
-				Assert(o_index >= 0);
-				if (!process_outer_partition(&outer_map,
-											 &inner_map,
-											 outer_has_default,
-											 inner_has_default,
-											 o_index,
-											 inner_default,
-											 jointype,
-											 outer_bi->strategy,
-											 &next_index,
-											 &default_index,
-											 &merged_index))
-					return NULL;
-			}
+			if (!process_outer_partition(&outer_map,
+										 &inner_map,
+										 outer_has_default,
+										 inner_has_default,
+										 outer_bi->indexes[cnto],
+										 inner_default,
+										 jointype,
+										 outer_bi->strategy,
+										 &next_index,
+										 &default_index,
+										 &merged_index))
+				return NULL;
 
 			/* Move to the next datum on the outer side. */
 			cnto++;
@@ -3890,24 +3872,18 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 			/* A datum missing from the outer side. */
 			merged_datum = idatums;
 
-			if (outer_has_default || jointype == JOIN_FULL)
-			{
-				int			i_index = inner_bi->indexes[cnti];
-
-				Assert(i_index >= 0);
-				if (!process_inner_partition(&outer_map,
-											 &inner_map,
-											 outer_has_default,
-											 inner_has_default,
-											 i_index,
-											 outer_default,
-											 jointype,
-											 outer_bi->strategy,
-											 &next_index,
-											 &default_index,
-											 &merged_index))
-					return NULL;
-			}
+			if (!process_inner_partition(&outer_map,
+										 &inner_map,
+										 outer_has_default,
+										 inner_has_default,
+										 inner_bi->indexes[cnti],
+										 outer_default,
+										 jointype,
+										 outer_bi->strategy,
+										 &next_index,
+										 &default_index,
+										 &merged_index))
+				return NULL;
 
 			/* Move to the next datum on the inner side. */
 			cnti++;
@@ -4171,6 +4147,8 @@ process_outer_partition(PartitionMap *outer_map,
 						int *default_index,
 						int *merged_index)
 {
+	Assert(*merged_index == -1);
+
 	/*
 	 * If the inner side has the default partition, the outer partition has to
 	 * be joined with the default partition; try merging them.  Otherwise, we
@@ -4189,6 +4167,7 @@ process_outer_partition(PartitionMap *outer_map,
 		if (outer_has_default)
 			return false;
 
+		Assert(outer_index >= 0);
 		*merged_index = map_and_merge_partitions(outer_map, inner_map,
 												 outer_index, inner_default,
 												 next_index);
@@ -4210,17 +4189,17 @@ process_outer_partition(PartitionMap *outer_map,
 			*merged_index = -1;
 		}
 	}
-	else
+	else if (IS_OUTER_JOIN(jointype))
 	{
-		Assert(jointype == JOIN_LEFT || jointype == JOIN_ANTI ||
-			   jointype == JOIN_FULL);
-
+		Assert (jointype == JOIN_LEFT || jointype == JOIN_ANTI ||
+				jointype == JOIN_FULL);
 		/*
 		 * In range partitioning, if the given outer partition is already
 		 * merged (eg, because we found an overlapping range earlier), we know
 		 * where it fits in the join result; nothing to do in that case.  Else
 		 * create a new merged partition.
 		 */
+		Assert(outer_index >= 0);
 		if (outer_map->merged_indexes[outer_index] >= 0)
 		{
 			if (strategy == PARTITION_STRATEGY_LIST)
@@ -4259,6 +4238,8 @@ process_inner_partition(PartitionMap *outer_map,
 						int *default_index,
 						int *merged_index)
 {
+	Assert(*merged_index == -1);
+
 	/*
 	 * If the outer side has the default partition, the inner partition has to
 	 * be joined with the default partition; try merging them.  Otherwise, we
@@ -4277,6 +4258,7 @@ process_inner_partition(PartitionMap *outer_map,
 		if (inner_has_default)
 			return false;
 
+		Assert(inner_index >= 0);
 		*merged_index = map_and_merge_partitions(outer_map, inner_map,
 												 outer_default, inner_index,
 												 next_index);
@@ -4288,9 +4270,11 @@ process_inner_partition(PartitionMap *outer_map,
 		 * default partition of the join; record the index in *default_index
 		 * if not done yet.
 		 */
-		if (jointype == JOIN_LEFT || jointype == JOIN_ANTI ||
-			jointype == JOIN_FULL)
+		if (IS_OUTER_JOIN(jointype))
 		{
+			Assert(jointype == JOIN_LEFT || jointype == JOIN_ANTI ||
+				   jointype == JOIN_FULL);
+
 			if (*default_index == -1)
 				*default_index = *merged_index;
 			else
@@ -4299,16 +4283,15 @@ process_inner_partition(PartitionMap *outer_map,
 			*merged_index = -1;
 		}
 	}
-	else
+	else if (jointype == JOIN_FULL)
 	{
-		Assert(jointype == JOIN_FULL);
-
 		/*
 		 * In range partitioning, if the given outer partition is already
 		 * merged (eg, because we found an overlapping range earlier), we know
 		 * where it fits in the join result; nothing to do in that case.  Else
 		 * create a new merged partition.
 		 */
+		Assert(inner_index >= 0);
 		if (inner_map->merged_indexes[inner_index] >= 0)
 		{
 			if (strategy == PARTITION_STRATEGY_LIST)
@@ -4528,11 +4511,12 @@ merge_default_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
 
 	if (outer_has_default && !inner_has_default)
 	{
-		if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
-			jointype == JOIN_ANTI)
+		if (IS_OUTER_JOIN(jointype))
 		{
 			int			merged_index;
 
+			Assert (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+					jointype == JOIN_ANTI);
 			Assert(outer_default >= 0 && outer_default < outer_map->nparts);
 			merged_index = outer_map->merged_indexes[outer_default];
 			if (merged_index == -1)
@@ -4621,24 +4605,19 @@ merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
 		 * contain the NULL values and thus become the NULL partition of the
 		 * the join.
 		 */
-		if (inner_has_default ||
-			jointype == JOIN_LEFT ||
-			jointype == JOIN_ANTI ||
-			jointype == JOIN_FULL)
-		{
-			if (!process_outer_partition(outer_map,
-										 inner_map,
-										 outer_has_default,
-										 inner_has_default,
-										 outer_bi->null_index,
-										 inner_bi->default_index,
-										 jointype,
-										 outer_bi->strategy,
-										 next_index,
-										 default_index,
-										 &merged_index))
-				return false;
-		}
+		if (!process_outer_partition(outer_map,
+									 inner_map,
+									 outer_has_default,
+									 inner_has_default,
+									 outer_bi->null_index,
+									 inner_bi->default_index,
+									 jointype,
+									 outer_bi->strategy,
+									 next_index,
+									 default_index,
+									 &merged_index))
+			return false;
+
 		*null_index = merged_index;
 	}
 	else if (!outer_has_null && inner_has_null)
@@ -4651,21 +4630,19 @@ merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
 		 * contain the NULL values and thus become the NULL partition of the
 		 * the join.
 		 */
-		if (outer_has_default || jointype == JOIN_FULL)
-		{
-			if (!process_inner_partition(outer_map,
-										 inner_map,
-										 outer_has_default,
-										 inner_has_default,
-										 inner_bi->null_index,
-										 outer_bi->default_index,
-										 jointype,
-										 outer_bi->strategy,
-										 next_index,
-										 default_index,
-										 &merged_index))
-				return false;
-		}
+		if (!process_inner_partition(outer_map,
+									 inner_map,
+									 outer_has_default,
+									 inner_has_default,
+									 inner_bi->null_index,
+									 outer_bi->default_index,
+									 jointype,
+									 outer_bi->strategy,
+									 next_index,
+									 default_index,
+									 &merged_index))
+			return false;
+
 		*null_index = merged_index;
 	}
 	else
-- 
2.18.0

#99Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: amul sul (#98)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Hi Amul,

On Fri, Oct 25, 2019 at 6:59 PM amul sul <sulamul@gmail.com> wrote:

On Wed, Oct 16, 2019 at 6:20 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

So I'd like to propose to introduce separate functions like
process_outer_partition() and process_inner_partition() in the
attached, instead of handle_missing_partition(). (I added a fast path
to these functions that if both outer/inner sides have the default
partitions, give up on merging partitions. Also, I modified the
caller sites of proposed functions so that they call these if
necessary.)

Agree -- process_outer_partition() & process_inner_partition() approach looks
much cleaner than before.

Here are the few comments:

Thanks for the review!

Note that LHS numbers are the line numbers in your previously posted patch[1].

455 + if (inner_has_default ||
456 + jointype == JOIN_LEFT ||
457 + jointype == JOIN_ANTI ||
458 + jointype == JOIN_FULL)
459 + {
460 + if (!process_outer_partition(&outer_map,

512 + if (outer_has_default || jointype == JOIN_FULL)
513 + {
514 + if (!process_inner_partition(&outer_map,

How about adding these conditions to the else block of process_outer_partition()
& process_inner_partition() function respectively so that these functions can be
called unconditionally? Thoughts/Comments?

I'm not sure that's a good idea; these functions might be called many
times, so I just thought it would be better to call these functions
conditionally, to avoid useless function call overhead.

456 + jointype == JOIN_LEFT ||
457 + jointype == JOIN_ANTI ||
458 + jointype == JOIN_FULL)

Also, how about using IS_OUTER_JOIN() instead. But we need an assertion to
restrict JOIN_RIGHT or something.

Seems like a good idea.

For the convenience, I did both aforesaid changes in the attached delta patch that
can be applied atop of your previously posted patch[1]. Kindly have a look & share
your thoughts, thanks.

Thanks for the patch!

1273 + * *next_index is incremented when creating a new merged partition associated
1274 + * with the given outer partition.
1275 + */

Correction: s/outer/inner
---

1338 + * In range partitioning, if the given outer partition is already
1339 + * merged (eg, because we found an overlapping range earlier), we know
1340 + * where it fits in the join result; nothing to do in that case. Else
1341 + * create a new merged partition.

Correction: s/outer/inner.
---

1712 /*
1713 * If the NULL partition was missing from the inner side of the join,

s/inner side/outer side
--

Good catch! Will fix.

Best regards,
Etsuro Fujita

#100Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Etsuro Fujita (#99)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Tue, Oct 29, 2019 at 7:29 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

On Fri, Oct 25, 2019 at 6:59 PM amul sul <sulamul@gmail.com> wrote:

On Wed, Oct 16, 2019 at 6:20 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

So I'd like to propose to introduce separate functions like
process_outer_partition() and process_inner_partition() in the
attached, instead of handle_missing_partition(). (I added a fast path
to these functions that if both outer/inner sides have the default
partitions, give up on merging partitions. Also, I modified the
caller sites of proposed functions so that they call these if
necessary.)

Agree -- process_outer_partition() & process_inner_partition() approach looks
much cleaner than before.

Note that LHS numbers are the line numbers in your previously posted patch[1].

455 + if (inner_has_default ||
456 + jointype == JOIN_LEFT ||
457 + jointype == JOIN_ANTI ||
458 + jointype == JOIN_FULL)
459 + {
460 + if (!process_outer_partition(&outer_map,

512 + if (outer_has_default || jointype == JOIN_FULL)
513 + {
514 + if (!process_inner_partition(&outer_map,

How about adding these conditions to the else block of process_outer_partition()
& process_inner_partition() function respectively so that these functions can be
called unconditionally? Thoughts/Comments?

I'm not sure that's a good idea; these functions might be called many
times, so I just thought it would be better to call these functions
conditionally, to avoid useless function call overhead.

The overhead might be small, but isn't zero, so I still think that we
should call these functions if necessary.

456 + jointype == JOIN_LEFT ||
457 + jointype == JOIN_ANTI ||
458 + jointype == JOIN_FULL)

Also, how about using IS_OUTER_JOIN() instead. But we need an assertion to
restrict JOIN_RIGHT or something.

Seems like a good idea.

Done.

For the convenience, I did both aforesaid changes in the attached delta patch that
can be applied atop of your previously posted patch[1]. Kindly have a look & share
your thoughts, thanks.

Thanks for the patch!

1273 + * *next_index is incremented when creating a new merged partition associated
1274 + * with the given outer partition.
1275 + */

Correction: s/outer/inner
---

1338 + * In range partitioning, if the given outer partition is already
1339 + * merged (eg, because we found an overlapping range earlier), we know
1340 + * where it fits in the join result; nothing to do in that case. Else
1341 + * create a new merged partition.

Correction: s/outer/inner.
---

1712 /*
1713 * If the NULL partition was missing from the inner side of the join,

s/inner side/outer side
--

Good catch! Will fix.

Done.

I also added some assertions to process_outer_partition() and
process_inner_partition(), including ones as proposed in your patch.
Attached is an updated version. If no objections, I'll merge this
with the main patch [1]/messages/by-id/CAPmGK14WHKckT1P6UJV2B63TZAxPyMn8iZJ99XF=AZuNhG6vow@mail.gmail.com.

Best regards,
Etsuro Fujita

[1]: /messages/by-id/CAPmGK14WHKckT1P6UJV2B63TZAxPyMn8iZJ99XF=AZuNhG6vow@mail.gmail.com

Attachments:

modify-partbounds-changes-2.patchapplication/octet-stream; name=modify-partbounds-changes-2.patchDownload
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 3b2008e0f1..d35c9817cd 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -72,8 +72,13 @@ typedef struct PartitionRangeBound
 
 typedef struct PartitionMap
 {
-	int from;
-	int to;
+	int			nparts;			/* number of partitions */
+	int		   *merged_indexes;	/* indexes of merged partitions */
+	bool	   *merged;			/* flags to indicate whether partitions are
+								 * merged with non-dummy partitions */
+	bool		did_remapping;	/* did we remap partitions? */
+	int		   *old_indexes;	/* old indexes of merged partitions if
+								 * did_remapping */
 } PartitionMap;
 
 static int32 qsort_partition_hbound_cmp(const void *a, const void *b);
@@ -115,20 +120,49 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 									 Expr **keyCol,
 									 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
-static PartitionBoundInfo partition_range_bounds_merge(
+static PartitionBoundInfo partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
 							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
-							 List **outer_parts, List **inner_parts,
-							 JoinType jointype, int partnatts,
-							 FmgrInfo *supfuncs, Oid *collations);
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts);
 static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
 							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
-							List **outer_parts, List **inner_parts,
-							JoinType jointype);
+							JoinType jointype,
+							List **outer_parts, List **inner_parts);
+static void init_partition_map(RelOptInfo *rel, PartitionMap *map);
+static void free_partition_map(PartitionMap *map);
+static int map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_part, int inner_part, int *next_index);
+static int merge_partition_with_dummy(PartitionMap *map, int index,
+						   int *next_index);
+static bool process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index);
+static bool process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index);
+static void fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes);
 static void generate_matching_part_pairs(RelOptInfo *rel1,
 							 RelOptInfo *rel2,
-							 PartitionMap *partmaps1,
-							 PartitionMap *partmaps2,
-							 int nparts1, int nparts2,
+							 PartitionMap *map1,
+							 PartitionMap *map2,
 							 int nparts,
 							 List **matched_parts1,
 							 List **matched_parts2);
@@ -136,9 +170,6 @@ static PartitionBoundInfo build_merged_partition_bounds(char strategy,
 							  List *merged_datums, List *merged_indexes,
 							  List *merged_contents, int null_index,
 							  int default_index);
-static int map_and_merge_partitions(PartitionMap *outer_maps,
-										PartitionMap *inner_maps,
-										int index1, int index2, int *next_index);
 static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
 						  Oid *collations, PartitionRangeBound *bound1,
 						  PartitionRangeBound *bound2);
@@ -153,19 +184,21 @@ static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
 							  PartitionRangeDatumKind *next_lb_kind,
 							  List **merged_datums, List **merged_kinds,
 							  List **merged_indexes);
-static bool merge_default_partitions(PartitionBoundInfo outer_bi,
-						 PartitionBoundInfo inner_bi,
-						 PartitionMap *outer_maps,
-						 PartitionMap *inner_maps,
+static bool merge_default_partitions(PartitionMap *outer_map,
+						 PartitionMap *inner_map,
+						 bool outer_has_default,
+						 int outer_default,
+						 bool inner_has_default,
+						 int inner_default,
 						 JoinType jointype,
-						 int *next_index, int *default_index);
-static bool merge_null_partitions(PartitionBoundInfo outer_bi,
-								  PartitionBoundInfo inner_bi,
-								  PartitionMap *outer_maps,
-								  PartitionMap *inner_maps,
-								  JoinType jointype,
-								  int *next_index, int *null_index,
-								  int *default_index);
+						 int *next_index,
+						 int *default_index);
+static bool merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_map, PartitionMap *inner_map,
+					  bool outer_has_default, bool inner_has_default,
+					  bool outer_has_null, bool inner_has_null,
+					  JoinType jointype, int *next_index,
+					  int *default_index, int *null_index);
 
 /*
  * get_qual_from_partbound
@@ -3081,8 +3114,8 @@ partition_bounds_merge(int partnatts,
 					   int16 *parttyplen, bool *parttypbyval,
 					   FmgrInfo *partsupfunc, Oid *partcollation,
 					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
-					   JoinType jointype,
-					   List **outer_parts, List **inner_parts)
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
 {
 	PartitionBoundInfo 	merged_bounds;
 	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
@@ -3110,17 +3143,22 @@ partition_bounds_merge(int partnatts,
 		case PARTITION_STRATEGY_LIST:
 			merged_bounds = partition_list_bounds_merge(partsupfunc,
 														partcollation,
-														outer_rel, inner_rel,
-														outer_parts, inner_parts,
-														jointype);
+														outer_rel,
+														inner_rel,
+														jointype,
+														outer_parts,
+														inner_parts);
 			break;
 
 		case PARTITION_STRATEGY_RANGE:
-			merged_bounds = partition_range_bounds_merge(outer_rel, inner_rel,
-														 outer_parts, inner_parts,
-														 jointype, partnatts,
+			merged_bounds = partition_range_bounds_merge(partnatts,
 														 partsupfunc,
-														 partcollation);
+														 partcollation,
+														 outer_rel,
+														 inner_rel,
+														 jointype,
+														 outer_parts,
+														 inner_parts);
 			break;
 
 		default:
@@ -3335,8 +3373,6 @@ partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
 		default:
 			elog(ERROR, "unexpected join type %d", jointype);
 	}
-
-	return;
 }
 
 /*
@@ -3396,157 +3432,34 @@ partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
 	return true;
 }
 
-/*
- * handle_missing_partition
- *
- * If a range appears in one of the joining relations but not the other, a row
- * in the corresponding partition will not have any join partner in the other
- * relation, unless the other relation has a default partition. If a given list
- * value is present in one joining relation but not the other, the default
- * partition on the other side may contain that value.
- *
- * In both these cases, such an extra partition forms a joining pair with the
- * default partition, if any,  on the other side.
- *
- * If the default partition happens to be on the outer side of the join, the
- * resultant partition will act as the default partition of the join relation.
- * Otherwise the resultant partition will be associated with the range.
- *
- * When the default partition is not present in the other relation, the rows in
- * the extra partition will be included in the bounds of the join result, if it
- * appears on the outer side of the join, since all rows from the outer side
- * are included in the join result.
- *
- * This function handles all these cases.
- *
- * maps_with_missing and missing_side_default are the partition maps (See
- * partition_range/list_bounds_merge() for details) and the index of default
- * partition respectively corresponding the side with missing partition.
- *
- * maps_with_extra and extra_part are the partition maps (See
- * partition_range/list_bounds_merge() for details) and the index of extra
- * partition respectively corresponding to the side with the extra partition.
- *
- * It returns true if the matching succeeds, otherwise returns false.
- */
-static bool
-handle_missing_partition(PartitionMap *maps_with_missing,
-						 PartitionMap *maps_with_extra,
-						 int missing_side_default,
-						 int extra_part,
-						 bool missing_side_outer,
-						 bool missing_side_inner,
-						 int *next_index, int *default_index,
-						 int *merged_index)
-{
-	bool missing_has_default = (missing_side_default != -1);
-
-	if (missing_has_default)
-	{
-		*merged_index = map_and_merge_partitions(maps_with_missing,
-												 maps_with_extra,
-												 missing_side_default,
-												 extra_part,
-												 next_index);
-		if (*merged_index < 0)
-			return false;
-
-		if (missing_side_outer)
-		{
-			/*
-			 * Default partition on the outer side forms the default
-			 * partition of the join result.
-			 */
-			if (*default_index < 0)
-				*default_index = *merged_index;
-			else if(*default_index != *merged_index)
-			{
-				/*
-				 * Ended up with default partition on the outer side
-				 * being joined with multiple partitions on the inner
-				 * side. We don't support this case.
-				 */
-				return false;
-			}
-
-			/*
-			 * Since the merged partition acts as a default partition, it
-			 * doesn't need a separate index.
-			 */
-			*merged_index = -1;
-		}
-	}
-	else if (missing_side_inner)
-	{
-		/*
-		 * If this partition has already been mapped (say because we
-		 * found an overlapping range earlier), we know where does it
-		 * fit in the join result. Nothing to do in that case. Else
-		 * create a new merged partition.
-		 */
-		PartitionMap *extra_map = &maps_with_extra[extra_part];
-		if (extra_map->to < 0)
-		{
-			extra_map->to = *next_index;
-			*next_index = *next_index + 1;
-			*merged_index = extra_map->to;
-		}
-	}
-	else
-		*merged_index = -1;
-
-	return true;
-}
-
-static PartitionMap*
-init_partition_map(RelOptInfo *rel)
-{
-	int i, nparts = rel->nparts;
-	PartitionMap *map;
-
-	map = (PartitionMap *) palloc(sizeof(PartitionMap) * nparts);
-
-	for (i = 0; i < nparts; i++)
-	{
-		map[i].from = -1;
-		map[i].to = -1;
-	}
-
-	return map;
-}
-
 /*
  * partition_range_bounds_merge
  *
  * partition_bounds_merge()'s arm for range partitioned tables.
  */
 static PartitionBoundInfo
-partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
-							 List **outer_parts, List **inner_parts,
-							 JoinType jointype, int partnatts,
-							 FmgrInfo *partsupfuncs, Oid *partcollations)
-
+partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts)
 {
-	PartitionMap *outer_maps = NULL;
-	PartitionMap *inner_maps = NULL;
-	int			outer_part = 0;
-	int			inner_part = 0;
 	PartitionBoundInfo merged_bounds = NULL;
-	int			outer_lb_index;
-	int			inner_lb_index;
-	int			next_index;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			next_index = 0;
 	int			default_index = -1;
 	List	   *merged_datums = NIL;
-	List	   *merged_indexes = NIL;
 	List	   *merged_kinds = NIL;
-	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
-					   inner_bi = inner_rel->boundinfo;
-	int			inner_default = inner_bi->default_index;
-	int			outer_default = outer_bi->default_index;
-	bool		inner_has_default = partition_bound_has_default(inner_bi);
-	bool		outer_has_default = partition_bound_has_default(outer_bi);
-	int 			   outer_nparts = outer_rel->nparts,
-					   inner_nparts = inner_rel->nparts;
+	List	   *merged_indexes = NIL;
+	int			outer_lb_index;
+	int			inner_lb_index;
 
 	Assert(outer_bi->strategy == inner_bi->strategy &&
 		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
@@ -3554,8 +3467,8 @@ partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 	Assert(*outer_parts == NIL);
 	Assert(*inner_parts == NIL);
 
-	outer_maps = init_partition_map(outer_rel);
-	inner_maps = init_partition_map(inner_rel);
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
 
 	/*
 	 * Merge the ranges (partitions) from both sides. Every iteration compares
@@ -3566,40 +3479,30 @@ partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 	 * lb_index, for inner or outer side, keeps track of the index of lower bound
 	 * datum in PartitionBoundInfo::datums of that side.
 	 */
-	outer_lb_index = 0;
-	inner_lb_index = 0;
-	next_index = 0;
+	outer_lb_index = inner_lb_index = 0;
 	while (outer_lb_index < outer_bi->ndatums ||
 		   inner_lb_index < inner_bi->ndatums)
 	{
-		PartitionRangeBound outer_lb, outer_ub,
-							inner_lb, inner_ub,
-							*merged_lb = NULL,
-							*merged_ub = NULL;
-
+		PartitionRangeBound *merged_lb = NULL;
+		PartitionRangeBound *merged_ub = NULL;
 		int			merged_index = -1;
+		PartitionRangeBound outer_lb;
+		PartitionRangeBound outer_ub;
+		PartitionRangeBound inner_lb;
+		PartitionRangeBound inner_ub;
+		int			outer_part = -1;
+		int			inner_part = -1;
 		bool		overlap;
-		bool		finished_outer = false;
-		bool		finished_inner = false;
-
-		/* Result of bounds comparison per partition_rbound_cmp(). */
-		int			ub_cmpval;	/* Upper bounds comparison result. */
-		int			lb_cmpval;	/* Lower bounds comparison result. */
+		int			ub_cmpval;
+		int			lb_cmpval;
 
 		/* Get the range bounds of the next pair of partitions. */
 		if (outer_lb_index < outer_bi->ndatums)
 			outer_part = partition_get_range_bounds(outer_bi, outer_lb_index,
-												&outer_lb, &outer_ub);
-		else
-			finished_outer = true;
-
+													&outer_lb, &outer_ub);
 		if (inner_lb_index < inner_bi->ndatums)
 			inner_part = partition_get_range_bounds(inner_bi, inner_lb_index,
-												&inner_lb, &inner_ub);
-		else
-			finished_inner = true;
-
-		Assert(!finished_outer || !finished_inner);
+													&inner_lb, &inner_ub);
 
 		/*
 		 * We run this loop till both the sides finish. This allows to avoid
@@ -3611,13 +3514,13 @@ partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 		 * side. That way we advance the partitions on that side till all of
 		 * them are  exhausted.
 		 */
-		if (finished_outer)
+		if (outer_lb_index >= outer_bi->ndatums)
 		{
 			overlap = false;
 			ub_cmpval = 1;
 			lb_cmpval = 1;
 		}
-		else if (finished_inner)
+		else if (inner_lb_index >= inner_bi->ndatums)
 		{
 			overlap = false;
 			ub_cmpval = -1;
@@ -3641,7 +3544,7 @@ partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 								  jointype, &outer_lb, &outer_ub, &inner_lb,
 								  &inner_ub, &merged_lb, &merged_ub);
 
-			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
 													outer_part, inner_part,
 													&next_index);
 
@@ -3680,6 +3583,8 @@ partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 			Assert(overlap);
 
 			/* Move to the next pair of partitions. */
+			Assert(outer_lb_index < outer_bi->ndatums);
+			Assert(inner_lb_index < inner_bi->ndatums);
 			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
 															   outer_lb_index);
 			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
@@ -3695,37 +3600,28 @@ partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 			}
 			else
 			{
-				/* A range missing from the inner side. */
-				bool		missing_side_outer;
-				bool		missing_side_inner;
+				if (inner_has_default || IS_OUTER_JOIN(jointype))
+				{
+					if (!process_outer_partition(&outer_map,
+												 &inner_map,
+												 outer_has_default,
+												 inner_has_default,
+												 outer_part,
+												 inner_default,
+												 jointype,
+												 outer_bi->strategy,
+												 &next_index,
+												 &default_index,
+												 &merged_index))
+						return NULL;
+				}
 
 				merged_lb = &outer_lb;
 				merged_ub = &outer_ub;
-
-				/*
-				 * For a FULL join, inner relation acts as both OUTER and INNER
-				 * relation.  For LEFT and ANTI join the inner relation acts as
-				 * INNER relation. For INNER and SEMI join OUTER and INNER
-				 * differentiation is immaterial.
-				 */
-				missing_side_inner = (jointype == JOIN_FULL ||
-									  jointype == JOIN_LEFT ||
-									  jointype == JOIN_ANTI);
-				missing_side_outer = (jointype == JOIN_FULL);
-				if (!handle_missing_partition(inner_maps,
-											  outer_maps,
-											  inner_default,
-											  outer_part,
-											  missing_side_outer,
-											  missing_side_inner,
-											  &next_index,
-											  &default_index,
-											  &merged_index))
-					return NULL;
 			}
 
 			/* Move to the next partition on the outer side. */
-			Assert(!finished_outer);
+			Assert(outer_lb_index < outer_bi->ndatums);
 			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
 															   outer_lb_index);
 		}
@@ -3740,38 +3636,28 @@ partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 			}
 			else
 			{
-				/* A range missing from the outer side. */
-				bool		missing_side_outer;
-				bool		missing_side_inner;
+				if (outer_has_default || jointype == JOIN_FULL)
+				{
+					if (!process_inner_partition(&outer_map,
+												 &inner_map,
+												 outer_has_default,
+												 inner_has_default,
+												 inner_part,
+												 outer_default,
+												 jointype,
+												 outer_bi->strategy,
+												 &next_index,
+												 &default_index,
+												 &merged_index))
+						return NULL;
+				}
 
 				merged_lb = &inner_lb;
 				merged_ub = &inner_ub;
-
-				/*
-				 * For a FULL join, outer relation acts as both OUTER and INNER
-				 * relation.  For LEFT and ANTI join the outer relation acts as
-				 * OUTER relation. For INNER and SEMI join OUTER and INNER
-				 * differentiation is immaterial.
-				 */
-				missing_side_outer = (jointype == JOIN_FULL ||
-									  jointype == JOIN_LEFT ||
-									  jointype == JOIN_ANTI);
-				missing_side_inner = (jointype == JOIN_FULL);
-
-				if (!handle_missing_partition(outer_maps,
-											  inner_maps,
-											  outer_default,
-											  inner_part,
-											  missing_side_outer,
-											  missing_side_inner,
-											  &next_index,
-											  &default_index,
-											  &merged_index))
-					return NULL;
 			}
 
 			/* Move to the next partition on the inner side. */
-			Assert (!finished_inner);
+			Assert(inner_lb_index < inner_bi->ndatums);
 			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
 															   inner_lb_index);
 		}
@@ -3802,16 +3688,28 @@ partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 		merged_indexes = lappend_int(merged_indexes, merged_index);
 	}
 
-	if (!merge_default_partitions(outer_bi, inner_bi,
-										  outer_maps, inner_maps,
-										  jointype, &next_index,
-										  &default_index))
-		return NULL;
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+	{
+		if (!merge_default_partitions(&outer_map,
+									  &inner_map,
+									  outer_has_default,
+									  outer_default,
+									  inner_has_default,
+									  inner_default,
+									  jointype,
+									  &next_index,
+									  &default_index))
+			return NULL;
+	}
+	else
+		Assert(default_index == -1);
+
+	Assert(!outer_map.did_remapping && !inner_map.did_remapping);
 
 	/* Use maps to match partition from the joining relations. */
 	generate_matching_part_pairs(outer_rel, inner_rel,
-								 outer_maps, inner_maps,
-								 outer_nparts, inner_nparts,
+								 &outer_map, &inner_map,
 								 next_index,
 								 outer_parts, inner_parts);
 
@@ -3828,6 +3726,8 @@ partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 	}
 
 	/* Free any memory we used in this function. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
 	list_free(merged_datums);
 	list_free(merged_indexes);
 	list_free(merged_kinds);
@@ -3844,27 +3744,27 @@ partition_range_bounds_merge(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 static PartitionBoundInfo
 partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
-							List **outer_parts, List **inner_parts,
-							JoinType jointype)
+							JoinType jointype,
+							List **outer_parts, List **inner_parts)
 {
-	PartitionMap *outer_maps = NULL;
-	PartitionMap *inner_maps = NULL;
-	int			cnto;
-	int			cnti;
-	List	   *merged_datums = NIL;
-	List	   *merged_indexes = NIL;
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	PartitionMap outer_map;
+	PartitionMap inner_map;
 	int			next_index = 0;
 	int			null_index = -1;
 	int			default_index = -1;
-	PartitionBoundInfo merged_bounds = NULL;
-	PartitionBoundInfo outer_bi = outer_rel->boundinfo,
-					   inner_bi = inner_rel->boundinfo;
-	int			      *outer_indexes = outer_bi->indexes;
-	int			      *inner_indexes = inner_bi->indexes;
-	int				   outer_default = outer_bi->default_index;
-	int				   inner_default = inner_bi->default_index;
-	int 			   outer_nparts = outer_rel->nparts,
-					   inner_nparts = inner_rel->nparts;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			cnto;
+	int			cnti;
 
 	Assert(*outer_parts == NIL);
 	Assert(*inner_parts == NIL);
@@ -3875,8 +3775,8 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 	/* List partitions do not require unbounded ranges. */
 	Assert(!outer_bi->kind && !inner_bi->kind);
 
-	outer_maps = init_partition_map(outer_rel);
-	inner_maps = init_partition_map(inner_rel);
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
 
 	/*
 	 * Merge the list value datums from both sides. Every iteration compares a
@@ -3888,15 +3788,15 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 	cnto = cnti = 0;
 	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
 	{
+		Datum	   *merged_datum = NULL;
+		int			merged_index = -1;
 		Datum	   *odatums;
 		Datum	   *idatums;
-		int			o_index;
-		int			i_index;
 		int			cmpval;
-		int			merged_index = -1;
-		Datum	   *merged_datum;
-		bool		finished_inner;
-		bool		finished_outer;
+
+		/* Get the list datums of the next pair of partitions. */
+		odatums = cnto < outer_bi->ndatums ? outer_bi->datums[cnto] : NULL;
+		idatums = cnti < inner_bi->ndatums ? inner_bi->datums[cnti] : NULL;
 
 		/*
 		 * We run this loop till both the sides finish. This allows to avoid
@@ -3908,51 +3808,23 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 		 * its datums are exhausted.
 		 */
 		if (cnto >= outer_bi->ndatums)
-		{
-			finished_outer = true;
-			odatums = NULL;
-			o_index = -1;
-		}
-		else
-		{
-			finished_outer = false;
-			odatums = outer_bi->datums[cnto];
-			o_index = outer_indexes[cnto];
-		}
-
-		if (cnti >= inner_bi->ndatums)
-		{
-			finished_inner = true;
-			idatums = NULL;
-			i_index = -1;
-		}
-		else
-		{
-			finished_inner = false;
-			idatums = inner_bi->datums[cnti];
-			i_index = inner_indexes[cnti];
-		}
-
-		/* If we exhausted both the sides, we won't enter the loop. */
-		Assert(!finished_inner || !finished_outer);
-
-		if (finished_outer)
 			cmpval = 1;
-		else if (finished_inner)
+		else if (cnti >= inner_bi->ndatums)
 			cmpval = -1;
 		else
 		{
-			/* Every list datum should map to a valid partition index. */
-			Assert(o_index >= 0 && i_index >= 0 &&
-				   odatums != NULL && idatums != NULL);
-
+			Assert(odatums != NULL && idatums != NULL);
 			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
 													 partcollation[0],
-													 odatums[0], idatums[0]));
+													 odatums[0],
+													 idatums[0]));
 		}
 
 		if (cmpval == 0)
 		{
+			int			o_index = outer_bi->indexes[cnto];
+			int			i_index = inner_bi->indexes[cnti];
+
 			/*
 			 * Datums match. Rows on either side with these datums as partition
 			 * key value will join and will be part of the partition of the
@@ -3962,7 +3834,8 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 			 * partition containing it.
 			 */
 			merged_datum = odatums;
-			merged_index = map_and_merge_partitions(outer_maps, inner_maps,
+			Assert(o_index >= 0 && i_index >= 0);
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
 													o_index, i_index,
 													&next_index);
 
@@ -3975,74 +3848,61 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 		}
 		else if (cmpval < 0)
 		{
-			bool		missing_side_outer;
-			bool		missing_side_inner;
+			Assert(cnto < outer_bi->ndatums);
 
 			/* A datum missing from the inner side. */
-			merged_index = -1;
 			merged_datum = odatums;
 
-			/*
-			 * For a FULL join, inner relation acts as both OUTER and INNER
-			 * relation.  For LEFT and ANTI join the inner relation acts as
-			 * INNER relation. For INNER and SEMI join OUTER and INNER
-			 * differentiation is immaterial.
-			 */
-			missing_side_inner = (jointype == JOIN_FULL ||
-								  jointype == JOIN_LEFT ||
-								  jointype == JOIN_ANTI);
-			missing_side_outer = (jointype == JOIN_FULL);
-
-			if (!handle_missing_partition(inner_maps,
-										  outer_maps,
-										  inner_default,
-										  o_index,
-										  missing_side_outer,
-										  missing_side_inner,
-										  &next_index,
-										  &default_index,
-										  &merged_index))
-				return NULL;
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				int			o_index = outer_bi->indexes[cnto];
+
+				Assert(o_index >= 0);
+				if (!process_outer_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 o_index,
+											 inner_default,
+											 jointype,
+											 outer_bi->strategy,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
+			}
 
 			/* Move to the next datum on the outer side. */
-			Assert(!finished_outer);
 			cnto++;
 		}
 		else
 		{
-			bool		missing_side_outer;
-			bool		missing_side_inner;
-
 			Assert(cmpval > 0);
+			Assert(cnti < inner_bi->ndatums);
 
 			/* A datum missing from the outer side. */
-			merged_index = -1;
 			merged_datum = idatums;
 
-			/*
-			 * For a FULL join, outer relation acts as both OUTER and INNER
-			 * relation.  For LEFT and ANTI join the outer relation acts as
-			 * OUTER relation. For INNER and SEMI join OUTER and INNER
-			 * differentiation is immaterial.
-			 */
-			missing_side_outer = (jointype == JOIN_FULL ||
-								  jointype == JOIN_LEFT ||
-								  jointype == JOIN_ANTI);
-			missing_side_inner = (jointype == JOIN_FULL);
-
-			if (!handle_missing_partition(outer_maps,
-										  inner_maps,
-										  outer_default,
-										  i_index,
-										  missing_side_outer,
-										  missing_side_inner,
-										  &next_index,
-										  &default_index,
-										  &merged_index))
-				return NULL;
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				int			i_index = inner_bi->indexes[cnti];
+
+				Assert(i_index >= 0);
+				if (!process_inner_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 i_index,
+											 outer_default,
+											 jointype,
+											 outer_bi->strategy,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
+			}
 
-			/* Move to the next datum on the right side. */
-			Assert(!finished_inner);
+			/* Move to the next datum on the inner side. */
 			cnti++;
 		}
 
@@ -4057,22 +3917,43 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 		}
 	}
 
-	if (!merge_null_partitions(outer_bi, inner_bi,
-							   outer_maps, inner_maps,
-							   jointype, &next_index, &null_index,
-							   &default_index))
-		return NULL;
+	/* Merge null partitions if any. */
+	if (outer_has_null || inner_has_null)
+	{
+		if (!merge_null_partitions(outer_bi, inner_bi,
+								   &outer_map, &inner_map,
+								   outer_has_default, inner_has_default,
+								   outer_has_null, inner_has_null,
+								   jointype, &next_index, &default_index,
+								   &null_index))
+			return NULL;
+	}
+	else
+		Assert(null_index == -1);
 
-	if (!merge_default_partitions(outer_bi, inner_bi,
-								  outer_maps, inner_maps,
-								  jointype, &next_index,
-								  &default_index))
-		return NULL;
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+	{
+		if (!merge_default_partitions(&outer_map, &inner_map,
+									  outer_has_default, outer_default,
+									  inner_has_default, inner_default,
+									  jointype, &next_index, &default_index))
+			return NULL;
+	}
+	else
+		Assert(default_index == -1);
+
+	/* Fix the merged_indexes list if necessary. */
+	if (outer_map.did_remapping || inner_map.did_remapping)
+	{
+		Assert(jointype == JOIN_FULL);
+		fix_merged_indexes(&outer_map, &inner_map,
+						   next_index, merged_indexes);
+	}
 
 	/* Use maps to match partition from the joining relations. */
 	generate_matching_part_pairs(outer_rel, inner_rel,
-								 outer_maps, inner_maps,
-								 outer_nparts, inner_nparts,
+								 &outer_map, &inner_map,
 								 next_index,
 								 outer_parts, inner_parts);
 
@@ -4080,7 +3961,7 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 	if (*outer_parts && *inner_parts)
 	{
 		Assert(list_length(*outer_parts) == list_length(*inner_parts));
-		Assert(list_length(*outer_parts) == next_index);
+		Assert(list_length(*outer_parts) <= next_index);
 		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
 													  merged_datums,
 													  merged_indexes, NIL,
@@ -4088,6 +3969,8 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 	}
 
 	/* Free up all extra memory before returning from this function. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
 	list_free(merged_datums);
 	list_free(merged_indexes);
 
@@ -4095,132 +3978,408 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 }
 
 /*
- * map_and_merge_partitions
- *
- * If the two given partitions (given by index1 and index2 resp.) are
- * already mapped to each other return the index of corresponding partition in
- * the merged set of partitions.  If they do not have a merged partition
- * associated with them, assign a new merged partition index.  If the
- * partitions are already mapped and their mapped partitions are different from
- * each other, they can not be merged, so return -1.
- *
- * partmaps1[i] gives the mapping of partitions for both relations. It
- * describes which partition of relation 2 matches ith partition of relation 1,
- * and which partition in the merged set matches ith partition of relation 1
- * maps to. Similarly for partmap2.
+ * init_partition_map
  *
- * index1 and index2 are the indexes of matching partition from respective
- * relations.
- *
- * *next_index is used and incremented when the given partitions require a new
- * merged partition.
+ * Initialize a PartitionMap struct for given relation.
  */
-
-static int
-map_and_merge_partitions(PartitionMap *partmaps1, PartitionMap *partmaps2,
-						 int index1, int index2, int *next_index)
+static void
+init_partition_map(RelOptInfo *rel, PartitionMap *map)
 {
-	PartitionMap 	*partmap1 = &partmaps1[index1];
-	PartitionMap 	*partmap2 = &partmaps2[index2];
-	int				merged_index;
+	int			nparts = rel->nparts;
+	int			i;
 
-	/*
-	 * If both the partitions are not mapped to each other, update the
-	 * maps.
-	 */
-	if (partmap1->from < 0 && partmap2->from < 0)
+	map->nparts = nparts;
+	map->merged_indexes = (int *) palloc(sizeof(int) * nparts);
+	map->merged = (bool *) palloc(sizeof(bool) * nparts);
+	map->did_remapping = false;
+	map->old_indexes = (int *) palloc(sizeof(int) * nparts);
+	for (i = 0; i < nparts; i++)
 	{
-		partmap1->from = index2;
-		partmap2->from = index1;
+		map->merged_indexes[i] = map->old_indexes[i] = -1;
+		map->merged[i] = false;
 	}
+}
+
+/*
+ * free_partition_map
+ */
+static void
+free_partition_map(PartitionMap *map)
+{
+	pfree(map->merged_indexes);
+	pfree(map->merged);
+	pfree(map->old_indexes);
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given partitions.
+ */
+static int
+map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_index, int inner_index, int *next_index)
+{
+	int 		outer_merged_index;
+	bool 		outer_merged;
+	int 		inner_merged_index;
+	bool 		inner_merged;
+
+	Assert(outer_index >= 0 && outer_index < outer_map->nparts);
+	outer_merged_index = outer_map->merged_indexes[outer_index];
+	outer_merged = outer_map->merged[outer_index];
+	Assert(inner_index >= 0 && inner_index < inner_map->nparts);
+	inner_merged_index = inner_map->merged_indexes[inner_index];
+	inner_merged = inner_map->merged[inner_index];
 
 	/*
-	 * If the given to partitions map to each other, find the corresponding
-	 * merged partition index .
+	 * Handle cases where both partitions are mapped to merged partitions.
 	 */
-	if (partmap1->from == index2 && partmap2->from == index1)
+	if (outer_merged_index >= 0 && inner_merged_index >= 0)
 	{
 		/*
-		 * If both the partitions are mapped to the same merged partition, get
-		 * the index of merged partition.
+		 * If the mereged partitions are the same, no need to do anything;
+		 * return the index of the merged partition.  Otherwise, if both
+		 * partitions are merged with dummy partitions, re-merge them; map
+		 * them to the merged partition with the smaller of the two merged
+		 * indexes and return the smaller index.  Otherwise they can't be
+		 * merged, so return -1.
 		 */
-		if (partmap1->to == partmap2->to)
+		if (outer_merged_index == inner_merged_index)
+		{
+			Assert(outer_merged);
+			Assert(inner_merged);
+			return outer_merged_index;
+		}
+		if (!outer_merged && !inner_merged)
 		{
-			merged_index = partmap1->to;
-
 			/*
-			 * If the given two partitions do not have a merged partition
-			 * associated with them, allocate a new merged partition.
+			 * Note that we will fix the larger index that have been added to
+			 * the merged_indexes list so far in fix_merged_indexes().
 			 */
-			if (merged_index < 0)
+			if (outer_merged_index < inner_merged_index)
+			{
+				outer_map->merged[outer_index] = true;
+				inner_map->merged_indexes[inner_index] = outer_merged_index;
+				inner_map->merged[inner_index] = true;
+				inner_map->did_remapping = true;
+				inner_map->old_indexes[inner_index] = inner_merged_index;
+				return outer_merged_index;
+			}
+			else
 			{
-				merged_index = *next_index;
-				*next_index = *next_index + 1;
-				partmap1->to = merged_index;
-				partmap2->to = merged_index;
+				inner_map->merged[inner_index] = true;
+				outer_map->merged_indexes[outer_index] = inner_merged_index;
+				outer_map->merged[outer_index] = true;
+				outer_map->did_remapping = true;
+				outer_map->old_indexes[outer_index] = outer_merged_index;
+				return inner_merged_index;
 			}
 		}
+		return -1;
+	}
+
+	/* At least one partition isn't mapped to a merged partition. */
+	Assert(outer_merged_index == -1 || inner_merged_index == -1);
+
+	/*
+	 * If neither of partitions isn't mapped, assign them a new merged
+	 * partition and return the index of the merged partition.  Otherwise, if
+	 * one of partitions is merged with a dummy relation (and the other isn't
+	 * merged), re-merge it with the other, with the same index, and return
+	 * the index.  Otherwise they can't be merged, so return -1.
+	 */
+	if (outer_merged_index == -1 && inner_merged_index == -1)
+	{
+		int 		merged_index = *next_index;
+
+		Assert(!outer_merged);
+		Assert(!inner_merged);
+		outer_map->merged_indexes[outer_index] = merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged_indexes[inner_index] = merged_index;
+		inner_map->merged[inner_index] = true;
+		*next_index = *next_index + 1;
+		return merged_index;
+	}
+	if (outer_merged_index >= 0 && !outer_map->merged[outer_index])
+	{
+		Assert(inner_merged_index == -1);
+		Assert(!inner_merged);
+		inner_map->merged_indexes[inner_index] = outer_merged_index;
+		inner_map->merged[inner_index] = true;
+		outer_map->merged[outer_index] = true;
+		return outer_merged_index;
+	}
+	if (inner_merged_index >= 0 && !inner_map->merged[inner_index])
+	{
+		Assert(outer_merged_index == -1);
+		Assert(!outer_merged);
+		outer_map->merged_indexes[outer_index] = inner_merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged[inner_index] = true;
+		return inner_merged_index;
+	}
+	return -1;
+}
+
+/*
+ * merge_partition_with_dummy
+ *
+ * *next_index is incremented.
+ */
+static int
+merge_partition_with_dummy(PartitionMap *map, int index, int *next_index)
+{
+	int 		merged_index = *next_index;
+
+	Assert(index >= 0 && index < map->nparts);
+	Assert(map->merged_indexes[index] == -1);
+	Assert(!map->merged[index]);
+	map->merged_indexes[index] = merged_index;
+	/* Leave the merged flag alone! */
+	*next_index = *next_index + 1;
+	return merged_index;
+}
+
+/*
+ * process_outer_partition
+ *
+ * Determine the merged partition associated with the given outer partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given outer partition.
+ */
+static bool
+process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index)
+{
+	Assert(outer_index >= 0);
+
+	/*
+	 * If the inner side has the default partition, the outer partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an outer join, in which case the outer partition has to be
+	 * scanned all the way anyway; if the outer partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the outer partition with a dummy partition.
+	 */
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0);
+
+		/*
+		 * If the outer side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (outer_has_default)
+			return false;
+
+		*merged_index = map_and_merge_partitions(outer_map, inner_map,
+												 outer_index, inner_default,
+												 next_index);
+		if (*merged_index == -1)
+			return false;
 
 		/*
-		 * If partition from one relation was mapped to a merged partition but
-		 * not the partition from the other relation, map the same merged
-		 * partition to the partition from other relation, since matching
-		 * partitions map to the same merged partition.
+		 * If this is a FULL join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
 		 */
-		else if (partmap1->to >= 0 && partmap2->to < 0)
+		if (jointype == JOIN_FULL)
 		{
-			partmap2->to = partmap1->to;
-			merged_index = partmap1->to;
+			if (*default_index == -1)
+				*default_index = *merged_index;
+			else
+				Assert(*merged_index == *default_index);
+			/* Don't add this index to the list of merged indexes. */
+			*merged_index = -1;
 		}
-		else if (partmap1->to < 0 && partmap2->to >= 0)
+	}
+	else
+	{
+		Assert(IS_OUTER_JOIN(jointype));
+		Assert(jointype != JOIN_RIGHT);
+
+		/*
+		 * In range partitioning, if the given outer partition is already
+		 * merged (eg, because we found an overlapping range earlier), we know
+		 * where it fits in the join result; nothing to do in that case.  Else
+		 * create a new merged partition.
+		 */
+		if (outer_map->merged_indexes[outer_index] >= 0)
 		{
-			partmap1->to = partmap2->to;
-			merged_index = partmap2->to;
+			if (strategy == PARTITION_STRATEGY_LIST)
+				*merged_index = outer_map->merged_indexes[outer_index];
+			else
+			{
+				Assert(strategy == PARTITION_STRATEGY_RANGE);
+				*merged_index = -1;
+			}
 		}
 		else
-		{
-			Assert(partmap1->to != partmap2->to &&
-				   partmap1->to >= 0 && partmap2->to >= 0);
+			*merged_index = merge_partition_with_dummy(outer_map, outer_index,
+													   next_index);
+	}
+	return true;
+}
 
-			/*
-			 * Both the partitions map to different merged partitions. This
-			 * means that multiple partitions from one relation matches to one
-			 * partition from the other relation. Partition-wise join does not
-			 * handle this case right now, since it requires ganging multiple
-			 * partitions together (into one RelOptInfo).
-			 */
-			merged_index = -1;
+/*
+ * process_inner_partition
+ *
+ * Determine the merged partition associated with the given inner partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given inner partition.
+ */
+static bool
+process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index)
+{
+	Assert(inner_index >= 0);
+
+	/*
+	 * If the outer side has the default partition, the inner partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an FULL join, in which case the inner partition has to be
+	 * scanned all the way anyway; if the inner partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the inner partition with a dummy partition.
+	 */
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0);
+
+		/*
+		 * If the inner side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (inner_has_default)
+			return false;
+
+		*merged_index = map_and_merge_partitions(outer_map, inner_map,
+												 outer_default, inner_index,
+												 next_index);
+		if (*merged_index == -1)
+			return false;
+
+		/*
+		 * If this is an outer join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (jointype == JOIN_LEFT || jointype == JOIN_ANTI ||
+			jointype == JOIN_FULL)
+		{
+			if (*default_index == -1)
+				*default_index = *merged_index;
+			else
+				Assert(*merged_index == *default_index);
+			/* Don't add this index to the list of merged indexes. */
+			*merged_index = -1;
 		}
 	}
 	else
 	{
+		Assert(jointype == JOIN_FULL);
+
 		/*
-		 * Multiple partitions from one relation map to one partition from the
-		 * other relation. Partition-wise join does not handle this case right
-		 * now, since it requires ganging multiple partitions together (into
-		 * one RelOptInfo).
+		 * In range partitioning, if the given inner partition is already
+		 * merged (eg, because we found an overlapping range earlier), we know
+		 * where it fits in the join result; nothing to do in that case.  Else
+		 * create a new merged partition.
 		 */
-		merged_index = -1;
+		if (inner_map->merged_indexes[inner_index] >= 0)
+		{
+			if (strategy == PARTITION_STRATEGY_LIST)
+				*merged_index = inner_map->merged_indexes[inner_index];
+			else
+			{
+				Assert(strategy == PARTITION_STRATEGY_RANGE);
+				*merged_index = -1;
+			}
+		}
+		else
+			*merged_index = merge_partition_with_dummy(inner_map, inner_index,
+													   next_index);
 	}
+	return true;
+}
 
-	return merged_index;
+/*
+ * fix_merged_indexes
+ */
+static void
+fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes)
+{
+	int		   *new_indexes;
+	int			merged_index;
+	int			i;
+	ListCell   *lc;
+
+	new_indexes = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		new_indexes[i] = -1;
+
+	/* Build the mapping of old merged indexes to new merged indexes. */
+	if (outer_map->did_remapping)
+	{
+		for (i = 0; i < outer_map->nparts; i++)
+		{
+			merged_index = outer_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = outer_map->merged_indexes[i];
+		}
+	}
+	if (inner_map->did_remapping)
+	{
+		for (i = 0; i < inner_map->nparts; i++)
+		{
+			merged_index = inner_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = inner_map->merged_indexes[i];
+		}
+	}
+
+	/* Fix the merged_indexes list using the mapping. */
+	foreach(lc, merged_indexes)
+	{
+		merged_index = lfirst_int(lc);
+		Assert(merged_index >= 0);
+		if (new_indexes[merged_index] >= 0)
+			lfirst_int(lc) = new_indexes[merged_index];
+	}
+
+	pfree(new_indexes);
 }
 
 /*
  * generate_matching_part_pairs
  *
- * partmaps1 map each partition from either side of the join to a merged
- * partition resp. E.g. partmaps1[i].to gives the merged partition to which ith
- * partition of first relation maps. Similarly for partmap2. If
- * partmaps1[i].to == partmaps2[j].to, i and j form the matching pair of
- * partitions.
- *
- * Given these maps this function produces the list pairs of partitions which
- * when joined produce the merged partitions in the order of merged partition
- * indexes.
- *
- * nparts1 and nparts2 are the number of partitions of the joining relations
- * resp.
+ * This function produces the list pairs of partitions which produce merged
+ * partitions in the order of merged partition indexes.
  *
  * nparts is the number of merged partitions.
  *
@@ -4230,15 +4389,20 @@ map_and_merge_partitions(PartitionMap *partmaps1, PartitionMap *partmaps2,
  */
 static void
 generate_matching_part_pairs(RelOptInfo *rel1, RelOptInfo *rel2,
-							 PartitionMap *partmaps1, PartitionMap *partmaps2,
-							 int nparts1, int nparts2, int nparts,
+							 PartitionMap *map1, PartitionMap *map2,
+							 int nparts,
 							 List **matched_parts1, List **matched_parts2)
 {
+	int			nparts1 = map1->nparts;
+	int			nparts2 = map2->nparts;
 	int		   *matching1,
 			   *matching2;
 	int 		i;
 	int			max_nparts;
 
+	Assert(map1->nparts == rel1->nparts);
+	Assert(map2->nparts == rel2->nparts);
+
 	*matched_parts1 = NIL;
 	*matched_parts2 = NIL;
 
@@ -4253,23 +4417,22 @@ generate_matching_part_pairs(RelOptInfo *rel1, RelOptInfo *rel2,
 	{
 		if (i < nparts1)
 		{
-			PartitionMap outer_map = partmaps1[i];
+			int 		merged_index = map1->merged_indexes[i];
 
-			if (outer_map.to >= 0)
+			if (merged_index >= 0)
 			{
-				Assert(outer_map.to < nparts);
-				matching1[outer_map.to] = i;
+				Assert(merged_index < nparts);
+				matching1[merged_index] = i;
 			}
 		}
-
 		if (i < nparts2)
 		{
-			PartitionMap inner_map = partmaps2[i];
+			int 		merged_index = map2->merged_indexes[i];
 
-			if (inner_map.to >= 0)
+			if (merged_index >= 0)
 			{
-				Assert(inner_map.to < nparts);
-				matching2[inner_map.to] = i;
+				Assert(merged_index < nparts);
+				matching2[merged_index] = i;
 			}
 		}
 	}
@@ -4279,8 +4442,8 @@ generate_matching_part_pairs(RelOptInfo *rel1, RelOptInfo *rel2,
 		int			part1 = matching1[i];
 		int			part2 = matching2[i];
 
-		/* At least one of the partitions should exist. */
-		Assert(part1 >= 0 || part2 >= 0);
+		if (part1 == -1 && part2 == -1)
+			continue;
 
 		*matched_parts1 = lappend(*matched_parts1,
 								  part1 >= 0 ? rel1->part_rels[part1] : NULL);
@@ -4356,57 +4519,53 @@ build_merged_partition_bounds(char strategy, List *merged_datums,
  * appear in the join result, so create a default merged partition.
  */
 static bool
-merge_default_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
-						 PartitionMap *outer_maps, PartitionMap *inner_maps,
-						 JoinType jointype, int *next_index, int *default_index)
+merge_default_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 bool outer_has_default, int outer_default,
+						 bool inner_has_default, int inner_default,
+						 JoinType jointype, int *next_index,
+						 int *default_index)
 {
-	int				outer_default = outer_bi->default_index;
-	int				inner_default = inner_bi->default_index;
-	bool			outer_has_default = partition_bound_has_default(outer_bi);
-	bool			inner_has_default = partition_bound_has_default(inner_bi);
-	bool			merged = true;
-	PartitionMap 	*outer_default_map = NULL;
-	PartitionMap 	*inner_default_map = NULL;
+	Assert(outer_has_default || inner_has_default);
 
-	if (outer_has_default)
-		outer_default_map = &outer_maps[outer_default];
-
-	if (inner_has_default)
-		inner_default_map = &inner_maps[inner_default];
-
-	if (!outer_has_default && !inner_has_default)
-		Assert(*default_index < 0);
-	else if (outer_default_map != NULL && inner_default_map == NULL)
+	if (outer_has_default && !inner_has_default)
 	{
 		if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
 			jointype == JOIN_ANTI)
 		{
-			if (outer_default_map->to < 0)
+			int			merged_index;
+
+			Assert(outer_default >= 0 && outer_default < outer_map->nparts);
+			merged_index = outer_map->merged_indexes[outer_default];
+			if (merged_index == -1)
 			{
-				outer_default_map->to = *next_index;
-				*next_index = *next_index + 1;
-				Assert(*default_index < 0);
-				*default_index = outer_default_map->to;
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(outer_map,
+															outer_default,
+															next_index);
 			}
 			else
-				Assert(*default_index == outer_default_map->to);
+				Assert(*default_index == merged_index);
 		}
 		else
 			Assert(*default_index < 0);
 	}
-	else if (outer_default_map == NULL && inner_default_map != NULL)
+	else if (!outer_has_default && inner_has_default)
 	{
 		if (jointype == JOIN_FULL)
 		{
-			if (inner_default_map->to < 0)
+			int			merged_index;
+
+			Assert(inner_default >= 0 && inner_default < inner_map->nparts);
+			merged_index = inner_map->merged_indexes[inner_default];
+			if (merged_index == -1)
 			{
-				inner_default_map->to = *next_index;
-				*next_index = *next_index + 1;
-				Assert(*default_index < 0);
-				*default_index = inner_default_map->to;
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(inner_map,
+															inner_default,
+															next_index);
 			}
 			else
-				Assert(*default_index == inner_default_map->to);
+				Assert(*default_index == merged_index);
 		}
 		else
 			Assert(*default_index < 0);
@@ -4415,15 +4574,16 @@ merge_default_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_b
 	{
 		Assert(outer_has_default && inner_has_default);
 
-		*default_index = map_and_merge_partitions(outer_maps, inner_maps,
-												  outer_default, inner_default,
+		*default_index = map_and_merge_partitions(outer_map,
+												  inner_map,
+												  outer_default,
+												  inner_default,
 												  next_index);
-
-		if (*default_index < 0)
-			merged = false;
+		if (*default_index == -1)
+			return false;
 	}
 
-	return merged;
+	return true;
 }
 
 /*
@@ -4443,101 +4603,80 @@ merge_default_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_b
  */
 static bool
 merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
-					  PartitionMap *outer_maps, PartitionMap *inner_maps,
+					  PartitionMap *outer_map, PartitionMap *inner_map,
+					  bool outer_has_default, bool inner_has_default,
+					  bool outer_has_null, bool inner_has_null,
 					  JoinType jointype, int *next_index,
-					  int *null_index, int *default_index)
+					  int *default_index, int *null_index)
 {
-	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
-	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
-	int			outer_ni = outer_bi->null_index;
-	int			inner_ni = inner_bi->null_index;
-	int			outer_default = outer_bi->default_index;
-	int			inner_default = inner_bi->default_index;
-	bool		merged = true;
+	Assert(outer_has_null || inner_has_null);
+	Assert(*null_index == -1);
 
-	if (!outer_has_null && !inner_has_null)
-		Assert(*null_index < 0);
-	else if (outer_has_null && !inner_has_null)
+	if (outer_has_null && !inner_has_null)
 	{
 		int			merged_index = -1;
-		bool		missing_side_outer;
-		bool		missing_side_inner;
-
-		/*
-		 * For a FULL join, inner relation acts as both OUTER and INNER
-		 * relation.  For LEFT and ANTI join the inner relation acts as
-		 * INNER relation. For INNER and SEMI join OUTER and INNER
-		 * differentiation is immaterial.
-		 */
-		missing_side_inner = (jointype == JOIN_FULL ||
-							  jointype == JOIN_LEFT ||
-							  jointype == JOIN_ANTI);
-		missing_side_outer = (jointype == JOIN_FULL);
-
-		merged = handle_missing_partition(inner_maps,
-										  outer_maps,
-										  inner_default,
-										  outer_ni,
-										  missing_side_outer,
-										  missing_side_inner, next_index,
-										  default_index, &merged_index);
-		*null_index = merged_index;
 
 		/*
 		 * If the NULL partition was missing from the inner side of the join,
-		 * the partition of the join to which the outer null partition maps
-		 * will contain the NULL values and thus becomes the NULL partition of
+		 * the partition of the join to which the NULL partition matches will
+		 * contain the NULL values and thus become the NULL partition of the
 		 * the join.
 		 */
-		if (missing_side_inner)
-			*null_index = outer_maps[outer_ni].to;
+		if (inner_has_default || IS_OUTER_JOIN(jointype))
+		{
+			if (!process_outer_partition(outer_map,
+										 inner_map,
+										 outer_has_default,
+										 inner_has_default,
+										 outer_bi->null_index,
+										 inner_bi->default_index,
+										 jointype,
+										 outer_bi->strategy,
+										 next_index,
+										 default_index,
+										 &merged_index))
+				return false;
+		}
+		*null_index = merged_index;
 	}
 	else if (!outer_has_null && inner_has_null)
 	{
 		int			merged_index = -1;
-		bool		missing_side_outer;
-		bool		missing_side_inner;
 
 		/*
-		 * For a FULL join, outer relation acts as both OUTER and INNER
-		 * relation.  For LEFT and ANTI join the outer relation acts as OUTER
-		 * relation. For INNER and SEMI join OUTER and INNER differentiation is
-		 * immaterial.
-		 */
-		missing_side_outer = (jointype == JOIN_FULL ||
-							  jointype == JOIN_LEFT ||
-							  jointype == JOIN_ANTI);
-		missing_side_inner = (jointype == JOIN_FULL);
-		merged = handle_missing_partition(outer_maps,
-										  inner_maps,
-										  outer_default,
-										  inner_ni,
-										  missing_side_outer,
-										  missing_side_inner,
-										  next_index, default_index,
-										  &merged_index);
-		*null_index = merged_index;
-
-		/*
-		 * If the NULL partition was missing from the inner side of the join,
-		 * the partition of the join, to which the outer side null partition maps,
-		 * will contain the NULL values and thus becomes the NULL partition of
+		 * If the NULL partition was missing from the outer side of the join,
+		 * the partition of the join to which the NULL partition matches will
+		 * contain the NULL values and thus become the NULL partition of the
 		 * the join.
 		 */
-		if (missing_side_inner)
-			*null_index = inner_maps[inner_ni].to;
+		if (outer_has_default || jointype == JOIN_FULL)
+		{
+			if (!process_inner_partition(outer_map,
+										 inner_map,
+										 outer_has_default,
+										 inner_has_default,
+										 inner_bi->null_index,
+										 outer_bi->default_index,
+										 jointype,
+										 outer_bi->strategy,
+										 next_index,
+										 default_index,
+										 &merged_index))
+				return false;
+		}
+		*null_index = merged_index;
 	}
 	else
 	{
 		/* Both the relations have NULL partitions, try merging them. */
-		*null_index = map_and_merge_partitions(outer_maps,
-											   inner_maps,
-											   outer_ni,
-											   inner_ni,
+		*null_index = map_and_merge_partitions(outer_map,
+											   inner_map,
+											   outer_bi->null_index,
+											   inner_bi->null_index,
 											   next_index);
-		if (*null_index < 0)
-			merged = false;
+		if (*null_index == -1)
+			return false;
 	}
 
-	return merged;
+	return true;
 }
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index b5b5c8a260..89f0c6a9ee 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -4769,3 +4769,182 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
                            Filter: (b = 0)
 (23 rows)
 
+DROP TABLE plt1;
+DROP TABLE plt2;
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
+ANALYZE plt1;
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+ANALYZE plt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.c = t2.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt1_p1 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t2_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_1
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+  0 | 0000 |    | 
+  5 | 0000 |    | 
+ 10 | 0000 |    | 
+ 15 | 0000 |    | 
+ 20 | 0000 |    | 
+  2 | 0002 |  2 | 0002
+  2 | 0002 |  7 | 0002
+  2 | 0002 | 12 | 0002
+  2 | 0002 | 17 | 0002
+  2 | 0002 | 22 | 0002
+  7 | 0002 |  2 | 0002
+  7 | 0002 |  7 | 0002
+  7 | 0002 | 12 | 0002
+  7 | 0002 | 17 | 0002
+  7 | 0002 | 22 | 0002
+ 12 | 0002 |  2 | 0002
+ 12 | 0002 |  7 | 0002
+ 12 | 0002 | 12 | 0002
+ 12 | 0002 | 17 | 0002
+ 12 | 0002 | 22 | 0002
+ 17 | 0002 |  2 | 0002
+ 17 | 0002 |  7 | 0002
+ 17 | 0002 | 12 | 0002
+ 17 | 0002 | 17 | 0002
+ 17 | 0002 | 22 | 0002
+ 22 | 0002 |  2 | 0002
+ 22 | 0002 |  7 | 0002
+ 22 | 0002 | 12 | 0002
+ 22 | 0002 | 17 | 0002
+ 22 | 0002 | 22 | 0002
+    |      |  1 | 0001
+    |      |  6 | 0001
+    |      | 11 | 0001
+    |      | 16 | 0001
+    |      | 21 | 0001
+(35 rows)
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1;
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2;
+CREATE TABLE plt3 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt3_p1 PARTITION OF plt3 FOR VALUES IN ('0001');
+CREATE TABLE plt3_p2 PARTITION OF plt3 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2.a, t3.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.c = t3.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1.c = t2.c)
+                     ->  Seq Scan on plt1_p1 t1
+                     ->  Hash
+                           ->  Seq Scan on plt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on plt3_p1 t3
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t3_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_1.c = t2_1.c)
+                     ->  Seq Scan on plt1_p2 t1_1
+                     ->  Hash
+                           ->  Seq Scan on plt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt3_p2 t3_1
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+ a  |  c   | a  |  c   | a  |  c   
+----+------+----+------+----+------
+  0 | 0000 |    |      |    | 
+  5 | 0000 |    |      |    | 
+ 10 | 0000 |    |      |    | 
+ 15 | 0000 |    |      |    | 
+ 20 | 0000 |    |      |    | 
+  1 | 0001 |    |      |  1 | 0001
+  1 | 0001 |    |      |  6 | 0001
+  1 | 0001 |    |      | 11 | 0001
+  1 | 0001 |    |      | 16 | 0001
+  1 | 0001 |    |      | 21 | 0001
+  6 | 0001 |    |      |  1 | 0001
+  6 | 0001 |    |      |  6 | 0001
+  6 | 0001 |    |      | 11 | 0001
+  6 | 0001 |    |      | 16 | 0001
+  6 | 0001 |    |      | 21 | 0001
+ 11 | 0001 |    |      |  1 | 0001
+ 11 | 0001 |    |      |  6 | 0001
+ 11 | 0001 |    |      | 11 | 0001
+ 11 | 0001 |    |      | 16 | 0001
+ 11 | 0001 |    |      | 21 | 0001
+ 16 | 0001 |    |      |  1 | 0001
+ 16 | 0001 |    |      |  6 | 0001
+ 16 | 0001 |    |      | 11 | 0001
+ 16 | 0001 |    |      | 16 | 0001
+ 16 | 0001 |    |      | 21 | 0001
+ 21 | 0001 |    |      |  1 | 0001
+ 21 | 0001 |    |      |  6 | 0001
+ 21 | 0001 |    |      | 11 | 0001
+ 21 | 0001 |    |      | 16 | 0001
+ 21 | 0001 |    |      | 21 | 0001
+  2 | 0002 |  2 | 0002 |    | 
+  2 | 0002 |  7 | 0002 |    | 
+  2 | 0002 | 12 | 0002 |    | 
+  2 | 0002 | 17 | 0002 |    | 
+  2 | 0002 | 22 | 0002 |    | 
+  7 | 0002 |  2 | 0002 |    | 
+  7 | 0002 |  7 | 0002 |    | 
+  7 | 0002 | 12 | 0002 |    | 
+  7 | 0002 | 17 | 0002 |    | 
+  7 | 0002 | 22 | 0002 |    | 
+ 12 | 0002 |  2 | 0002 |    | 
+ 12 | 0002 |  7 | 0002 |    | 
+ 12 | 0002 | 12 | 0002 |    | 
+ 12 | 0002 | 17 | 0002 |    | 
+ 12 | 0002 | 22 | 0002 |    | 
+ 17 | 0002 |  2 | 0002 |    | 
+ 17 | 0002 |  7 | 0002 |    | 
+ 17 | 0002 | 12 | 0002 |    | 
+ 17 | 0002 | 17 | 0002 |    | 
+ 17 | 0002 | 22 | 0002 |    | 
+ 22 | 0002 |  2 | 0002 |    | 
+ 22 | 0002 |  7 | 0002 |    | 
+ 22 | 0002 | 12 | 0002 |    | 
+ 22 | 0002 | 17 | 0002 |    | 
+ 22 | 0002 | 22 | 0002 |    | 
+(55 rows)
+
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 34ae92135f..b0ab002eee 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -800,3 +800,47 @@ ANALYZE prt2;
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
+ANALYZE plt1;
+
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+ANALYZE plt2;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1;
+
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2;
+
+CREATE TABLE plt3 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt3_p1 PARTITION OF plt3 FOR VALUES IN ('0001');
+CREATE TABLE plt3_p2 PARTITION OF plt3 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
#101Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Etsuro Fujita (#100)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Thu, Oct 31, 2019 at 6:49 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

Attached is an updated version. If no objections, I'll merge this
with the main patch [1].

Done. Attached is a new version of the patch.

Other changes: in generate_matching_part_pairs(), I changed variable
names to match other functions, simplified assertions, and
adjusted/added comments a bit.

Will continue to review.

Best regards,
Etsuro Fujita

Attachments:

Improve-partition-matching-for-partitionwise-joins-v25.patchapplication/octet-stream; name=Improve-partition-matching-for-partitionwise-joins-v25.patchDownload
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index b0dcd02ff6..b815157402 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -2281,6 +2281,8 @@ _outRelOptInfo(StringInfo str, const RelOptInfo *node)
 	WRITE_BOOL_FIELD(has_eclass_joins);
 	WRITE_BOOL_FIELD(consider_partitionwise_join);
 	WRITE_BITMAPSET_FIELD(top_parent_relids);
+	WRITE_BOOL_FIELD(merged);
+	WRITE_BITMAPSET_FIELD(all_partrels);
 	WRITE_NODE_FIELD(partitioned_child_rels);
 }
 
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 6a480ab764..ed7bc23c7b 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -48,6 +48,9 @@ static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 												Relids left_relids, Relids right_relids);
 static int	match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel,
 										 bool strict_op);
+static void get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2);
 
 
 /*
@@ -1357,25 +1360,30 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	bool		merged = false;
+	List	   *parts1 = NIL;
+	List	   *parts2 = NIL;
+	ListCell   *lcr1 = NULL;
+	ListCell   *lcr2 = NULL;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1390,34 +1398,107 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		   joinrel->part_scheme == rel2->part_scheme);
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * If we don't have the partition bounds for the join rel yet, try to
+	 * create it along with pairs of partitions to be joined; else generate
+	 * those using the partitioning info for the join rel we already have.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	if (joinrel->nparts == -1)
+	{
+		PartitionScheme part_scheme = joinrel->part_scheme;
+		PartitionBoundInfo boundinfo = NULL;
+		int			nparts = 0;
+
+		Assert(joinrel->boundinfo == NULL);
+		Assert(joinrel->part_rels == NULL);
+
+		/*
+		 * See if the partition bounds for inputs are exactly the same, in
+		 * which case we don't need to work hard: partitions with the same
+		 * partition indexes will form join pairs, and the join rel will have
+		 * the same partition bounds as inputs; otherwise try to merge the
+		 * partition bounds along with generating join pairs.
+		 *
+		 * Even if one or both inputs have merged partition bounds, it'd be
+		 * possible for the partition bounds to be exactly the same, but it
+		 * seems unlikely to be worth the cycles to check; do this check only
+		 * if both inputs have non-merged partition bounds.
+		 */
+		if (!rel1->merged &&
+			!rel2->merged &&
+			rel1->nparts == rel2->nparts &&
+			partition_bounds_equal(part_scheme->partnatts,
+								   part_scheme->parttyplen,
+								   part_scheme->parttypbyval,
+								   rel1->boundinfo, rel2->boundinfo))
+		{
+			boundinfo = rel1->boundinfo;
+			nparts = rel1->nparts;
+		}
+		else
+		{
+			boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											   part_scheme->parttyplen,
+											   part_scheme->parttypbyval,
+											   part_scheme->partsupfunc,
+											   part_scheme->partcollation,
+											   rel1, rel2,
+											   parent_sjinfo->jointype,
+											   &parts1, &parts2);
+			if (boundinfo == NULL)
+			{
+				joinrel->nparts = 0;
+				return;
+			}
+			nparts = list_length(parts1);
+			merged = true;
+		}
+
+		Assert(nparts > 0);
+		joinrel->boundinfo = boundinfo;
+		joinrel->merged = merged;
+		joinrel->nparts = nparts;
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+	}
+	else
+	{
+		Assert(joinrel->nparts > 0);
+		Assert(joinrel->boundinfo);
+		Assert(joinrel->part_rels);
+
+		/*
+		 * If the partition bounds for the join rel are not merged ones,
+		 * inputs are guaranteed to have the same partition bounds, so
+		 * partitions with the same partition indexes will form join pairs;
+		 * else let get_matching_part_pairs() do the work.
+		 */
+		if (joinrel->merged)
+		{
+			get_matching_part_pairs(root, joinrel, rel1, rel2,
+									&parts1, &parts2);
+			Assert(list_length(parts1) == joinrel->nparts);
+			Assert(list_length(parts2) == joinrel->nparts);
+			merged = true;
+		}
+	}
 
-	nparts = joinrel->nparts;
+	if (merged)
+	{
+		lcr1 = list_head(parts1);
+		lcr2 = list_head(parts2);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
-		bool		rel1_empty = (child_rel1 == NULL ||
-								  IS_DUMMY_REL(child_rel1));
-		bool		rel2_empty = (child_rel2 == NULL ||
-								  IS_DUMMY_REL(child_rel2));
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		bool		rel1_empty;
+		bool		rel2_empty;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1425,6 +1506,22 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		if (merged)
+		{
+			child_rel1 = lfirst_node(RelOptInfo, lcr1);
+			child_rel2 = lfirst_node(RelOptInfo, lcr2);
+			lcr1 = lnext(parts1, lcr1);
+			lcr2 = lnext(parts2, lcr2);
+		}
+		else
+		{
+			child_rel1 = rel1->part_rels[cnt_parts];
+			child_rel2 = rel2->part_rels[cnt_parts];
+		}
+
+		rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1));
+		rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2));
+
 		/*
 		 * Check for cases where we can prove that this segment of the join
 		 * returns no rows, due to one or both inputs being empty (including
@@ -1522,6 +1619,8 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 												 child_sjinfo,
 												 child_sjinfo->jointype);
 			joinrel->part_rels[cnt_parts] = child_joinrel;
+			joinrel->all_partrels = bms_add_members(joinrel->all_partrels,
+													child_joinrel->relids);
 		}
 
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
@@ -1529,6 +1628,7 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
+
 	}
 }
 
@@ -1738,3 +1838,99 @@ match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op)
 
 	return -1;
 }
+
+/*
+ * get_matching_part_pairs
+ *		Generate join pairs of partitions for the two inputs
+ */
+static void
+get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2)
+{
+	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
+	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
+	int 		cnt_parts;
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
+	{
+		RelOptInfo *child_joinrel = joinrel->part_rels[cnt_parts];
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		Relids		child_relids1;
+		Relids		child_relids2;
+
+		/*
+		 * If this segment of the join is empty, it means that this segment
+		 * was ignored when previously creating child-join paths for it in
+		 * try_partitionwise_join() as it would not contribute to the join
+		 * result, due to one or both inputs being empty; add NULL to each of
+		 * the given lists so that this segment will be ignored again in that
+		 * function.
+		 */
+		if (!child_joinrel)
+		{
+			*parts1 = lappend(*parts1, NULL);
+			*parts2 = lappend(*parts2, NULL);
+			continue;
+		}
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel1 side.
+		 */
+		child_relids1 = bms_intersect(child_joinrel->relids,
+									  rel1->all_partrels);
+		Assert(bms_num_members(child_relids1) == bms_num_members(rel1->relids));
+
+		/*
+		 * Get a child rel for rel1 with the relids.  Note that we should have
+		 * the child rel even if rel1 is a join rel, because in that case the
+		 * partitions specified in the relids would have matching/overlapping
+		 * boundaries, so those partitions should be considered as ones to be
+		 * joined even when planning partitionwise joins of rel1, meaning that
+		 * the child rel would have been built by the time we get here.
+		 */
+		if (rel1_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids1);
+
+			child_rel1 = find_base_rel(root, varno);
+		}
+		else
+			child_rel1 = find_join_rel(root, child_relids1);
+		Assert(child_rel1);
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel2 side.
+		 */
+		child_relids2 = bms_intersect(child_joinrel->relids,
+									  rel2->all_partrels);
+		Assert(bms_num_members(child_relids2) == bms_num_members(rel2->relids));
+
+		/*
+		 * Get a child rel for rel2 with the relids.  See above comments.
+		 */
+		if (rel2_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids2);
+
+			child_rel2 = find_base_rel(root, varno);
+		}
+		else
+			child_rel2 = find_join_rel(root, child_relids2);
+		Assert(child_rel2);
+
+		/*
+		 * The join of rel1 and rel2 is legal, so is the join of the child
+		 * rels obtained above; add them to the given lists as a join pair
+		 * producing this join segment.
+		 */
+		*parts1 = lappend(*parts1, child_rel1);
+		*parts2 = lappend(*parts2, child_rel2);
+	}
+}
diff --git a/src/backend/optimizer/util/inherit.c b/src/backend/optimizer/util/inherit.c
index 38bc61e687..caf6039c10 100644
--- a/src/backend/optimizer/util/inherit.c
+++ b/src/backend/optimizer/util/inherit.c
@@ -376,6 +376,8 @@ expand_partitioned_rtentry(PlannerInfo *root, RelOptInfo *relinfo,
 		/* Create the otherrel RelOptInfo too. */
 		childrelinfo = build_simple_rel(root, childRTindex, relinfo);
 		relinfo->part_rels[i] = childrelinfo;
+		relinfo->all_partrels = bms_add_members(relinfo->all_partrels,
+												childrelinfo->relids);
 
 		/* If this child is itself partitioned, recurse */
 		if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 85415381fb..248a9e9093 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -240,10 +240,12 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->has_eclass_joins = false;
 	rel->consider_partitionwise_join = false;	/* might get changed later */
 	rel->part_scheme = NULL;
-	rel->nparts = 0;
+	rel->nparts = -1;
 	rel->boundinfo = NULL;
+	rel->merged = false;
 	rel->partition_qual = NIL;
 	rel->part_rels = NULL;
+	rel->all_partrels = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->partitioned_child_rels = NIL;
@@ -653,10 +655,12 @@ build_join_rel(PlannerInfo *root,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -829,10 +833,12 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -1626,7 +1632,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1639,24 +1645,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1668,17 +1656,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 4bc88104ee..ff1965f5ab 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -25,6 +25,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/pathnodes.h"
 #include "parser/parse_coerce.h"
 #include "partitioning/partbounds.h"
 #include "partitioning/partdesc.h"
@@ -69,6 +70,17 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+typedef struct PartitionMap
+{
+	int			nparts;			/* number of partitions */
+	int		   *merged_indexes;	/* indexes of merged partitions */
+	bool	   *merged;			/* flags to indicate whether partitions are
+								 * merged with non-dummy partitions */
+	bool		did_remapping;	/* did we remap partitions? */
+	int		   *old_indexes;	/* old indexes of merged partitions if
+								 * did_remapping */
+} PartitionMap;
+
 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);
@@ -108,6 +120,85 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 									 Expr **keyCol,
 									 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static PartitionBoundInfo partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts);
+static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							JoinType jointype,
+							List **outer_parts, List **inner_parts);
+static void init_partition_map(RelOptInfo *rel, PartitionMap *map);
+static void free_partition_map(PartitionMap *map);
+static int map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_part, int inner_part, int *next_index);
+static int merge_partition_with_dummy(PartitionMap *map, int index,
+						   int *next_index);
+static bool process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index);
+static bool process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index);
+static void fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes);
+static void generate_matching_part_pairs(RelOptInfo *outer_rel,
+							 RelOptInfo *inner_rel,
+							 PartitionMap *outer_map,
+							 PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_part_list,
+							 List **inner_part_list);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *collations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2);
+static bool partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
+						   Oid *collations, PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, int *ub_cmpval,
+						   int *lb_cmpval);
+static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes);
+static bool merge_default_partitions(PartitionMap *outer_map,
+						 PartitionMap *inner_map,
+						 bool outer_has_default,
+						 int outer_default,
+						 bool inner_has_default,
+						 int inner_default,
+						 JoinType jointype,
+						 int *next_index,
+						 int *default_index);
+static bool merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_map, PartitionMap *inner_map,
+					  bool outer_has_default, bool inner_has_default,
+					  bool outer_has_null, bool inner_has_null,
+					  JoinType jointype, int *next_index,
+					  int *default_index, int *null_index);
 
 /*
  * get_qual_from_partbound
@@ -2995,3 +3086,1603 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * The function produces the partition bounds for a join between two relations
+ * whose partition bounds are given. The function also returns two lists of
+ * partition indexes one for each of the joining relations. Both the lists
+ * contain the same number of elements. The partition indexes at the same
+ * positions in the lists indicate the pair partitions, one from each side, to
+ * be joined and the position itself corresponds to the index of partition
+ * produced by that child-join in the partitioned join.
+ *
+ * The function returns NULL if we can not find the matching pair of
+ * partitions. This happens if 1. multiple partitions on one side match with
+ * one partition on the other side. 2. a given partition on the outer side
+ * doesn't have a matching partition on the inner side. We can not support the
+ * first case since we don't have a way to represent multiple partitions as a
+ * single relation (RelOptInfo) and then perform join using the ganged
+ * relation. We can not support the second case since the missing inner
+ * partition needs to be represented as an empty relation and we don't have a
+ * way to introduce empty relation during join planning after creating paths
+ * for all the base relations.
+ */
+PartitionBoundInfo
+partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
+{
+	PartitionBoundInfo 	merged_bounds;
+	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
+						inner_binfo = inner_rel->boundinfo;
+	char				strategy = outer_binfo->strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = NULL;
+
+			break;
+
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partsupfunc,
+														partcollation,
+														outer_rel,
+														inner_rel,
+														jointype,
+														outer_parts,
+														inner_parts);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(partnatts,
+														 partsupfunc,
+														 partcollation,
+														 outer_rel,
+														 inner_rel,
+														 jointype,
+														 outer_parts,
+														 inner_parts);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	return merged_bounds;
+}
+
+/*
+ * partition_get_range_bounds
+ *
+ * Given the index of lower bound in datums array, return lower and upper
+ * bounds and the index of the partition with that lower bound.
+ */
+static int
+partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
+						   PartitionRangeBound *lower,
+						   PartitionRangeBound *upper)
+{
+	int			part_index;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The lower bound should correspond to a valid partition. */
+	part_index = bi->indexes[lb_index + 1];
+	Assert(part_index >= 0);
+
+	lower->kind = bi->kind[lb_index];
+	lower->datums = bi->datums[lb_index];
+	lower->lower = true;
+	upper->kind = bi->kind[lb_index + 1];
+	upper->datums = bi->datums[lb_index + 1];
+	upper->lower = false;
+
+	return part_index;
+}
+
+/*
+ * partition_range_get_next_lb_index
+ *
+ * Given the index of lower bound in datums array return the
+ * index of lower bound of the next partition. When the given index corresponds
+ * to the last partition, return number of datums (ndatums).
+ */
+static int
+partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
+{
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The partition index corresponding to the upper bound should be valid. */
+	Assert(bi->indexes[lb_index + 1] >= 0);
+
+	/*
+	 * If there are no bounds left beyond the upper bound, we have reached the
+	 * last partition.
+	 */
+	if (lb_index + 2 < bi->ndatums)
+	{
+		/*
+		 * If the bound next to the upper bound corresponds to no partition,
+		 * that's the next lower bound of the next partition. Otherwise, the
+		 * current upper bound is the lower bound of the next partition.
+		 */
+		if (bi->indexes[lb_index + 2] < 0)
+			return lb_index + 2;
+		else
+			return lb_index + 1;
+	}
+	else
+		return bi->ndatums;
+}
+
+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *partcollations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2)
+{
+	return partition_rbound_cmp(partnatts, partsupfunc, partcollations,
+								bound1->datums, bound1->kind, bound1->lower,
+								bound2);
+}
+
+/*
+ * partition_range_cmp
+ *
+ * Compare the bounds of two range partitions. Set ub_cmpval <, = or > 0, if the
+ * first partition's upper bound is lower than, equal to or higher than the
+ * second partition's upper bound resp. Similarly set lb_cmpval <, =  or > 0,
+ * if the first partition's lower bound is lower than, equal to or higher than
+ * the second partition's lower bound resp.
+ *
+ * Return true, if the ranges overlap, otherwise return false.
+ */
+static bool
+partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
+					PartitionRangeBound *lower_bound1,
+					PartitionRangeBound *upper_bound1,
+					PartitionRangeBound *lower_bound2,
+					PartitionRangeBound *upper_bound2, int *ub_cmpval,
+					int *lb_cmpval)
+{
+	bool		overlap;
+
+	/*
+	 * Compare upper bound of the first partition with the lower bound of the
+	 * second and vice-versa. If lower bound is higher than the upper bound,
+	 * the partitions are not overlapping. All other cases indicate overlapping
+	 * partitions.
+	 */
+	if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+								  lower_bound1, upper_bound2) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = 1;
+		*lb_cmpval = 1;
+	}
+	else if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+									   lower_bound2, upper_bound1) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = -1;
+		*lb_cmpval = -1;
+	}
+	else
+	{
+		overlap = true;
+		*ub_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, upper_bound1,
+											   upper_bound2);
+		*lb_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, lower_bound1,
+											   lower_bound2);
+	}
+
+	return overlap;
+}
+
+/*
+ * partition_range_merge
+ *
+ * Merge the partition bounds of given two partitions such that the join
+ * between the given two partitions fits merged bounds.
+ *
+ * "merged_upper" will be set to one of the given upper bounds and
+ * "merged_lower" will be set to one of the given lower bounds.
+ */
+static void
+partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
+					  Oid *partcollations, JoinType jointype,
+					  PartitionRangeBound *left_lb,
+					  PartitionRangeBound *left_ub,
+					  PartitionRangeBound *right_lb,
+					  PartitionRangeBound *right_ub,
+					  PartitionRangeBound **merged_lb,
+					  PartitionRangeBound **merged_ub)
+{
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = left_ub;
+			*merged_lb = left_lb;
+			break;
+
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) < 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) > 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		case JOIN_FULL:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) > 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) < 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+}
+
+/*
+ * Add the lower bound of the next range to the list of bounds, if the lower
+ * bound is higher or equal to the previous upper bound. If successful return
+ * true, otherwise false.
+ */
+static bool
+partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
+							  Oid *partcollations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		prev_ub.datums = llast(*merged_datums);
+		prev_ub.kind = llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  next_lb_datums, next_lb_kind, false,
+									  &prev_ub);
+	}
+
+	/*
+	 * The lower bound is lower than the last upper bound, thus does not fit
+	 * the bounds created so far and hence can not be merged with the existing
+	 * bounds.
+	 */
+	if (cmpval < 0)
+		return false;
+
+	/*
+	 * Add bounds of the new merged partition. If the next lower bound is
+	 * higher than the last upper bound, add new range with index
+	 * corresponding to the lower bound as -1. If the merged lower bound
+	 * is same as the last merged upper bound, the last upper bound will be
+	 * reused as the lower bound of the next range.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, next_lb_datums);
+		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	return true;
+}
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			next_index = 0;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_kinds = NIL;
+	List	   *merged_indexes = NIL;
+	int			outer_lb_index;
+	int			inner_lb_index;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = inner_lb_index = 0;
+	while (outer_lb_index < outer_bi->ndatums ||
+		   inner_lb_index < inner_bi->ndatums)
+	{
+		PartitionRangeBound *merged_lb = NULL;
+		PartitionRangeBound *merged_ub = NULL;
+		int			merged_index = -1;
+		PartitionRangeBound outer_lb;
+		PartitionRangeBound outer_ub;
+		PartitionRangeBound inner_lb;
+		PartitionRangeBound inner_ub;
+		int			outer_part = -1;
+		int			inner_part = -1;
+		bool		overlap;
+		int			ub_cmpval;
+		int			lb_cmpval;
+
+		/* Get the range bounds of the next pair of partitions. */
+		if (outer_lb_index < outer_bi->ndatums)
+			outer_part = partition_get_range_bounds(outer_bi, outer_lb_index,
+													&outer_lb, &outer_ub);
+		if (inner_lb_index < inner_bi->ndatums)
+			inner_part = partition_get_range_bounds(inner_bi, inner_lb_index,
+													&inner_lb, &inner_ub);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (outer_lb_index >= outer_bi->ndatums)
+		{
+			overlap = false;
+			ub_cmpval = 1;
+			lb_cmpval = 1;
+		}
+		else if (inner_lb_index >= inner_bi->ndatums)
+		{
+			overlap = false;
+			ub_cmpval = -1;
+			lb_cmpval = -1;
+		}
+		else
+			overlap = partition_range_cmp(partnatts, partsupfuncs, partcollations,
+										  &outer_lb, &outer_ub, &inner_lb,
+										  &inner_ub, &ub_cmpval, &lb_cmpval);
+
+		if (overlap)
+		{
+			/*
+			 * The rows from overlapping portion of ranges on both sides may
+			 * join, hence the corresponding pair of partitions form a joining
+			 * pair. Match them and produce the bounds of the joint partition
+			 * and its index by merging the bounds according to the type of
+			 * join.
+			 */
+			partition_range_merge(partnatts, partsupfuncs, partcollations,
+								  jointype, &outer_lb, &outer_ub, &inner_lb,
+								  &inner_ub, &merged_lb, &merged_ub);
+
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
+													outer_part, inner_part,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				/* Failed to match the partitions. */
+				return NULL;
+			}
+
+			/*
+			 * If the ranges overlap but don't exactly match, a row from
+			 * non-overlapping portion of the range from one side of join may
+			 * find its join partner in the previous or next overlapping
+			 * partition or default partition on the other side , if such a
+			 * partition exists. All those cases, if true, will cause one
+			 * partition from that side to match at least two partitions on the
+			 * other side; a case that we do not support now. Previous
+			 * partition has been delt with in the previous iteration of this
+			 * loop, next partition will be delt in the next iteration. We will
+			 * deal with the default partition here.
+			 */
+			if ((lb_cmpval < 0 && inner_has_default) ||
+				/* Non-overlapping range on the lower side of outer range. */
+				(lb_cmpval > 0 && outer_has_default) ||
+				/* Non-overlapping range on the lower side of inner range. */
+				(ub_cmpval < 0 && outer_has_default) ||
+				/* Non-overlapping range on the upper side of inner range. */
+				(ub_cmpval > 0 && inner_has_default))
+				/* Non-overlapping range on the upper side of outer range. */
+				return NULL;
+		}
+
+		if (ub_cmpval == 0)
+		{
+			/* Upper bounds of both the ranges match. */
+			Assert(overlap);
+
+			/* Move to the next pair of partitions. */
+			Assert(outer_lb_index < outer_bi->ndatums);
+			Assert(inner_lb_index < inner_bi->ndatums);
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				if (inner_has_default || IS_OUTER_JOIN(jointype))
+				{
+					if (!process_outer_partition(&outer_map,
+												 &inner_map,
+												 outer_has_default,
+												 inner_has_default,
+												 outer_part,
+												 inner_default,
+												 jointype,
+												 outer_bi->strategy,
+												 &next_index,
+												 &default_index,
+												 &merged_index))
+						return NULL;
+				}
+
+				merged_lb = &outer_lb;
+				merged_ub = &outer_ub;
+			}
+
+			/* Move to the next partition on the outer side. */
+			Assert(outer_lb_index < outer_bi->ndatums);
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+		}
+		else
+		{
+			Assert(ub_cmpval > 0);
+
+			/* Upper bound of outer range higher than that of the inner. */
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				if (outer_has_default || jointype == JOIN_FULL)
+				{
+					if (!process_inner_partition(&outer_map,
+												 &inner_map,
+												 outer_has_default,
+												 inner_has_default,
+												 inner_part,
+												 outer_default,
+												 jointype,
+												 outer_bi->strategy,
+												 &next_index,
+												 &default_index,
+												 &merged_index))
+						return NULL;
+				}
+
+				merged_lb = &inner_lb;
+				merged_ub = &inner_ub;
+			}
+
+			/* Move to the next partition on the inner side. */
+			Assert(inner_lb_index < inner_bi->ndatums);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+
+		if (merged_index < 0)
+		{
+			/* We didn't find a new merged partition. */
+			continue;
+		}
+
+		/*
+		 * We have a valid partition index for the next partition of join. The
+		 * partition should have valid range.
+		 */
+		Assert(merged_lb && merged_ub);
+
+		/* Try merging new lower bound with the last upper bound. */
+		if (!partition_range_merge_next_lb(partnatts, partsupfuncs,
+										   partcollations,
+										   merged_lb->datums,
+										   merged_lb->kind, &merged_datums,
+										   &merged_kinds, &merged_indexes))
+			return NULL;
+
+		/* Add upper bound with the merged partition index. */
+		merged_datums = lappend(merged_datums, merged_ub->datums);
+		merged_kinds = lappend(merged_kinds, merged_ub->kind);
+		merged_indexes = lappend_int(merged_indexes, merged_index);
+	}
+
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+	{
+		if (!merge_default_partitions(&outer_map,
+									  &inner_map,
+									  outer_has_default,
+									  outer_default,
+									  inner_has_default,
+									  inner_default,
+									  jointype,
+									  &next_index,
+									  &default_index))
+			return NULL;
+	}
+	else
+		Assert(default_index == -1);
+
+	Assert(!outer_map.did_remapping && !inner_map.did_remapping);
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 &outer_map, &inner_map,
+								 next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes,
+													  merged_kinds,
+													  -1, default_index);
+	}
+
+	/* Free any memory we used in this function. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							JoinType jointype,
+							List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			next_index = 0;
+	int			null_index = -1;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			cnto;
+	int			cnti;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *merged_datum = NULL;
+		int			merged_index = -1;
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			cmpval;
+
+		/* Get the list datums of the next pair of partitions. */
+		odatums = cnto < outer_bi->ndatums ? outer_bi->datums[cnto] : NULL;
+		idatums = cnti < inner_bi->ndatums ? inner_bi->datums[cnti] : NULL;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+			cmpval = 1;
+		else if (cnti >= inner_bi->ndatums)
+			cmpval = -1;
+		else
+		{
+			Assert(odatums != NULL && idatums != NULL);
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0],
+													 idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			int			o_index = outer_bi->indexes[cnto];
+			int			i_index = inner_bi->indexes[cnti];
+
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			Assert(o_index >= 0 && i_index >= 0);
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
+													o_index, i_index,
+													&next_index);
+
+			if (merged_index < 0)
+				return NULL;
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			Assert(cnto < outer_bi->ndatums);
+
+			/* A datum missing from the inner side. */
+			merged_datum = odatums;
+
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				int			o_index = outer_bi->indexes[cnto];
+
+				Assert(o_index >= 0);
+				if (!process_outer_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 o_index,
+											 inner_default,
+											 jointype,
+											 outer_bi->strategy,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next datum on the outer side. */
+			cnto++;
+		}
+		else
+		{
+			Assert(cmpval > 0);
+			Assert(cnti < inner_bi->ndatums);
+
+			/* A datum missing from the outer side. */
+			merged_datum = idatums;
+
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				int			i_index = inner_bi->indexes[cnti];
+
+				Assert(i_index >= 0);
+				if (!process_inner_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 i_index,
+											 outer_default,
+											 jointype,
+											 outer_bi->strategy,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next datum on the inner side. */
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	/* Merge null partitions if any. */
+	if (outer_has_null || inner_has_null)
+	{
+		if (!merge_null_partitions(outer_bi, inner_bi,
+								   &outer_map, &inner_map,
+								   outer_has_default, inner_has_default,
+								   outer_has_null, inner_has_null,
+								   jointype, &next_index, &default_index,
+								   &null_index))
+			return NULL;
+	}
+	else
+		Assert(null_index == -1);
+
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+	{
+		if (!merge_default_partitions(&outer_map, &inner_map,
+									  outer_has_default, outer_default,
+									  inner_has_default, inner_default,
+									  jointype, &next_index, &default_index))
+			return NULL;
+	}
+	else
+		Assert(default_index == -1);
+
+	/* Fix the merged_indexes list if necessary. */
+	if (outer_map.did_remapping || inner_map.did_remapping)
+	{
+		Assert(jointype == JOIN_FULL);
+		fix_merged_indexes(&outer_map, &inner_map,
+						   next_index, merged_indexes);
+	}
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 &outer_map, &inner_map,
+								 next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) <= next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes, NIL,
+													  null_index, default_index);
+	}
+
+	/* Free up all extra memory before returning from this function. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+	list_free(merged_datums);
+	list_free(merged_indexes);
+
+	return merged_bounds;
+}
+
+/*
+ * init_partition_map
+ *
+ * Initialize a PartitionMap struct for given relation.
+ */
+static void
+init_partition_map(RelOptInfo *rel, PartitionMap *map)
+{
+	int			nparts = rel->nparts;
+	int			i;
+
+	map->nparts = nparts;
+	map->merged_indexes = (int *) palloc(sizeof(int) * nparts);
+	map->merged = (bool *) palloc(sizeof(bool) * nparts);
+	map->did_remapping = false;
+	map->old_indexes = (int *) palloc(sizeof(int) * nparts);
+	for (i = 0; i < nparts; i++)
+	{
+		map->merged_indexes[i] = map->old_indexes[i] = -1;
+		map->merged[i] = false;
+	}
+}
+
+/*
+ * free_partition_map
+ */
+static void
+free_partition_map(PartitionMap *map)
+{
+	pfree(map->merged_indexes);
+	pfree(map->merged);
+	pfree(map->old_indexes);
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given partitions.
+ */
+static int
+map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_index, int inner_index, int *next_index)
+{
+	int 		outer_merged_index;
+	bool 		outer_merged;
+	int 		inner_merged_index;
+	bool 		inner_merged;
+
+	Assert(outer_index >= 0 && outer_index < outer_map->nparts);
+	outer_merged_index = outer_map->merged_indexes[outer_index];
+	outer_merged = outer_map->merged[outer_index];
+	Assert(inner_index >= 0 && inner_index < inner_map->nparts);
+	inner_merged_index = inner_map->merged_indexes[inner_index];
+	inner_merged = inner_map->merged[inner_index];
+
+	/*
+	 * Handle cases where both partitions are mapped to merged partitions.
+	 */
+	if (outer_merged_index >= 0 && inner_merged_index >= 0)
+	{
+		/*
+		 * If the mereged partitions are the same, no need to do anything;
+		 * return the index of the merged partition.  Otherwise, if both
+		 * partitions are merged with dummy partitions, re-merge them; map
+		 * them to the merged partition with the smaller of the two merged
+		 * indexes and return the smaller index.  Otherwise they can't be
+		 * merged, so return -1.
+		 */
+		if (outer_merged_index == inner_merged_index)
+		{
+			Assert(outer_merged);
+			Assert(inner_merged);
+			return outer_merged_index;
+		}
+		if (!outer_merged && !inner_merged)
+		{
+			/*
+			 * Note that we will fix the larger index that have been added to
+			 * the merged_indexes list so far in fix_merged_indexes().
+			 */
+			if (outer_merged_index < inner_merged_index)
+			{
+				outer_map->merged[outer_index] = true;
+				inner_map->merged_indexes[inner_index] = outer_merged_index;
+				inner_map->merged[inner_index] = true;
+				inner_map->did_remapping = true;
+				inner_map->old_indexes[inner_index] = inner_merged_index;
+				return outer_merged_index;
+			}
+			else
+			{
+				inner_map->merged[inner_index] = true;
+				outer_map->merged_indexes[outer_index] = inner_merged_index;
+				outer_map->merged[outer_index] = true;
+				outer_map->did_remapping = true;
+				outer_map->old_indexes[outer_index] = outer_merged_index;
+				return inner_merged_index;
+			}
+		}
+		return -1;
+	}
+
+	/* At least one partition isn't mapped to a merged partition. */
+	Assert(outer_merged_index == -1 || inner_merged_index == -1);
+
+	/*
+	 * If neither of partitions isn't mapped, assign them a new merged
+	 * partition and return the index of the merged partition.  Otherwise, if
+	 * one of partitions is merged with a dummy relation (and the other isn't
+	 * merged), re-merge it with the other, with the same index, and return
+	 * the index.  Otherwise they can't be merged, so return -1.
+	 */
+	if (outer_merged_index == -1 && inner_merged_index == -1)
+	{
+		int 		merged_index = *next_index;
+
+		Assert(!outer_merged);
+		Assert(!inner_merged);
+		outer_map->merged_indexes[outer_index] = merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged_indexes[inner_index] = merged_index;
+		inner_map->merged[inner_index] = true;
+		*next_index = *next_index + 1;
+		return merged_index;
+	}
+	if (outer_merged_index >= 0 && !outer_map->merged[outer_index])
+	{
+		Assert(inner_merged_index == -1);
+		Assert(!inner_merged);
+		inner_map->merged_indexes[inner_index] = outer_merged_index;
+		inner_map->merged[inner_index] = true;
+		outer_map->merged[outer_index] = true;
+		return outer_merged_index;
+	}
+	if (inner_merged_index >= 0 && !inner_map->merged[inner_index])
+	{
+		Assert(outer_merged_index == -1);
+		Assert(!outer_merged);
+		outer_map->merged_indexes[outer_index] = inner_merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged[inner_index] = true;
+		return inner_merged_index;
+	}
+	return -1;
+}
+
+/*
+ * merge_partition_with_dummy
+ *
+ * *next_index is incremented.
+ */
+static int
+merge_partition_with_dummy(PartitionMap *map, int index, int *next_index)
+{
+	int 		merged_index = *next_index;
+
+	Assert(index >= 0 && index < map->nparts);
+	Assert(map->merged_indexes[index] == -1);
+	Assert(!map->merged[index]);
+	map->merged_indexes[index] = merged_index;
+	/* Leave the merged flag alone! */
+	*next_index = *next_index + 1;
+	return merged_index;
+}
+
+/*
+ * process_outer_partition
+ *
+ * Determine the merged partition associated with the given outer partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given outer partition.
+ */
+static bool
+process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index)
+{
+	Assert(outer_index >= 0);
+
+	/*
+	 * If the inner side has the default partition, the outer partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an outer join, in which case the outer partition has to be
+	 * scanned all the way anyway; if the outer partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the outer partition with a dummy partition.
+	 */
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0);
+
+		/*
+		 * If the outer side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (outer_has_default)
+			return false;
+
+		*merged_index = map_and_merge_partitions(outer_map, inner_map,
+												 outer_index, inner_default,
+												 next_index);
+		if (*merged_index == -1)
+			return false;
+
+		/*
+		 * If this is a FULL join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (*default_index == -1)
+				*default_index = *merged_index;
+			else
+				Assert(*merged_index == *default_index);
+			/* Don't add this index to the list of merged indexes. */
+			*merged_index = -1;
+		}
+	}
+	else
+	{
+		Assert(IS_OUTER_JOIN(jointype));
+		Assert(jointype != JOIN_RIGHT);
+
+		/*
+		 * In range partitioning, if the given outer partition is already
+		 * merged (eg, because we found an overlapping range earlier), we know
+		 * where it fits in the join result; nothing to do in that case.  Else
+		 * create a new merged partition.
+		 */
+		if (outer_map->merged_indexes[outer_index] >= 0)
+		{
+			if (strategy == PARTITION_STRATEGY_LIST)
+				*merged_index = outer_map->merged_indexes[outer_index];
+			else
+			{
+				Assert(strategy == PARTITION_STRATEGY_RANGE);
+				*merged_index = -1;
+			}
+		}
+		else
+			*merged_index = merge_partition_with_dummy(outer_map, outer_index,
+													   next_index);
+	}
+	return true;
+}
+
+/*
+ * process_inner_partition
+ *
+ * Determine the merged partition associated with the given inner partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given inner partition.
+ */
+static bool
+process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index)
+{
+	Assert(inner_index >= 0);
+
+	/*
+	 * If the outer side has the default partition, the inner partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an FULL join, in which case the inner partition has to be
+	 * scanned all the way anyway; if the inner partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the inner partition with a dummy partition.
+	 */
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0);
+
+		/*
+		 * If the inner side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (inner_has_default)
+			return false;
+
+		*merged_index = map_and_merge_partitions(outer_map, inner_map,
+												 outer_default, inner_index,
+												 next_index);
+		if (*merged_index == -1)
+			return false;
+
+		/*
+		 * If this is an outer join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (jointype == JOIN_LEFT || jointype == JOIN_ANTI ||
+			jointype == JOIN_FULL)
+		{
+			if (*default_index == -1)
+				*default_index = *merged_index;
+			else
+				Assert(*merged_index == *default_index);
+			/* Don't add this index to the list of merged indexes. */
+			*merged_index = -1;
+		}
+	}
+	else
+	{
+		Assert(jointype == JOIN_FULL);
+
+		/*
+		 * In range partitioning, if the given inner partition is already
+		 * merged (eg, because we found an overlapping range earlier), we know
+		 * where it fits in the join result; nothing to do in that case.  Else
+		 * create a new merged partition.
+		 */
+		if (inner_map->merged_indexes[inner_index] >= 0)
+		{
+			if (strategy == PARTITION_STRATEGY_LIST)
+				*merged_index = inner_map->merged_indexes[inner_index];
+			else
+			{
+				Assert(strategy == PARTITION_STRATEGY_RANGE);
+				*merged_index = -1;
+			}
+		}
+		else
+			*merged_index = merge_partition_with_dummy(inner_map, inner_index,
+													   next_index);
+	}
+	return true;
+}
+
+/*
+ * fix_merged_indexes
+ */
+static void
+fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes)
+{
+	int		   *new_indexes;
+	int			merged_index;
+	int			i;
+	ListCell   *lc;
+
+	new_indexes = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		new_indexes[i] = -1;
+
+	/* Build the mapping of old merged indexes to new merged indexes. */
+	if (outer_map->did_remapping)
+	{
+		for (i = 0; i < outer_map->nparts; i++)
+		{
+			merged_index = outer_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = outer_map->merged_indexes[i];
+		}
+	}
+	if (inner_map->did_remapping)
+	{
+		for (i = 0; i < inner_map->nparts; i++)
+		{
+			merged_index = inner_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = inner_map->merged_indexes[i];
+		}
+	}
+
+	/* Fix the merged_indexes list using the mapping. */
+	foreach(lc, merged_indexes)
+	{
+		merged_index = lfirst_int(lc);
+		Assert(merged_index >= 0);
+		if (new_indexes[merged_index] >= 0)
+			lfirst_int(lc) = new_indexes[merged_index];
+	}
+
+	pfree(new_indexes);
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * This function produces the list pairs of partitions which produce merged
+ * partitions in the order of merged partition indexes.
+ *
+ * nmerged is the number of merged partitions.
+ *
+ * If successful, the list pairs are returned as two separate lists,
+ * outer_part_list and inner_part_list, one for each side.  Otherwise, those
+ * lists will be set to NIL.
+ */
+static void
+generate_matching_part_pairs(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 PartitionMap *outer_map, PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_part_list, List **inner_part_list)
+{
+	int			outer_nparts = outer_map->nparts;
+	int			inner_nparts = inner_map->nparts;
+	int		   *outer_part_array;
+	int		   *inner_part_array;
+	int			max_nparts;
+	int 		i;
+
+	Assert(outer_nparts == outer_rel->nparts);
+	Assert(inner_nparts == inner_rel->nparts);
+
+	outer_part_array = (int *) palloc(sizeof(int) * nmerged);
+	inner_part_array = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		outer_part_array[i] = inner_part_array[i] = -1;
+
+	/* Set pairs of matching partitions. */
+	max_nparts = Max(outer_nparts, inner_nparts);
+	for (i = 0; i < max_nparts; i++)
+	{
+		if (i < outer_nparts)
+		{
+			int 		merged_index = outer_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				outer_part_array[merged_index] = i;
+			}
+		}
+		if (i < inner_nparts)
+		{
+			int 		merged_index = inner_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				inner_part_array[merged_index] = i;
+			}
+		}
+	}
+
+	*outer_part_list = NIL;
+	*inner_part_list = NIL;
+
+	/* Build the list pairs. */
+	for (i = 0; i < nmerged; i++)
+	{
+		int			outer_part = outer_part_array[i];
+		int			inner_part = inner_part_array[i];
+
+		/*
+		 * If both partitions are dummy, it means the merged partition was
+		 * removed by partition re-merging in map_and_merge_partitions();
+		 * ignore the merged partition
+		 */
+		if (outer_part == -1 && inner_part == -1)
+			continue;
+
+		*outer_part_list = lappend(*outer_part_list, outer_part >= 0 ?
+								   outer_rel->part_rels[outer_part] : NULL);
+		*inner_part_list = lappend(*inner_part_list, inner_part >= 0 ?
+								   inner_rel->part_rels[inner_part] : NULL);
+	}
+
+	pfree(outer_part_array);
+	pfree(inner_part_array);
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * Merge default partitions from both sides, if any, and assign the default
+ * partition for the join result, if necessary.
+ *
+ * If both the relations have default partitions, try mapping those to each
+ * other. If the mapping succeeds corresponding merged partition will act as
+ * the default partition of the join result.
+ *
+ * If inner side of the join has default but not the outer side, rows in it
+ * won't appear in the join result. So don't create a default partition. If
+ * outer side of the join has default but not the inner side, rows in it will
+ * appear in the join result, so create a default merged partition.
+ */
+static bool
+merge_default_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 bool outer_has_default, int outer_default,
+						 bool inner_has_default, int inner_default,
+						 JoinType jointype, int *next_index,
+						 int *default_index)
+{
+	Assert(outer_has_default || inner_has_default);
+
+	if (outer_has_default && !inner_has_default)
+	{
+		if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+			jointype == JOIN_ANTI)
+		{
+			int			merged_index;
+
+			Assert(outer_default >= 0 && outer_default < outer_map->nparts);
+			merged_index = outer_map->merged_indexes[outer_default];
+			if (merged_index == -1)
+			{
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(outer_map,
+															outer_default,
+															next_index);
+			}
+			else
+				Assert(*default_index == merged_index);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else if (!outer_has_default && inner_has_default)
+	{
+		if (jointype == JOIN_FULL)
+		{
+			int			merged_index;
+
+			Assert(inner_default >= 0 && inner_default < inner_map->nparts);
+			merged_index = inner_map->merged_indexes[inner_default];
+			if (merged_index == -1)
+			{
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(inner_map,
+															inner_default,
+															next_index);
+			}
+			else
+				Assert(*default_index == merged_index);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		*default_index = map_and_merge_partitions(outer_map,
+												  inner_map,
+												  outer_default,
+												  inner_default,
+												  next_index);
+		if (*default_index == -1)
+			return false;
+	}
+
+	return true;
+}
+
+/*
+ * merge_null_partitions
+ *
+ * Merge NULL partitions, i.e. a partition that can hold NULL values for a list
+ * partitioned table, if any. Find the index of merged partition to which the
+ * NULL values would belong in the join result. If one joining relation has a
+ * NULL partition but not the other, try matching it with the default partition
+ * from the other relation since the default partition may have rows with NULL
+ * partition key. We can eliminate a NULL partition when it appears only on the
+ * inner side of the join and the outer side doesn't have a default partition.
+ *
+ * When the equality operator used for join is strict, two NULL values will not
+ * be considered as equal, and thus a NULL partition can be eliminated for an
+ * inner join. But we don't check the strictness operator here.
+ */
+static bool
+merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_map, PartitionMap *inner_map,
+					  bool outer_has_default, bool inner_has_default,
+					  bool outer_has_null, bool inner_has_null,
+					  JoinType jointype, int *next_index,
+					  int *default_index, int *null_index)
+{
+	Assert(outer_has_null || inner_has_null);
+	Assert(*null_index == -1);
+
+	if (outer_has_null && !inner_has_null)
+	{
+		int			merged_index = -1;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join to which the NULL partition matches will
+		 * contain the NULL values and thus become the NULL partition of the
+		 * the join.
+		 */
+		if (inner_has_default || IS_OUTER_JOIN(jointype))
+		{
+			if (!process_outer_partition(outer_map,
+										 inner_map,
+										 outer_has_default,
+										 inner_has_default,
+										 outer_bi->null_index,
+										 inner_bi->default_index,
+										 jointype,
+										 outer_bi->strategy,
+										 next_index,
+										 default_index,
+										 &merged_index))
+				return false;
+		}
+		*null_index = merged_index;
+	}
+	else if (!outer_has_null && inner_has_null)
+	{
+		int			merged_index = -1;
+
+		/*
+		 * If the NULL partition was missing from the outer side of the join,
+		 * the partition of the join to which the NULL partition matches will
+		 * contain the NULL values and thus become the NULL partition of the
+		 * the join.
+		 */
+		if (outer_has_default || jointype == JOIN_FULL)
+		{
+			if (!process_inner_partition(outer_map,
+										 inner_map,
+										 outer_has_default,
+										 inner_has_default,
+										 inner_bi->null_index,
+										 outer_bi->default_index,
+										 jointype,
+										 outer_bi->strategy,
+										 next_index,
+										 default_index,
+										 &merged_index))
+				return false;
+		}
+		*null_index = merged_index;
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		*null_index = map_and_merge_partitions(outer_map,
+											   inner_map,
+											   outer_bi->null_index,
+											   inner_bi->null_index,
+											   next_index);
+		if (*null_index == -1)
+			return false;
+	}
+
+	return true;
+}
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index 23a06d718e..e06eb9aaae 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -577,8 +577,10 @@ typedef struct PartitionSchemeData *PartitionScheme;
  *		part_scheme - Partitioning scheme of the relation
  *		nparts - Number of partitions
  *		boundinfo - Partition bounds
+ *		merged - true if partition bounds are merged ones
  *		partition_qual - Partition constraint if not the root
  *		part_rels - RelOptInfos for each partition
+ *		all_partrels - Relids set of all partition relids
  *		partexprs, nullable_partexprs - Partition key expressions
  *		partitioned_child_rels - RT indexes of unpruned partitions of
  *								 this relation that are partitioned tables
@@ -718,9 +720,12 @@ typedef struct RelOptInfo
 	PartitionScheme part_scheme;	/* Partitioning scheme. */
 	int			nparts;			/* number of partitions */
 	struct PartitionBoundInfoData *boundinfo;	/* Partition bounds */
+	bool		merged;			/* true if partition bounds were created by
+								 * partition_bounds_merge() */
 	List	   *partition_qual; /* partition constraint */
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of partitions,
 									 * stored in the same order of bounds */
+	Relids		all_partrels;	/* Relids set of all partition relids */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 	List	   *partitioned_child_rels; /* List of RT indexes. */
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index 0d0fd42b18..9292aa11e5 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -16,6 +16,7 @@
 #include "nodes/pg_list.h"
 #include "partitioning/partdefs.h"
 #include "utils/relcache.h"
+struct RelOptInfo;				/* avoid including pathnodes.h here */
 
 
 /*
@@ -108,5 +109,11 @@ extern int	partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 										  int nvalues, Datum *values, bool *is_equal);
 extern int	partition_hash_bsearch(PartitionBoundInfo boundinfo,
 								   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   struct RelOptInfo *outer_rel, struct RelOptInfo *inner_rel,
+					   JoinType jointype,
+					   List **outer_parts, List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index cad8dd591a..89f0c6a9ee 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -8,59 +8,86 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-(4 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+(7 rows)
 
 -- left outer join, with whole-row reference; partitionwise join does not apply
 EXPLAIN (COSTS OFF)
@@ -72,35 +99,50 @@ SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER
    ->  Hash Right Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(22 rows)
 
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-      t1      |      t2      
---------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
- (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
- (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
- (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
- (550,0,0550) | 
-(12 rows)
+       t1       |       t2       
+----------------+----------------
+ (-250,0,-0250) | 
+ (-200,0,-0200) | 
+ (-150,0,-0150) | (0,-150,-0150)
+ (-100,0,-0100) | 
+ (-50,0,-0050)  | 
+ (0,0,0000)     | (0,0,0000)
+ (50,0,0050)    | 
+ (100,0,0100)   | 
+ (150,0,0150)   | (0,150,0150)
+ (200,0,0200)   | 
+ (250,0,0250)   | 
+ (300,0,0300)   | (0,300,0300)
+ (350,0,0350)   | 
+ (400,0,0400)   | 
+ (450,0,0450)   | (0,450,0450)
+ (500,0,0500)   | 
+ (550,0,0550)   | 
+ (600,0,0600)   | (0,600,0600)
+ (650,0,0650)   | 
+ (700,0,0700)   | 
+ (750,0,0750)   | (0,750,0750)
+(21 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
@@ -112,35 +154,53 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
    ->  Append
          ->  Hash Right Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Right Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t2_2.b)
-(20 rows)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+      |       | -225 | -0225
+      |       |  -75 | -0075
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(14 rows)
 
 -- full outer join, with placeholder vars
 EXPLAIN (COSTS OFF)
@@ -148,8 +208,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                             QUERY PLAN                            
 ------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               Filter: (((50) = prt1_p0.a) OR ((75) = prt2_p0.b))
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
@@ -174,7 +242,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               Filter: (((50) = prt1_p4.a) OR ((75) = prt2_p4.b))
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
@@ -211,35 +287,44 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Hash Right Join
-         Hash Cond: (prt2_p2.b = prt1_p1.a)
+         Hash Cond: (prt2_p2.b = prt1_p0.a)
          ->  Append
                ->  Seq Scan on prt2_p2
                      Filter: (b > 250)
                ->  Seq Scan on prt2_p3
                      Filter: (b > 250)
+               ->  Seq Scan on prt2_p4
+                     Filter: (b > 250)
          ->  Hash
                ->  Append
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p1
                            Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p2
                            Filter: ((a < 450) AND (b = 0))
-(15 rows)
+(19 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-(9 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+(14 rows)
 
 -- Currently we can't do partitioned join if nullable-side partitions are pruned
 EXPLAIN (COSTS OFF)
@@ -247,11 +332,13 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                       QUERY PLAN                      
 ------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Hash Full Join
-         Hash Cond: (prt1_p1.a = prt2_p2.b)
-         Filter: ((prt1_p1.b = 0) OR (prt2_p2.a = 0))
+         Hash Cond: (prt1_p0.a = prt2_p2.b)
+         Filter: ((prt1_p0.b = 0) OR (prt2_p2.a = 0))
          ->  Append
+               ->  Seq Scan on prt1_p0
+                     Filter: (a < 450)
                ->  Seq Scan on prt1_p1
                      Filter: (a < 450)
                ->  Seq Scan on prt1_p2
@@ -262,64 +349,147 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                            Filter: (b > 250)
                      ->  Seq Scan on prt2_p3
                            Filter: (b > 250)
-(16 rows)
+                     ->  Seq Scan on prt2_p4
+                           Filter: (b > 250)
+(20 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
-(12 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+      |       | 375 | 0375
+      |       | 450 | 0450
+      |       | 525 | 0525
+      |       | 600 | 0600
+      |       | 675 | 0675
+      |       | 750 | 0750
+(20 rows)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Semi Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Semi Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                            Filter: (a = 0)
-         ->  Nested Loop Semi Join
-               Join Filter: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
-               ->  Materialize
-                     ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
-(24 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt1_p3 t2_3
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+                           Filter: (b = 0)
+(37 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
 
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
@@ -330,27 +500,82 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
    ->  Append
          ->  Hash Anti Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Hash Anti Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Hash Anti Join
                Hash Cond: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
-(17 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(27 rows)
 
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
-  sum  |         avg          | sum  |         avg         
--------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+  sum  |         avg          | sum  |        avg         
+-------+----------------------+------+--------------------
+ 95550 | 273.0000000000000000 | 2200 | 6.2857142857142857
 (1 row)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Append
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p0 t1
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+               Index Cond: (a = t1.b)
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.b = t2_1.a)
+         ->  Seq Scan on prt2_p1 t1_1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p1 t2_1
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p2 t1_2
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+               Index Cond: (a = t1_2.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p3 t1_3
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+               Index Cond: (a = t1_3.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p4 t1_4
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+               Index Cond: (a = t1_4.b)
+(27 rows)
+
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+  b   |   c   
+------+-------
+ -225 | -0225
+  -75 | -0075
+   75 | 0075
+  225 | 0225
+  375 | 0375
+  525 | 0525
+  675 | 0675
+(7 rows)
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -362,49 +587,74 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
+                     ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
                            Index Cond: (a = t1.a)
-                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t3
                            Index Cond: (b = t2.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_1
+                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
                            Index Cond: (a = t1_1.a)
-                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3_1
                            Index Cond: (b = t2_1.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_2
+                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
                            Index Cond: (a = t1_2.a)
-                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_2
                            Index Cond: (b = t2_2.a)
-(27 rows)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                           Index Cond: (a = t1_3.a)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_3
+                           Index Cond: (b = t2_3.a)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                           Index Cond: (a = t1_4.a)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t3_4
+                           Index Cond: (b = t2_4.a)
+(43 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   | t2a | t3a | least 
------+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
- 100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
- 250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
- 400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
- 550 | 0 | 0550 |     |     |      
-(12 rows)
+  a   | b |   c   | t2a  | t3a | least 
+------+---+-------+------+-----+-------
+ -250 | 0 | -0250 |      |     |      
+ -200 | 0 | -0200 |      |     |      
+ -150 | 0 | -0150 | -150 |   0 |  -150
+ -100 | 0 | -0100 |      |     |      
+  -50 | 0 | -0050 |      |     |      
+    0 | 0 | 0000  |    0 |   0 |     0
+   50 | 0 | 0050  |      |     |      
+  100 | 0 | 0100  |      |     |      
+  150 | 0 | 0150  |  150 |   0 |   150
+  200 | 0 | 0200  |      |     |      
+  250 | 0 | 0250  |      |     |      
+  300 | 0 | 0300  |  300 |   0 |   300
+  350 | 0 | 0350  |      |     |      
+  400 | 0 | 0400  |      |     |      
+  450 | 0 | 0450  |  450 |   0 |   450
+  500 | 0 | 0500  |      |     |      
+  550 | 0 | 0550  |      |     |      
+  600 | 0 | 0600  |  600 |   0 |   600
+  650 | 0 | 0650  |      |     |      
+  700 | 0 | 0700  |      |     |      
+  750 | 0 | 0750  |  750 |   0 |   750
+(21 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
@@ -418,64 +668,95 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
          Hash Cond: ((t1.c)::text = (t2.c)::text)
          Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Seq Scan on prt1_p1 t1
-               ->  Seq Scan on prt1_p2 t1_1
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
          ->  Hash
                ->  Append
                      ->  Hash Join
                            Hash Cond: (t2.a = t3.b)
-                           ->  Seq Scan on prt1_p1 t2
+                           ->  Seq Scan on prt1_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t3
+                                 ->  Seq Scan on prt2_p0 t3
                      ->  Hash Join
                            Hash Cond: (t2_1.a = t3_1.b)
-                           ->  Seq Scan on prt1_p2 t2_1
+                           ->  Seq Scan on prt1_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t3_1
+                                 ->  Seq Scan on prt2_p1 t3_1
                      ->  Hash Join
                            Hash Cond: (t2_2.a = t3_2.b)
-                           ->  Seq Scan on prt1_p3 t2_2
+                           ->  Seq Scan on prt1_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p3 t3_2
-(26 rows)
+                                 ->  Seq Scan on prt2_p2 t3_2
+                     ->  Hash Join
+                           Hash Cond: (t2_3.a = t3_3.b)
+                           ->  Seq Scan on prt1_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t3_3
+                     ->  Hash Join
+                           Hash Cond: (t2_4.a = t3_4.b)
+                           ->  Seq Scan on prt1_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t3_4
+(38 rows)
 
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, t2.b t2b, t2.c t2c, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
-  a  | t2a | t2c  
------+-----+------
-   0 |   0 | 0000
-  50 |     | 
- 100 |     | 
- 150 | 150 | 0150
- 200 |     | 
- 250 |     | 
- 300 | 300 | 0300
- 350 |     | 
- 400 |     | 
- 450 | 450 | 0450
- 500 |     | 
- 550 |     | 
-(12 rows)
+  a   | t2a  |  t2c  
+------+------+-------
+ -250 |      | 
+ -200 |      | 
+ -150 | -150 | -0150
+ -100 |      | 
+  -50 |      | 
+    0 |    0 | 0000
+   50 |      | 
+  100 |      | 
+  150 |  150 | 0150
+  200 |      | 
+  250 |      | 
+  300 |  300 | 0300
+  350 |      | 
+  400 |      | 
+  450 |  450 | 0450
+  500 |      | 
+  550 |      | 
+  600 |  600 | 0600
+  650 |      | 
+  700 |      | 
+  750 |  750 | 0750
+(21 rows)
 
 --
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
@@ -486,32 +767,49 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
    ->  Append
          ->  Hash Join
                Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
-               ->  Seq Scan on prt2_e_p1 t2
+               ->  Seq Scan on prt2_e_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 t1
+                     ->  Seq Scan on prt1_e_p0 t1
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
-               ->  Seq Scan on prt2_e_p2 t2_1
+               Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+               ->  Seq Scan on prt1_e_p1 t1_1
+                     Filter: (c = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 t1_1
-                           Filter: (c = 0)
+                     ->  Seq Scan on prt2_e_p1 t2_1
          ->  Hash Join
                Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
-               ->  Seq Scan on prt2_e_p3 t2_2
+               ->  Seq Scan on prt2_e_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 t1_2
+                     ->  Seq Scan on prt1_e_p2 t1_2
                            Filter: (c = 0)
-(21 rows)
+         ->  Hash Join
+               Hash Cond: (((t2_3.b + t2_3.a) / 2) = ((t1_3.a + t1_3.b) / 2))
+               ->  Seq Scan on prt2_e_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p3 t1_3
+                           Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (((t2_4.b + t2_4.a) / 2) = ((t1_4.a + t1_4.b) / 2))
+               ->  Seq Scan on prt2_e_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4 t1_4
+                           Filter: (c = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
- 150 | 0 | 150 | 0
- 300 | 0 | 300 | 0
- 450 | 0 | 450 | 0
-(4 rows)
+  a   | c |  b   | c 
+------+---+------+---
+ -250 | 0 | -250 | 0
+ -100 | 0 | -100 | 0
+    0 | 0 |    0 | 0
+    0 | 0 |    0 | 0
+  150 | 0 |  150 | 0
+  300 | 0 |  300 | 0
+  450 | 0 |  450 | 0
+  600 | 0 |  600 | 0
+  750 | 0 |  750 | 0
+(9 rows)
 
 --
 -- N-way join
@@ -524,154 +822,232 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = ((t3.a + t3.b) / 2))
+               Join Filter: (t1.a = t2.b)
                ->  Hash Join
-                     Hash Cond: (t2.b = t1.a)
-                     ->  Seq Scan on prt2_p1 t2
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on prt1_e_p0 t3
                      ->  Hash
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                     Index Cond: (((a + b) / 2) = t2.b)
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
+                     Index Cond: (b = ((t3.a + t3.b) / 2))
          ->  Nested Loop
                Join Filter: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_1.b = t1_1.a)
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
+               ->  Index Scan using iprt1_e_p4_ab2 on prt1_e_p1 t3_1
                      Index Cond: (((a + b) / 2) = t2_1.b)
          ->  Nested Loop
                Join Filter: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_2.b = t1_2.a)
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
+               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_2
                      Index Cond: (((a + b) / 2) = t2_2.b)
-(33 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = ((t3_3.a + t3_3.b) / 2))
+               ->  Nested Loop
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t2_3.b)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t2_4.b)
+               ->  Hash Join
+                     Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                     ->  Seq Scan on prt1_e_p4 t3_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_p4 t1_4
+                                 Filter: (b = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = ((t3_4.a + t3_4.b) / 2))
+(52 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
-(4 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(8 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                          QUERY PLAN                          
---------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Hash Right Join
                Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
-               ->  Seq Scan on prt1_e_p1 t3
+               ->  Seq Scan on prt1_e_p0 t3
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2.b = t1.a)
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_1.a)
-               ->  Seq Scan on prt1_e_p2 t3_1
+               ->  Seq Scan on prt1_e_p1 t3_1
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_1.b = t1_1.a)
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_2.a)
-               ->  Seq Scan on prt1_e_p3 t3_2
+               ->  Seq Scan on prt1_e_p2 t3_2
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_2.b = t1_2.a)
-                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                                        Filter: (b = 0)
-(33 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+               ->  Seq Scan on prt1_e_p4 t3_4
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_4.b = t1_4.a)
+                           ->  Seq Scan on prt2_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_p4 t1_4
+                                       Filter: (b = 0)
+(51 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                            QUERY PLAN                             
--------------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p1 t3
+                           ->  Seq Scan on prt1_e_p0 t3
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p1_b on prt2_p1 t2
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
                      Index Cond: (b = t1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p2 t3_1
+                           ->  Seq Scan on prt1_e_p1 t3_1
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
+               ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
                      Index Cond: (b = t1_1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p3 t3_2
+                           ->  Seq Scan on prt1_e_p2 t3_2
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
                      Index Cond: (b = t1_2.a)
-(30 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_e_p3 t3_3
+                           Filter: (c = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                           Index Cond: (a = ((t3_3.a + t3_3.b) / 2))
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Nested Loop Left Join
+               ->  Hash Right Join
+                     Hash Cond: (t1_4.a = ((t3_4.a + t3_4.b) / 2))
+                     ->  Seq Scan on prt1_p4 t1_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p4 t3_4
+                                 Filter: (c = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = t1_4.a)
+(47 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -680,21 +1056,34 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                                                    QUERY PLAN                                                   
 ----------------------------------------------------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   Sort Key: prt1_p0.a, prt2_p0.b, ((prt1_e_p0.a + prt1_e_p0.b))
    ->  Append
          ->  Hash Full Join
-               Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
-               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               Hash Cond: (prt1_p0.a = ((prt1_e_p0.a + prt1_e_p0.b) / 2))
+               Filter: ((prt1_p0.a = (50)) OR (prt2_p0.b = (75)) OR (((prt1_e_p0.a + prt1_e_p0.b) / 2) = (50)))
                ->  Hash Full Join
-                     Hash Cond: (prt1_p1.a = prt2_p1.b)
-                     ->  Seq Scan on prt1_p1
+                     Hash Cond: (prt1_p0.a = prt2_p0.b)
+                     ->  Seq Scan on prt1_p0
                            Filter: (b = 0)
                      ->  Hash
-                           ->  Seq Scan on prt2_p1
+                           ->  Seq Scan on prt2_p0
                                  Filter: (a = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1
+                     ->  Seq Scan on prt1_e_p0
                            Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = prt1_p1.a)
+               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               ->  Seq Scan on prt1_e_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on prt1_p1
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1
+                                       Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
                Filter: ((prt1_p2.a = (50)) OR (prt2_p2.b = (75)) OR (((prt1_e_p2.a + prt1_e_p2.b) / 2) = (50)))
@@ -721,7 +1110,20 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                ->  Hash
                      ->  Seq Scan on prt1_e_p3
                            Filter: (c = 0)
-(42 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = ((prt1_e_p4.a + prt1_e_p4.b) / 2))
+               Filter: ((prt1_p4.a = (50)) OR (prt2_p4.b = (75)) OR (((prt1_e_p4.a + prt1_e_p4.b) / 2) = (50)))
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = prt2_p4.b)
+                     ->  Seq Scan on prt1_p4
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4
+                                 Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4
+                           Filter: (c = 0)
+(68 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
@@ -739,172 +1141,260 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = t1_3.b)
+               Join Filter: (t1.a = t1_5.b)
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Join
-                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
-                           ->  Seq Scan on prt1_e_p1 t2
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t1_3
+                                 ->  Seq Scan on prt2_p0 t1_5
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
                      Index Cond: (a = ((t2.a + t2.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_1.a = t1_4.b)
+               Join Filter: (t1_1.a = t1_6.b)
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Join
-                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
-                           ->  Seq Scan on prt1_e_p2 t2_1
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_6.b)
+                           ->  Seq Scan on prt1_e_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t1_4
+                                 ->  Seq Scan on prt2_p1 t1_6
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
                      Index Cond: (a = ((t2_1.a + t2_1.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_2.a = t1_5.b)
+               Join Filter: (t1_2.a = t1_7.b)
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
+                           ->  Seq Scan on prt2_p2 t1_7
                                  Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
+                           ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t2_2
+                                 Index Cond: (((a + b) / 2) = t1_7.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t1_8.b)
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Nested Loop
+                           ->  Seq Scan on prt2_p3 t1_8
+                                 Filter: (a = 0)
+                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_3
+                                 Index Cond: (((a + b) / 2) = t1_8.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = ((t2_3.a + t2_3.b) / 2))
+                     Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t1_9.b)
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Join
+                           Hash Cond: (((t2_4.a + t2_4.b) / 2) = t1_9.b)
+                           ->  Seq Scan on prt1_e_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t1_9
+                                       Filter: (a = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = ((t2_4.a + t2_4.b) / 2))
+                     Filter: (b = 0)
+(66 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                               QUERY PLAN                                
--------------------------------------------------------------------------
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
-                           ->  Seq Scan on prt2_p1 t1_3
+                           Hash Cond: (t1_5.b = ((t1_10.a + t1_10.b) / 2))
+                           ->  Seq Scan on prt2_p0 t1_5
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t1_6
+                                 ->  Seq Scan on prt1_e_p0 t1_10
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
-                     Index Cond: (a = t1_3.b)
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t1_5.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
-                           ->  Seq Scan on prt2_p2 t1_4
+                           Hash Cond: (t1_6.b = ((t1_11.a + t1_11.b) / 2))
+                           ->  Seq Scan on prt2_p1 t1_6
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t1_7
+                                 ->  Seq Scan on prt1_e_p1 t1_11
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
-                     Index Cond: (a = t1_4.b)
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
+                     Index Cond: (a = t1_6.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
-                           ->  Seq Scan on prt2_p3 t1_5
+                           Hash Cond: (t1_7.b = ((t1_12.a + t1_12.b) / 2))
+                           ->  Seq Scan on prt2_p2 t1_7
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p3 t1_8
+                                 ->  Seq Scan on prt1_e_p2 t1_12
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t1_5.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t1_7.b)
                      Filter: (b = 0)
-(39 rows)
-
-SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_8.b = ((t1_13.a + t1_13.b) / 2))
+                           ->  Seq Scan on prt2_p3 t1_8
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p3 t1_13
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t1_8.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_9.b = ((t1_14.a + t1_14.b) / 2))
+                           ->  Seq Scan on prt2_p4 t1_9
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t1_14
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = t1_9.b)
+                     Filter: (b = 0)
+(63 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 -- test merge joins
 SET enable_hashjoin TO off;
 SET enable_nestloop TO off;
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                           QUERY PLAN                           
-----------------------------------------------------------------
+                            QUERY PLAN                            
+------------------------------------------------------------------
  Merge Append
    Sort Key: t1.a
    ->  Merge Semi Join
-         Merge Cond: (t1.a = t1_3.b)
+         Merge Cond: (t1.a = t1_5.b)
          ->  Sort
                Sort Key: t1.a
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+               Merge Cond: (t1_5.b = (((t1_10.a + t1_10.b) / 2)))
                ->  Sort
-                     Sort Key: t1_3.b
-                     ->  Seq Scan on prt2_p1 t1_3
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on prt2_p0 t1_5
                ->  Sort
-                     Sort Key: (((t1_6.a + t1_6.b) / 2))
-                     ->  Seq Scan on prt1_e_p1 t1_6
+                     Sort Key: (((t1_10.a + t1_10.b) / 2))
+                     ->  Seq Scan on prt1_e_p0 t1_10
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_1.a = t1_4.b)
+         Merge Cond: (t1_1.a = t1_6.b)
          ->  Sort
                Sort Key: t1_1.a
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+               Merge Cond: (t1_6.b = (((t1_11.a + t1_11.b) / 2)))
                ->  Sort
-                     Sort Key: t1_4.b
-                     ->  Seq Scan on prt2_p2 t1_4
+                     Sort Key: t1_6.b
+                     ->  Seq Scan on prt2_p1 t1_6
                ->  Sort
-                     Sort Key: (((t1_7.a + t1_7.b) / 2))
-                     ->  Seq Scan on prt1_e_p2 t1_7
+                     Sort Key: (((t1_11.a + t1_11.b) / 2))
+                     ->  Seq Scan on prt1_e_p1 t1_11
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_2.a = t1_5.b)
+         Merge Cond: (t1_2.a = t1_7.b)
          ->  Sort
                Sort Key: t1_2.a
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+               Merge Cond: (t1_7.b = (((t1_12.a + t1_12.b) / 2)))
                ->  Sort
-                     Sort Key: t1_5.b
-                     ->  Seq Scan on prt2_p3 t1_5
+                     Sort Key: t1_7.b
+                     ->  Seq Scan on prt2_p2 t1_7
                ->  Sort
-                     Sort Key: (((t1_8.a + t1_8.b) / 2))
-                     ->  Seq Scan on prt1_e_p3 t1_8
+                     Sort Key: (((t1_12.a + t1_12.b) / 2))
+                     ->  Seq Scan on prt1_e_p2 t1_12
                            Filter: (c = 0)
-(47 rows)
+   ->  Merge Semi Join
+         Merge Cond: (t1_3.a = t1_8.b)
+         ->  Sort
+               Sort Key: t1_3.a
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_8.b = (((t1_13.a + t1_13.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_8.b
+                     ->  Seq Scan on prt2_p3 t1_8
+               ->  Sort
+                     Sort Key: (((t1_13.a + t1_13.b) / 2))
+                     ->  Seq Scan on prt1_e_p3 t1_13
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_4.a = t1_9.b)
+         ->  Sort
+               Sort Key: t1_4.a
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_9.b = (((t1_14.a + t1_14.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_9.b
+                     ->  Seq Scan on prt2_p4 t1_9
+               ->  Sort
+                     Sort Key: (((t1_14.a + t1_14.b) / 2))
+                     ->  Seq Scan on prt1_e_p4 t1_14
+                           Filter: (c = 0)
+(77 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
@@ -921,14 +1411,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
                            ->  Sort
                                  Sort Key: (((t3.a + t3.b) / 2))
-                                 ->  Seq Scan on prt1_e_p1 t3
+                                 ->  Seq Scan on prt1_e_p0 t3
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1.a
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                ->  Sort
                      Sort Key: t2.b
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Merge Left Join
                Merge Cond: (t1_1.a = t2_1.b)
                ->  Sort
@@ -937,14 +1427,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
                            ->  Sort
                                  Sort Key: (((t3_1.a + t3_1.b) / 2))
-                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                 ->  Seq Scan on prt1_e_p1 t3_1
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_1.a
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                ->  Sort
                      Sort Key: t2_1.b
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Merge Left Join
                Merge Cond: (t1_2.a = t2_2.b)
                ->  Sort
@@ -953,32 +1443,74 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
                            ->  Sort
                                  Sort Key: (((t3_2.a + t3_2.b) / 2))
-                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                 ->  Seq Scan on prt1_e_p2 t3_2
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_2.a
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                ->  Sort
                      Sort Key: t2_2.b
-                     ->  Seq Scan on prt2_p3 t2_2
-(51 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Merge Left Join
+               Merge Cond: (t1_3.a = t2_3.b)
+               ->  Sort
+                     Sort Key: t1_3.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_3.a + t3_3.b) / 2)) = t1_3.a)
+                           ->  Sort
+                                 Sort Key: (((t3_3.a + t3_3.b) / 2))
+                                 ->  Seq Scan on prt1_e_p3 t3_3
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_3.a
+                                 ->  Seq Scan on prt1_p3 t1_3
+               ->  Sort
+                     Sort Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Merge Left Join
+               Merge Cond: (t1_4.a = t2_4.b)
+               ->  Sort
+                     Sort Key: t1_4.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_4.a + t3_4.b) / 2)) = t1_4.a)
+                           ->  Sort
+                                 Sort Key: (((t3_4.a + t3_4.b) / 2))
+                                 ->  Seq Scan on prt1_e_p4 t3_4
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_4.a
+                                 ->  Seq Scan on prt1_p4 t1_4
+               ->  Sort
+                     Sort Key: t2_4.b
+                     ->  Seq Scan on prt2_p4 t2_4
+(83 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- MergeAppend on nullable column
 -- This should generate a partitionwise join, but currently fails to
@@ -987,12 +1519,14 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Merge Left Join
-         Merge Cond: (prt1_p1.a = prt2_p2.b)
+         Merge Cond: (prt1_p0.a = prt2_p2.b)
          ->  Sort
-               Sort Key: prt1_p1.a
+               Sort Key: prt1_p0.a
                ->  Append
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p1
                            Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p2
@@ -1004,21 +1538,28 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                            Filter: (b > 250)
                      ->  Seq Scan on prt2_p3
                            Filter: (b > 250)
-(18 rows)
+                     ->  Seq Scan on prt2_p4
+                           Filter: (b > 250)
+(22 rows)
 
 SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  b  
------+-----
-   0 |    
-  50 |    
- 100 |    
- 150 |    
- 200 |    
- 250 |    
- 300 | 300
- 350 |    
- 400 |    
-(9 rows)
+  a   |  b  
+------+-----
+ -250 |    
+ -200 |    
+ -150 |    
+ -100 |    
+  -50 |    
+    0 |    
+   50 |    
+  100 |    
+  150 |    
+  200 |    
+  250 |    
+  300 | 300
+  350 |    
+  400 |    
+(14 rows)
 
 -- merge join when expression with whole-row reference needs to be sorted;
 -- partitionwise join does not apply
@@ -1032,175 +1573,2412 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
          Sort Key: t1.a, ((((t1.*)::prt1))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
    ->  Sort
          Sort Key: t2.b, ((((t2.*)::prt2))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt2_p1 t2
-                     ->  Seq Scan on prt2_p2 t2_1
-                     ->  Seq Scan on prt2_p3 t2_2
-(16 rows)
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+(20 rows)
 
 SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.b ORDER BY t1.a;
- a  | b  
-----+----
-  0 |  0
-  6 |  6
- 12 | 12
- 18 | 18
- 24 | 24
-(5 rows)
+  a  |  b  
+-----+-----
+ -24 | -24
+ -18 | -18
+ -12 | -12
+  -6 |  -6
+   0 |   0
+   6 |   6
+  12 |  12
+  18 |  18
+  24 |  24
+(9 rows)
 
 RESET enable_hashjoin;
 RESET enable_nestloop;
---
--- partitioned by multiple columns
---
-CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
-CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
-ANALYZE prt1_m;
-CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
-CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
-ANALYZE prt2_m;
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-                                                             QUERY PLAN                                                             
-------------------------------------------------------------------------------------------------------------------------------------
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
  Sort
-   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p4 t2_3
+               ->  Seq Scan on prt2_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_4
+                           Filter: (b = 0)
+(22 rows)
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
    ->  Append
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
-               ->  Seq Scan on prt1_m_p1
-                     Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p1 t2
                ->  Hash
-                     ->  Seq Scan on prt2_m_p1
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
-               ->  Seq Scan on prt1_m_p2
-                     Filter: (c = 0)
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p2 t2_1
                ->  Hash
-                     ->  Seq Scan on prt2_m_p2
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
-               ->  Seq Scan on prt1_m_p3
-                     Filter: (c = 0)
+                     ->  Seq Scan on prt1_p2 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p3 t2_2
                ->  Hash
-                     ->  Seq Scan on prt2_m_p3
-                           Filter: (c = 0)
-(24 rows)
+                     ->  Seq Scan on prt1_p3 t1_2
+                           Filter: (b = 0)
+(21 rows)
 
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
-  50 | 0 |     |  
- 100 | 0 |     |  
- 150 | 0 | 150 | 0
- 200 | 0 |     |  
- 250 | 0 |     |  
- 300 | 0 | 300 | 0
- 350 | 0 |     |  
- 400 | 0 |     |  
- 450 | 0 | 450 | 0
- 500 | 0 |     |  
- 550 | 0 |     |  
-     |   |  75 | 0
-     |   | 225 | 0
-     |   | 375 | 0
-     |   | 525 | 0
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -150 | -0150 | 0 | -0150
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+(7 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 |   | 
+ -200 | -0200 |   | 
+ -150 | -0150 | 0 | -0150
+ -100 | -0100 |   | 
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+(21 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -200 | 0 | -0200
+ -100 | 0 | -0100
+  -50 | 0 | -0050
+   50 | 0 | 0050
+  100 | 0 | 0100
+  200 | 0 | 0200
+  250 | 0 | 0250
+  350 | 0 | 0350
+  400 | 0 | 0400
+  500 | 0 | 0500
+  550 | 0 | 0550
+  650 | 0 | 0650
+  700 | 0 | 0700
+(14 rows)
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+                             QUERY PLAN                              
+---------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t3.c
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p0_a on prt1_p0 t3
+                           Index Cond: (a = t2.b)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t2.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_1.a = t2_1.b)
+                           ->  Seq Scan on prt1_p1 t3_1
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1 t2_1
+                                       Filter: (a = 0)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p2_a on prt1_p2 t3_2
+                           Index Cond: (a = t2_2.b)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t3_3
+                           Index Cond: (a = t2_3.b)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_4.a = t2_4.b)
+                           ->  Seq Scan on prt1_p4 t3_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t2_4
+                                       Filter: (a = 0)
+(47 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+  a   | a |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.b
+   ->  Hash Right Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p5 t2_5
+                           Filter: (a = 0)
+(24 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: (t1.a = t2.b)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.a, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
+                     Filter: (a = 0)
+(22 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
+               ->  Seq Scan on prt1_m_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p1
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
+               ->  Seq Scan on prt1_m_p2
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p2
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
+               ->  Seq Scan on prt1_m_p3
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p3
+                           Filter: (c = 0)
+(24 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+  a  | c |  b  | c 
+-----+---+-----+---
+   0 | 0 |   0 | 0
+  50 | 0 |     |  
+ 100 | 0 |     |  
+ 150 | 0 | 150 | 0
+ 200 | 0 |     |  
+ 250 | 0 |     |  
+ 300 | 0 | 300 | 0
+ 350 | 0 |     |  
+ 400 | 0 |     |  
+ 450 | 0 | 450 | 0
+ 500 | 0 |     |  
+ 550 | 0 |     |  
+     |   |  75 | 0
+     |   | 225 | 0
+     |   | 375 | 0
+     |   | 525 | 0
+(16 rows)
+
+--
+-- tests for list partitioned tables.
+--
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
+--
+-- list partitioned by expression
+--
+CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
+ANALYZE plt1_e;
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+                                           QUERY PLAN                                           
+------------------------------------------------------------------------------------------------
+ GroupAggregate
+   Group Key: t1.c, t2.c, t3.c
+   ->  Sort
+         Sort Key: t1.c, t3.c
+         ->  Append
+               ->  Hash Join
+                     Hash Cond: ((t1.c)::text = ltrim(t3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2.b = t1.b) AND ((t2.c)::text = (t1.c)::text))
+                           ->  Seq Scan on plt2_p4 t2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p4 t1
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p4 t3
+               ->  Hash Join
+                     Hash Cond: ((t1_1.c)::text = ltrim(t3_1.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t1_1.b = t2_1.b) AND ((t1_1.c)::text = (t2_1.c)::text))
+                           ->  Seq Scan on plt1_p1 t1_1
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p1 t2_1
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p1 t3_1
+               ->  Hash Join
+                     Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t1_2.b = t2_2.b) AND ((t1_2.c)::text = (t2_2.c)::text))
+                           ->  Seq Scan on plt1_p2 t1_2
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p2 t2_2
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p2 t3_2
+               ->  Hash Join
+                     Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2_3.b = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text))
+                           ->  Seq Scan on plt2_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p3 t3_3
+(41 rows)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p5 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_3
+                                 ->  Seq Scan on plt2_p3 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
 (16 rows)
 
---
--- tests for list partitioned tables.
---
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
---
--- list partitioned by expression
---
-CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1_e;
--- test partition matching with N-way join
+-- full join
 EXPLAIN (COSTS OFF)
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                   QUERY PLAN                                   
---------------------------------------------------------------------------------
- GroupAggregate
-   Group Key: t1.c, t2.c, t3.c
-   ->  Sort
-         Sort Key: t1.c, t3.c
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.c = ltrim(t3.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1.b = t2.b) AND (t1.c = t2.c))
-                           ->  Seq Scan on plt1_p1 t1
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.c = ltrim(t3_1.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
-                           ->  Seq Scan on plt1_p2 t1_1
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.c = ltrim(t3_2.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
-                           ->  Seq Scan on plt1_p3 t1_2
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p3 t3_2
-(32 rows)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
 
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  75.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 123.0000000000000000 | 123.0000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 225.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 273.0000000000000000 | 273.0000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 375.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 423.0000000000000000 | 423.0000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 525.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 573.0000000000000000 | 573.0000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Seq Scan on plt2_p2 t2_2
+                                 ->  Seq Scan on plt2_p3 t2_3
+(22 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1225,22 +4003,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
 --------------------------------------------------
  Hash Left Join
    Hash Cond: (t2.b = a)
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t3.a = t2.b)
-               ->  Seq Scan on prt1_p1 t3
-               ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
-         ->  Hash Join
-               Hash Cond: (t3_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t3_1
-               ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
-         ->  Hash Join
-               Hash Cond: (t3_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t3_2
-               ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
+   ->  Hash Join
+         Hash Cond: (t3.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t3
+               ->  Seq Scan on prt1_p1 t3_1
+               ->  Seq Scan on prt1_p2 t3_2
+               ->  Seq Scan on prt1_p3 t3_3
+               ->  Seq Scan on prt1_p4 t3_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
    ->  Hash
          ->  Result
                One-Time Filter: false
@@ -1255,16 +4033,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 --
 -- tests for hash partitioned tables.
@@ -1340,41 +4124,9 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  273.0000000000000000 | 273.0000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
 (6 rows)
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
- Sort
-   Sort Key: t1.a
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
-               ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
-               ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
-                           Filter: (b = 0)
-(21 rows)
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
@@ -1389,26 +4141,24 @@ SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c
    Sort Key: t1.c
    ->  HashAggregate
          Group Key: t1.c, t2.c
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t2.c = t1.c)
-                     ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p1 t1
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on plt1_p4 t1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_1.c = t1_1.c)
-                     ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_p2 t1_1
+                           ->  Seq Scan on plt1_p1 t1_1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_2.c = t1_2.c)
-                     ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Seq Scan on plt1_p2 t1_2
                                  Filter: ((a % 25) = 0)
-(23 rows)
+                           ->  Seq Scan on plt1_p3 t1_3
+                                 Filter: ((a % 25) = 0)
+(21 rows)
 
 --
 -- multiple levels of partitioning
@@ -1804,64 +4554,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
  Hash Join
    Hash Cond: (t1.a = t2.a)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt4_n_p1 t2
                ->  Seq Scan on prt4_n_p2 t2_1
                ->  Seq Scan on prt4_n_p3 t2_2
-(11 rows)
+(13 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a and t1.a = t3.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                        QUERY PLAN                        
+----------------------------------------------------------
  Hash Join
-   Hash Cond: (t2.a = t1.a)
+   Hash Cond: (t3.b = t1.a)
    ->  Append
-         ->  Seq Scan on prt4_n_p1 t2
-         ->  Seq Scan on prt4_n_p2 t2_1
-         ->  Seq Scan on prt4_n_p3 t2_2
+         ->  Seq Scan on prt2_p0 t3
+         ->  Seq Scan on prt2_p1 t3_1
+         ->  Seq Scan on prt2_p2 t3_2
+         ->  Seq Scan on prt2_p3 t3_3
+         ->  Seq Scan on prt2_p4 t3_4
+         ->  Seq Scan on prt2_p5 t3_5
    ->  Hash
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.a = t3_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_2.b)
-                     ->  Seq Scan on prt1_p3 t1_2
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3 t3_2
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt4_n_p1 t2
+                           ->  Seq Scan on prt4_n_p2 t2_1
+                           ->  Seq Scan on prt4_n_p3 t2_2
 (23 rows)
 
 -- partitionwise join can not be applied if there are no equi-join conditions
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (b > t1.a)
-(12 rows)
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+(16 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partitionwise join
@@ -1947,16 +4703,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
-                     ->  Seq Scan on plt1_p1 t3
-                     ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3
+                     ->  Seq Scan on plt1_p1 t3_1
+                     ->  Seq Scan on plt1_p2 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partitionwise join can not be applied for a join between list and range
 -- partitioned tables
@@ -1967,14 +4724,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
  Hash Full Join
    Hash Cond: ((t2.c)::text = (t1.c)::text)
    ->  Append
-         ->  Seq Scan on prt1_p1 t2
-         ->  Seq Scan on prt1_p2 t2_1
-         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p0 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p2 t2_2
+         ->  Seq Scan on prt1_p3 t2_3
+         ->  Seq Scan on prt1_p4 t2_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt1_n_p1 t1
                ->  Seq Scan on prt1_n_p2 t1_1
-(10 rows)
+(12 rows)
 
 -- partitionwise join can not be applied if only one of joining tables has
 -- default partition
@@ -1990,16 +4749,202 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
    ->  Hash Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
+ANALYZE plt1;
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+ANALYZE plt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.c = t2.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt1_p1 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t2_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_1
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+  0 | 0000 |    | 
+  5 | 0000 |    | 
+ 10 | 0000 |    | 
+ 15 | 0000 |    | 
+ 20 | 0000 |    | 
+  2 | 0002 |  2 | 0002
+  2 | 0002 |  7 | 0002
+  2 | 0002 | 12 | 0002
+  2 | 0002 | 17 | 0002
+  2 | 0002 | 22 | 0002
+  7 | 0002 |  2 | 0002
+  7 | 0002 |  7 | 0002
+  7 | 0002 | 12 | 0002
+  7 | 0002 | 17 | 0002
+  7 | 0002 | 22 | 0002
+ 12 | 0002 |  2 | 0002
+ 12 | 0002 |  7 | 0002
+ 12 | 0002 | 12 | 0002
+ 12 | 0002 | 17 | 0002
+ 12 | 0002 | 22 | 0002
+ 17 | 0002 |  2 | 0002
+ 17 | 0002 |  7 | 0002
+ 17 | 0002 | 12 | 0002
+ 17 | 0002 | 17 | 0002
+ 17 | 0002 | 22 | 0002
+ 22 | 0002 |  2 | 0002
+ 22 | 0002 |  7 | 0002
+ 22 | 0002 | 12 | 0002
+ 22 | 0002 | 17 | 0002
+ 22 | 0002 | 22 | 0002
+    |      |  1 | 0001
+    |      |  6 | 0001
+    |      | 11 | 0001
+    |      | 16 | 0001
+    |      | 21 | 0001
+(35 rows)
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1;
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2;
+CREATE TABLE plt3 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt3_p1 PARTITION OF plt3 FOR VALUES IN ('0001');
+CREATE TABLE plt3_p2 PARTITION OF plt3 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2.a, t3.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.c = t3.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1.c = t2.c)
+                     ->  Seq Scan on plt1_p1 t1
+                     ->  Hash
+                           ->  Seq Scan on plt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on plt3_p1 t3
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t3_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_1.c = t2_1.c)
+                     ->  Seq Scan on plt1_p2 t1_1
+                     ->  Hash
+                           ->  Seq Scan on plt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt3_p2 t3_1
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+ a  |  c   | a  |  c   | a  |  c   
+----+------+----+------+----+------
+  0 | 0000 |    |      |    | 
+  5 | 0000 |    |      |    | 
+ 10 | 0000 |    |      |    | 
+ 15 | 0000 |    |      |    | 
+ 20 | 0000 |    |      |    | 
+  1 | 0001 |    |      |  1 | 0001
+  1 | 0001 |    |      |  6 | 0001
+  1 | 0001 |    |      | 11 | 0001
+  1 | 0001 |    |      | 16 | 0001
+  1 | 0001 |    |      | 21 | 0001
+  6 | 0001 |    |      |  1 | 0001
+  6 | 0001 |    |      |  6 | 0001
+  6 | 0001 |    |      | 11 | 0001
+  6 | 0001 |    |      | 16 | 0001
+  6 | 0001 |    |      | 21 | 0001
+ 11 | 0001 |    |      |  1 | 0001
+ 11 | 0001 |    |      |  6 | 0001
+ 11 | 0001 |    |      | 11 | 0001
+ 11 | 0001 |    |      | 16 | 0001
+ 11 | 0001 |    |      | 21 | 0001
+ 16 | 0001 |    |      |  1 | 0001
+ 16 | 0001 |    |      |  6 | 0001
+ 16 | 0001 |    |      | 11 | 0001
+ 16 | 0001 |    |      | 16 | 0001
+ 16 | 0001 |    |      | 21 | 0001
+ 21 | 0001 |    |      |  1 | 0001
+ 21 | 0001 |    |      |  6 | 0001
+ 21 | 0001 |    |      | 11 | 0001
+ 21 | 0001 |    |      | 16 | 0001
+ 21 | 0001 |    |      | 21 | 0001
+  2 | 0002 |  2 | 0002 |    | 
+  2 | 0002 |  7 | 0002 |    | 
+  2 | 0002 | 12 | 0002 |    | 
+  2 | 0002 | 17 | 0002 |    | 
+  2 | 0002 | 22 | 0002 |    | 
+  7 | 0002 |  2 | 0002 |    | 
+  7 | 0002 |  7 | 0002 |    | 
+  7 | 0002 | 12 | 0002 |    | 
+  7 | 0002 | 17 | 0002 |    | 
+  7 | 0002 | 22 | 0002 |    | 
+ 12 | 0002 |  2 | 0002 |    | 
+ 12 | 0002 |  7 | 0002 |    | 
+ 12 | 0002 | 12 | 0002 |    | 
+ 12 | 0002 | 17 | 0002 |    | 
+ 12 | 0002 | 22 | 0002 |    | 
+ 17 | 0002 |  2 | 0002 |    | 
+ 17 | 0002 |  7 | 0002 |    | 
+ 17 | 0002 | 12 | 0002 |    | 
+ 17 | 0002 | 17 | 0002 |    | 
+ 17 | 0002 | 22 | 0002 |    | 
+ 22 | 0002 |  2 | 0002 |    | 
+ 22 | 0002 |  7 | 0002 |    | 
+ 22 | 0002 | 12 | 0002 |    | 
+ 22 | 0002 | 17 | 0002 |    | 
+ 22 | 0002 | 22 | 0002 |    | 
+(55 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index fb3ba18a26..b0ab002eee 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -10,25 +10,39 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -69,11 +83,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -95,20 +117,30 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 
 EXPLAIN (COSTS OFF)
@@ -172,6 +204,128 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -196,28 +350,79 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
 -- test partition matching with N-way join
@@ -225,6 +430,175 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
@@ -270,27 +644,18 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
 ALTER TABLE plt2 DETACH PARTITION plt2_p3;
 ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
 ANALYZE plt2;
-
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.a % 25 = 0 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
 --
 -- multiple levels of partitioning
 --
@@ -435,3 +800,47 @@ ANALYZE prt2;
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
+ANALYZE plt1;
+
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+ANALYZE plt2;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1;
+
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2;
+
+CREATE TABLE plt3 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt3_p1 PARTITION OF plt3 FOR VALUES IN ('0001');
+CREATE TABLE plt3_p2 PARTITION OF plt3 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
#102amul sul
sulamul@gmail.com
In reply to: Etsuro Fujita (#101)
4 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Fri, Nov 1, 2019 at 3:58 PM Etsuro Fujita <etsuro.fujita@gmail.com>
wrote:

On Thu, Oct 31, 2019 at 6:49 PM Etsuro Fujita <etsuro.fujita@gmail.com>
wrote:

Attached is an updated version. If no objections, I'll merge this
with the main patch [1].

Done. Attached is a new version of the patch.

Other changes: in generate_matching_part_pairs(), I changed variable
names to match other functions, simplified assertions, and
adjusted/added comments a bit.

Thanks for the update version.

A query and comments for v25:

583 + * The function returns NULL if we can not find the matching pair of
584 + * partitions. This happens if 1. multiple partitions on one side
match with
585 + * one partition on the other side. 2. a given partition on the outer
side
586 + * doesn't have a matching partition on the inner side. We can not
support the
587 + * first case since we don't have a way to represent multiple
partitions as a
588 + * single relation (RelOptInfo) and then perform join using the ganged
589 + * relation. We can not support the second case since the missing
inner
590 + * partition needs to be represented as an empty relation and we
don't have a
591 + * way to introduce empty relation during join planning after
creating paths
592 + * for all the base relations.
593 + */
594 +PartitionBoundInfo
595 +partition_bounds_merge(int partnatts,

I think the second condition mention for partition_bounds_merge() looks
outdated, do you think we should remove that or am I missing something here?
---

1768 +
1769 + /*
1770 + * If this is an outer join, the merged partition would act as
the
1771 + * default partition of the join; record the index in
*default_index
1772 + * if not done yet.
1773 + */
1774 + if (jointype == JOIN_LEFT || jointype == JOIN_ANTI ||
1775 + jointype == JOIN_FULL)
1776 + {

As decided need to replace this list by IS_OUTER_JOIN(jointype).
---

2020 + if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
2021 + jointype == JOIN_ANTI)
2022 + {

Same as the previous.
---

I tried a coverage testing and tried to adjust & add a few tests to
improved the
code coverage for the v25 patch. Please have a look at the attached 0002 &
also
attach the coverage output with & without this patch, TWIMW.

0001 is the same v25 patch, reattaching to make CFbot happy.

Regards,
Amul

Attachments:

0001-Improve-partition-matching-for-partitionwise-joins-v.patchapplication/octet-stream; name=0001-Improve-partition-matching-for-partitionwise-joins-v.patchDownload
From de83296ff6f5da1a381ae74fd0bcf014165bf6ec Mon Sep 17 00:00:00 2001
From: Amul Sul <amul.sul@enterprisedb.com>
Date: Sun, 3 Nov 2019 23:39:27 -0500
Subject: [PATCH 1/2] Improve-partition-matching-for-partitionwise-joins-v25

---
 src/backend/nodes/outfuncs.c                 |    2 +
 src/backend/optimizer/path/joinrels.c        |  246 +-
 src/backend/optimizer/util/inherit.c         |    2 +
 src/backend/optimizer/util/relnode.c         |   45 +-
 src/backend/partitioning/partbounds.c        | 1691 +++++++
 src/include/nodes/pathnodes.h                |    5 +
 src/include/partitioning/partbounds.h        |    7 +
 src/test/regress/expected/partition_join.out | 4123 +++++++++++++++---
 src/test/regress/sql/partition_join.sql      |  471 +-
 9 files changed, 5920 insertions(+), 672 deletions(-)

diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index b0dcd02ff68..b8151574024 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -2281,6 +2281,8 @@ _outRelOptInfo(StringInfo str, const RelOptInfo *node)
 	WRITE_BOOL_FIELD(has_eclass_joins);
 	WRITE_BOOL_FIELD(consider_partitionwise_join);
 	WRITE_BITMAPSET_FIELD(top_parent_relids);
+	WRITE_BOOL_FIELD(merged);
+	WRITE_BITMAPSET_FIELD(all_partrels);
 	WRITE_NODE_FIELD(partitioned_child_rels);
 }
 
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 6a480ab7644..ed7bc23c7be 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -48,6 +48,9 @@ static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 												Relids left_relids, Relids right_relids);
 static int	match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel,
 										 bool strict_op);
+static void get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2);
 
 
 /*
@@ -1357,25 +1360,30 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	bool		merged = false;
+	List	   *parts1 = NIL;
+	List	   *parts2 = NIL;
+	ListCell   *lcr1 = NULL;
+	ListCell   *lcr2 = NULL;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1390,34 +1398,107 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		   joinrel->part_scheme == rel2->part_scheme);
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * If we don't have the partition bounds for the join rel yet, try to
+	 * create it along with pairs of partitions to be joined; else generate
+	 * those using the partitioning info for the join rel we already have.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	if (joinrel->nparts == -1)
+	{
+		PartitionScheme part_scheme = joinrel->part_scheme;
+		PartitionBoundInfo boundinfo = NULL;
+		int			nparts = 0;
+
+		Assert(joinrel->boundinfo == NULL);
+		Assert(joinrel->part_rels == NULL);
+
+		/*
+		 * See if the partition bounds for inputs are exactly the same, in
+		 * which case we don't need to work hard: partitions with the same
+		 * partition indexes will form join pairs, and the join rel will have
+		 * the same partition bounds as inputs; otherwise try to merge the
+		 * partition bounds along with generating join pairs.
+		 *
+		 * Even if one or both inputs have merged partition bounds, it'd be
+		 * possible for the partition bounds to be exactly the same, but it
+		 * seems unlikely to be worth the cycles to check; do this check only
+		 * if both inputs have non-merged partition bounds.
+		 */
+		if (!rel1->merged &&
+			!rel2->merged &&
+			rel1->nparts == rel2->nparts &&
+			partition_bounds_equal(part_scheme->partnatts,
+								   part_scheme->parttyplen,
+								   part_scheme->parttypbyval,
+								   rel1->boundinfo, rel2->boundinfo))
+		{
+			boundinfo = rel1->boundinfo;
+			nparts = rel1->nparts;
+		}
+		else
+		{
+			boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											   part_scheme->parttyplen,
+											   part_scheme->parttypbyval,
+											   part_scheme->partsupfunc,
+											   part_scheme->partcollation,
+											   rel1, rel2,
+											   parent_sjinfo->jointype,
+											   &parts1, &parts2);
+			if (boundinfo == NULL)
+			{
+				joinrel->nparts = 0;
+				return;
+			}
+			nparts = list_length(parts1);
+			merged = true;
+		}
+
+		Assert(nparts > 0);
+		joinrel->boundinfo = boundinfo;
+		joinrel->merged = merged;
+		joinrel->nparts = nparts;
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+	}
+	else
+	{
+		Assert(joinrel->nparts > 0);
+		Assert(joinrel->boundinfo);
+		Assert(joinrel->part_rels);
+
+		/*
+		 * If the partition bounds for the join rel are not merged ones,
+		 * inputs are guaranteed to have the same partition bounds, so
+		 * partitions with the same partition indexes will form join pairs;
+		 * else let get_matching_part_pairs() do the work.
+		 */
+		if (joinrel->merged)
+		{
+			get_matching_part_pairs(root, joinrel, rel1, rel2,
+									&parts1, &parts2);
+			Assert(list_length(parts1) == joinrel->nparts);
+			Assert(list_length(parts2) == joinrel->nparts);
+			merged = true;
+		}
+	}
 
-	nparts = joinrel->nparts;
+	if (merged)
+	{
+		lcr1 = list_head(parts1);
+		lcr2 = list_head(parts2);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
-		bool		rel1_empty = (child_rel1 == NULL ||
-								  IS_DUMMY_REL(child_rel1));
-		bool		rel2_empty = (child_rel2 == NULL ||
-								  IS_DUMMY_REL(child_rel2));
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		bool		rel1_empty;
+		bool		rel2_empty;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1425,6 +1506,22 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		if (merged)
+		{
+			child_rel1 = lfirst_node(RelOptInfo, lcr1);
+			child_rel2 = lfirst_node(RelOptInfo, lcr2);
+			lcr1 = lnext(parts1, lcr1);
+			lcr2 = lnext(parts2, lcr2);
+		}
+		else
+		{
+			child_rel1 = rel1->part_rels[cnt_parts];
+			child_rel2 = rel2->part_rels[cnt_parts];
+		}
+
+		rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1));
+		rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2));
+
 		/*
 		 * Check for cases where we can prove that this segment of the join
 		 * returns no rows, due to one or both inputs being empty (including
@@ -1522,6 +1619,8 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 												 child_sjinfo,
 												 child_sjinfo->jointype);
 			joinrel->part_rels[cnt_parts] = child_joinrel;
+			joinrel->all_partrels = bms_add_members(joinrel->all_partrels,
+													child_joinrel->relids);
 		}
 
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
@@ -1529,6 +1628,7 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
+
 	}
 }
 
@@ -1738,3 +1838,99 @@ match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op)
 
 	return -1;
 }
+
+/*
+ * get_matching_part_pairs
+ *		Generate join pairs of partitions for the two inputs
+ */
+static void
+get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2)
+{
+	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
+	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
+	int 		cnt_parts;
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
+	{
+		RelOptInfo *child_joinrel = joinrel->part_rels[cnt_parts];
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		Relids		child_relids1;
+		Relids		child_relids2;
+
+		/*
+		 * If this segment of the join is empty, it means that this segment
+		 * was ignored when previously creating child-join paths for it in
+		 * try_partitionwise_join() as it would not contribute to the join
+		 * result, due to one or both inputs being empty; add NULL to each of
+		 * the given lists so that this segment will be ignored again in that
+		 * function.
+		 */
+		if (!child_joinrel)
+		{
+			*parts1 = lappend(*parts1, NULL);
+			*parts2 = lappend(*parts2, NULL);
+			continue;
+		}
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel1 side.
+		 */
+		child_relids1 = bms_intersect(child_joinrel->relids,
+									  rel1->all_partrels);
+		Assert(bms_num_members(child_relids1) == bms_num_members(rel1->relids));
+
+		/*
+		 * Get a child rel for rel1 with the relids.  Note that we should have
+		 * the child rel even if rel1 is a join rel, because in that case the
+		 * partitions specified in the relids would have matching/overlapping
+		 * boundaries, so those partitions should be considered as ones to be
+		 * joined even when planning partitionwise joins of rel1, meaning that
+		 * the child rel would have been built by the time we get here.
+		 */
+		if (rel1_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids1);
+
+			child_rel1 = find_base_rel(root, varno);
+		}
+		else
+			child_rel1 = find_join_rel(root, child_relids1);
+		Assert(child_rel1);
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel2 side.
+		 */
+		child_relids2 = bms_intersect(child_joinrel->relids,
+									  rel2->all_partrels);
+		Assert(bms_num_members(child_relids2) == bms_num_members(rel2->relids));
+
+		/*
+		 * Get a child rel for rel2 with the relids.  See above comments.
+		 */
+		if (rel2_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids2);
+
+			child_rel2 = find_base_rel(root, varno);
+		}
+		else
+			child_rel2 = find_join_rel(root, child_relids2);
+		Assert(child_rel2);
+
+		/*
+		 * The join of rel1 and rel2 is legal, so is the join of the child
+		 * rels obtained above; add them to the given lists as a join pair
+		 * producing this join segment.
+		 */
+		*parts1 = lappend(*parts1, child_rel1);
+		*parts2 = lappend(*parts2, child_rel2);
+	}
+}
diff --git a/src/backend/optimizer/util/inherit.c b/src/backend/optimizer/util/inherit.c
index 38bc61e6878..caf6039c106 100644
--- a/src/backend/optimizer/util/inherit.c
+++ b/src/backend/optimizer/util/inherit.c
@@ -376,6 +376,8 @@ expand_partitioned_rtentry(PlannerInfo *root, RelOptInfo *relinfo,
 		/* Create the otherrel RelOptInfo too. */
 		childrelinfo = build_simple_rel(root, childRTindex, relinfo);
 		relinfo->part_rels[i] = childrelinfo;
+		relinfo->all_partrels = bms_add_members(relinfo->all_partrels,
+												childrelinfo->relids);
 
 		/* If this child is itself partitioned, recurse */
 		if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 85415381fb1..248a9e90939 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -240,10 +240,12 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->has_eclass_joins = false;
 	rel->consider_partitionwise_join = false;	/* might get changed later */
 	rel->part_scheme = NULL;
-	rel->nparts = 0;
+	rel->nparts = -1;
 	rel->boundinfo = NULL;
+	rel->merged = false;
 	rel->partition_qual = NIL;
 	rel->part_rels = NULL;
+	rel->all_partrels = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->partitioned_child_rels = NIL;
@@ -653,10 +655,12 @@ build_join_rel(PlannerInfo *root,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -829,10 +833,12 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -1626,7 +1632,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1639,24 +1645,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1668,17 +1656,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 4bc88104ee9..ff1965f5ab6 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -25,6 +25,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/pathnodes.h"
 #include "parser/parse_coerce.h"
 #include "partitioning/partbounds.h"
 #include "partitioning/partdesc.h"
@@ -69,6 +70,17 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+typedef struct PartitionMap
+{
+	int			nparts;			/* number of partitions */
+	int		   *merged_indexes;	/* indexes of merged partitions */
+	bool	   *merged;			/* flags to indicate whether partitions are
+								 * merged with non-dummy partitions */
+	bool		did_remapping;	/* did we remap partitions? */
+	int		   *old_indexes;	/* old indexes of merged partitions if
+								 * did_remapping */
+} PartitionMap;
+
 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);
@@ -108,6 +120,85 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 									 Expr **keyCol,
 									 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static PartitionBoundInfo partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts);
+static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							JoinType jointype,
+							List **outer_parts, List **inner_parts);
+static void init_partition_map(RelOptInfo *rel, PartitionMap *map);
+static void free_partition_map(PartitionMap *map);
+static int map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_part, int inner_part, int *next_index);
+static int merge_partition_with_dummy(PartitionMap *map, int index,
+						   int *next_index);
+static bool process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index);
+static bool process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index);
+static void fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes);
+static void generate_matching_part_pairs(RelOptInfo *outer_rel,
+							 RelOptInfo *inner_rel,
+							 PartitionMap *outer_map,
+							 PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_part_list,
+							 List **inner_part_list);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *collations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2);
+static bool partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
+						   Oid *collations, PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, int *ub_cmpval,
+						   int *lb_cmpval);
+static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes);
+static bool merge_default_partitions(PartitionMap *outer_map,
+						 PartitionMap *inner_map,
+						 bool outer_has_default,
+						 int outer_default,
+						 bool inner_has_default,
+						 int inner_default,
+						 JoinType jointype,
+						 int *next_index,
+						 int *default_index);
+static bool merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_map, PartitionMap *inner_map,
+					  bool outer_has_default, bool inner_has_default,
+					  bool outer_has_null, bool inner_has_null,
+					  JoinType jointype, int *next_index,
+					  int *default_index, int *null_index);
 
 /*
  * get_qual_from_partbound
@@ -2995,3 +3086,1603 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * The function produces the partition bounds for a join between two relations
+ * whose partition bounds are given. The function also returns two lists of
+ * partition indexes one for each of the joining relations. Both the lists
+ * contain the same number of elements. The partition indexes at the same
+ * positions in the lists indicate the pair partitions, one from each side, to
+ * be joined and the position itself corresponds to the index of partition
+ * produced by that child-join in the partitioned join.
+ *
+ * The function returns NULL if we can not find the matching pair of
+ * partitions. This happens if 1. multiple partitions on one side match with
+ * one partition on the other side. 2. a given partition on the outer side
+ * doesn't have a matching partition on the inner side. We can not support the
+ * first case since we don't have a way to represent multiple partitions as a
+ * single relation (RelOptInfo) and then perform join using the ganged
+ * relation. We can not support the second case since the missing inner
+ * partition needs to be represented as an empty relation and we don't have a
+ * way to introduce empty relation during join planning after creating paths
+ * for all the base relations.
+ */
+PartitionBoundInfo
+partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
+{
+	PartitionBoundInfo 	merged_bounds;
+	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
+						inner_binfo = inner_rel->boundinfo;
+	char				strategy = outer_binfo->strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = NULL;
+
+			break;
+
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partsupfunc,
+														partcollation,
+														outer_rel,
+														inner_rel,
+														jointype,
+														outer_parts,
+														inner_parts);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(partnatts,
+														 partsupfunc,
+														 partcollation,
+														 outer_rel,
+														 inner_rel,
+														 jointype,
+														 outer_parts,
+														 inner_parts);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	return merged_bounds;
+}
+
+/*
+ * partition_get_range_bounds
+ *
+ * Given the index of lower bound in datums array, return lower and upper
+ * bounds and the index of the partition with that lower bound.
+ */
+static int
+partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
+						   PartitionRangeBound *lower,
+						   PartitionRangeBound *upper)
+{
+	int			part_index;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The lower bound should correspond to a valid partition. */
+	part_index = bi->indexes[lb_index + 1];
+	Assert(part_index >= 0);
+
+	lower->kind = bi->kind[lb_index];
+	lower->datums = bi->datums[lb_index];
+	lower->lower = true;
+	upper->kind = bi->kind[lb_index + 1];
+	upper->datums = bi->datums[lb_index + 1];
+	upper->lower = false;
+
+	return part_index;
+}
+
+/*
+ * partition_range_get_next_lb_index
+ *
+ * Given the index of lower bound in datums array return the
+ * index of lower bound of the next partition. When the given index corresponds
+ * to the last partition, return number of datums (ndatums).
+ */
+static int
+partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
+{
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The partition index corresponding to the upper bound should be valid. */
+	Assert(bi->indexes[lb_index + 1] >= 0);
+
+	/*
+	 * If there are no bounds left beyond the upper bound, we have reached the
+	 * last partition.
+	 */
+	if (lb_index + 2 < bi->ndatums)
+	{
+		/*
+		 * If the bound next to the upper bound corresponds to no partition,
+		 * that's the next lower bound of the next partition. Otherwise, the
+		 * current upper bound is the lower bound of the next partition.
+		 */
+		if (bi->indexes[lb_index + 2] < 0)
+			return lb_index + 2;
+		else
+			return lb_index + 1;
+	}
+	else
+		return bi->ndatums;
+}
+
+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *partcollations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2)
+{
+	return partition_rbound_cmp(partnatts, partsupfunc, partcollations,
+								bound1->datums, bound1->kind, bound1->lower,
+								bound2);
+}
+
+/*
+ * partition_range_cmp
+ *
+ * Compare the bounds of two range partitions. Set ub_cmpval <, = or > 0, if the
+ * first partition's upper bound is lower than, equal to or higher than the
+ * second partition's upper bound resp. Similarly set lb_cmpval <, =  or > 0,
+ * if the first partition's lower bound is lower than, equal to or higher than
+ * the second partition's lower bound resp.
+ *
+ * Return true, if the ranges overlap, otherwise return false.
+ */
+static bool
+partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
+					PartitionRangeBound *lower_bound1,
+					PartitionRangeBound *upper_bound1,
+					PartitionRangeBound *lower_bound2,
+					PartitionRangeBound *upper_bound2, int *ub_cmpval,
+					int *lb_cmpval)
+{
+	bool		overlap;
+
+	/*
+	 * Compare upper bound of the first partition with the lower bound of the
+	 * second and vice-versa. If lower bound is higher than the upper bound,
+	 * the partitions are not overlapping. All other cases indicate overlapping
+	 * partitions.
+	 */
+	if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+								  lower_bound1, upper_bound2) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = 1;
+		*lb_cmpval = 1;
+	}
+	else if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+									   lower_bound2, upper_bound1) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = -1;
+		*lb_cmpval = -1;
+	}
+	else
+	{
+		overlap = true;
+		*ub_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, upper_bound1,
+											   upper_bound2);
+		*lb_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, lower_bound1,
+											   lower_bound2);
+	}
+
+	return overlap;
+}
+
+/*
+ * partition_range_merge
+ *
+ * Merge the partition bounds of given two partitions such that the join
+ * between the given two partitions fits merged bounds.
+ *
+ * "merged_upper" will be set to one of the given upper bounds and
+ * "merged_lower" will be set to one of the given lower bounds.
+ */
+static void
+partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
+					  Oid *partcollations, JoinType jointype,
+					  PartitionRangeBound *left_lb,
+					  PartitionRangeBound *left_ub,
+					  PartitionRangeBound *right_lb,
+					  PartitionRangeBound *right_ub,
+					  PartitionRangeBound **merged_lb,
+					  PartitionRangeBound **merged_ub)
+{
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = left_ub;
+			*merged_lb = left_lb;
+			break;
+
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) < 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) > 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		case JOIN_FULL:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) > 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) < 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+}
+
+/*
+ * Add the lower bound of the next range to the list of bounds, if the lower
+ * bound is higher or equal to the previous upper bound. If successful return
+ * true, otherwise false.
+ */
+static bool
+partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
+							  Oid *partcollations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		prev_ub.datums = llast(*merged_datums);
+		prev_ub.kind = llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  next_lb_datums, next_lb_kind, false,
+									  &prev_ub);
+	}
+
+	/*
+	 * The lower bound is lower than the last upper bound, thus does not fit
+	 * the bounds created so far and hence can not be merged with the existing
+	 * bounds.
+	 */
+	if (cmpval < 0)
+		return false;
+
+	/*
+	 * Add bounds of the new merged partition. If the next lower bound is
+	 * higher than the last upper bound, add new range with index
+	 * corresponding to the lower bound as -1. If the merged lower bound
+	 * is same as the last merged upper bound, the last upper bound will be
+	 * reused as the lower bound of the next range.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, next_lb_datums);
+		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	return true;
+}
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			next_index = 0;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_kinds = NIL;
+	List	   *merged_indexes = NIL;
+	int			outer_lb_index;
+	int			inner_lb_index;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = inner_lb_index = 0;
+	while (outer_lb_index < outer_bi->ndatums ||
+		   inner_lb_index < inner_bi->ndatums)
+	{
+		PartitionRangeBound *merged_lb = NULL;
+		PartitionRangeBound *merged_ub = NULL;
+		int			merged_index = -1;
+		PartitionRangeBound outer_lb;
+		PartitionRangeBound outer_ub;
+		PartitionRangeBound inner_lb;
+		PartitionRangeBound inner_ub;
+		int			outer_part = -1;
+		int			inner_part = -1;
+		bool		overlap;
+		int			ub_cmpval;
+		int			lb_cmpval;
+
+		/* Get the range bounds of the next pair of partitions. */
+		if (outer_lb_index < outer_bi->ndatums)
+			outer_part = partition_get_range_bounds(outer_bi, outer_lb_index,
+													&outer_lb, &outer_ub);
+		if (inner_lb_index < inner_bi->ndatums)
+			inner_part = partition_get_range_bounds(inner_bi, inner_lb_index,
+													&inner_lb, &inner_ub);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (outer_lb_index >= outer_bi->ndatums)
+		{
+			overlap = false;
+			ub_cmpval = 1;
+			lb_cmpval = 1;
+		}
+		else if (inner_lb_index >= inner_bi->ndatums)
+		{
+			overlap = false;
+			ub_cmpval = -1;
+			lb_cmpval = -1;
+		}
+		else
+			overlap = partition_range_cmp(partnatts, partsupfuncs, partcollations,
+										  &outer_lb, &outer_ub, &inner_lb,
+										  &inner_ub, &ub_cmpval, &lb_cmpval);
+
+		if (overlap)
+		{
+			/*
+			 * The rows from overlapping portion of ranges on both sides may
+			 * join, hence the corresponding pair of partitions form a joining
+			 * pair. Match them and produce the bounds of the joint partition
+			 * and its index by merging the bounds according to the type of
+			 * join.
+			 */
+			partition_range_merge(partnatts, partsupfuncs, partcollations,
+								  jointype, &outer_lb, &outer_ub, &inner_lb,
+								  &inner_ub, &merged_lb, &merged_ub);
+
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
+													outer_part, inner_part,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				/* Failed to match the partitions. */
+				return NULL;
+			}
+
+			/*
+			 * If the ranges overlap but don't exactly match, a row from
+			 * non-overlapping portion of the range from one side of join may
+			 * find its join partner in the previous or next overlapping
+			 * partition or default partition on the other side , if such a
+			 * partition exists. All those cases, if true, will cause one
+			 * partition from that side to match at least two partitions on the
+			 * other side; a case that we do not support now. Previous
+			 * partition has been delt with in the previous iteration of this
+			 * loop, next partition will be delt in the next iteration. We will
+			 * deal with the default partition here.
+			 */
+			if ((lb_cmpval < 0 && inner_has_default) ||
+				/* Non-overlapping range on the lower side of outer range. */
+				(lb_cmpval > 0 && outer_has_default) ||
+				/* Non-overlapping range on the lower side of inner range. */
+				(ub_cmpval < 0 && outer_has_default) ||
+				/* Non-overlapping range on the upper side of inner range. */
+				(ub_cmpval > 0 && inner_has_default))
+				/* Non-overlapping range on the upper side of outer range. */
+				return NULL;
+		}
+
+		if (ub_cmpval == 0)
+		{
+			/* Upper bounds of both the ranges match. */
+			Assert(overlap);
+
+			/* Move to the next pair of partitions. */
+			Assert(outer_lb_index < outer_bi->ndatums);
+			Assert(inner_lb_index < inner_bi->ndatums);
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				if (inner_has_default || IS_OUTER_JOIN(jointype))
+				{
+					if (!process_outer_partition(&outer_map,
+												 &inner_map,
+												 outer_has_default,
+												 inner_has_default,
+												 outer_part,
+												 inner_default,
+												 jointype,
+												 outer_bi->strategy,
+												 &next_index,
+												 &default_index,
+												 &merged_index))
+						return NULL;
+				}
+
+				merged_lb = &outer_lb;
+				merged_ub = &outer_ub;
+			}
+
+			/* Move to the next partition on the outer side. */
+			Assert(outer_lb_index < outer_bi->ndatums);
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+		}
+		else
+		{
+			Assert(ub_cmpval > 0);
+
+			/* Upper bound of outer range higher than that of the inner. */
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				if (outer_has_default || jointype == JOIN_FULL)
+				{
+					if (!process_inner_partition(&outer_map,
+												 &inner_map,
+												 outer_has_default,
+												 inner_has_default,
+												 inner_part,
+												 outer_default,
+												 jointype,
+												 outer_bi->strategy,
+												 &next_index,
+												 &default_index,
+												 &merged_index))
+						return NULL;
+				}
+
+				merged_lb = &inner_lb;
+				merged_ub = &inner_ub;
+			}
+
+			/* Move to the next partition on the inner side. */
+			Assert(inner_lb_index < inner_bi->ndatums);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+
+		if (merged_index < 0)
+		{
+			/* We didn't find a new merged partition. */
+			continue;
+		}
+
+		/*
+		 * We have a valid partition index for the next partition of join. The
+		 * partition should have valid range.
+		 */
+		Assert(merged_lb && merged_ub);
+
+		/* Try merging new lower bound with the last upper bound. */
+		if (!partition_range_merge_next_lb(partnatts, partsupfuncs,
+										   partcollations,
+										   merged_lb->datums,
+										   merged_lb->kind, &merged_datums,
+										   &merged_kinds, &merged_indexes))
+			return NULL;
+
+		/* Add upper bound with the merged partition index. */
+		merged_datums = lappend(merged_datums, merged_ub->datums);
+		merged_kinds = lappend(merged_kinds, merged_ub->kind);
+		merged_indexes = lappend_int(merged_indexes, merged_index);
+	}
+
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+	{
+		if (!merge_default_partitions(&outer_map,
+									  &inner_map,
+									  outer_has_default,
+									  outer_default,
+									  inner_has_default,
+									  inner_default,
+									  jointype,
+									  &next_index,
+									  &default_index))
+			return NULL;
+	}
+	else
+		Assert(default_index == -1);
+
+	Assert(!outer_map.did_remapping && !inner_map.did_remapping);
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 &outer_map, &inner_map,
+								 next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes,
+													  merged_kinds,
+													  -1, default_index);
+	}
+
+	/* Free any memory we used in this function. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							JoinType jointype,
+							List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			next_index = 0;
+	int			null_index = -1;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			cnto;
+	int			cnti;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *merged_datum = NULL;
+		int			merged_index = -1;
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			cmpval;
+
+		/* Get the list datums of the next pair of partitions. */
+		odatums = cnto < outer_bi->ndatums ? outer_bi->datums[cnto] : NULL;
+		idatums = cnti < inner_bi->ndatums ? inner_bi->datums[cnti] : NULL;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+			cmpval = 1;
+		else if (cnti >= inner_bi->ndatums)
+			cmpval = -1;
+		else
+		{
+			Assert(odatums != NULL && idatums != NULL);
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0],
+													 idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			int			o_index = outer_bi->indexes[cnto];
+			int			i_index = inner_bi->indexes[cnti];
+
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			Assert(o_index >= 0 && i_index >= 0);
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
+													o_index, i_index,
+													&next_index);
+
+			if (merged_index < 0)
+				return NULL;
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			Assert(cnto < outer_bi->ndatums);
+
+			/* A datum missing from the inner side. */
+			merged_datum = odatums;
+
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				int			o_index = outer_bi->indexes[cnto];
+
+				Assert(o_index >= 0);
+				if (!process_outer_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 o_index,
+											 inner_default,
+											 jointype,
+											 outer_bi->strategy,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next datum on the outer side. */
+			cnto++;
+		}
+		else
+		{
+			Assert(cmpval > 0);
+			Assert(cnti < inner_bi->ndatums);
+
+			/* A datum missing from the outer side. */
+			merged_datum = idatums;
+
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				int			i_index = inner_bi->indexes[cnti];
+
+				Assert(i_index >= 0);
+				if (!process_inner_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 i_index,
+											 outer_default,
+											 jointype,
+											 outer_bi->strategy,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next datum on the inner side. */
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	/* Merge null partitions if any. */
+	if (outer_has_null || inner_has_null)
+	{
+		if (!merge_null_partitions(outer_bi, inner_bi,
+								   &outer_map, &inner_map,
+								   outer_has_default, inner_has_default,
+								   outer_has_null, inner_has_null,
+								   jointype, &next_index, &default_index,
+								   &null_index))
+			return NULL;
+	}
+	else
+		Assert(null_index == -1);
+
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+	{
+		if (!merge_default_partitions(&outer_map, &inner_map,
+									  outer_has_default, outer_default,
+									  inner_has_default, inner_default,
+									  jointype, &next_index, &default_index))
+			return NULL;
+	}
+	else
+		Assert(default_index == -1);
+
+	/* Fix the merged_indexes list if necessary. */
+	if (outer_map.did_remapping || inner_map.did_remapping)
+	{
+		Assert(jointype == JOIN_FULL);
+		fix_merged_indexes(&outer_map, &inner_map,
+						   next_index, merged_indexes);
+	}
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 &outer_map, &inner_map,
+								 next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) <= next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes, NIL,
+													  null_index, default_index);
+	}
+
+	/* Free up all extra memory before returning from this function. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+	list_free(merged_datums);
+	list_free(merged_indexes);
+
+	return merged_bounds;
+}
+
+/*
+ * init_partition_map
+ *
+ * Initialize a PartitionMap struct for given relation.
+ */
+static void
+init_partition_map(RelOptInfo *rel, PartitionMap *map)
+{
+	int			nparts = rel->nparts;
+	int			i;
+
+	map->nparts = nparts;
+	map->merged_indexes = (int *) palloc(sizeof(int) * nparts);
+	map->merged = (bool *) palloc(sizeof(bool) * nparts);
+	map->did_remapping = false;
+	map->old_indexes = (int *) palloc(sizeof(int) * nparts);
+	for (i = 0; i < nparts; i++)
+	{
+		map->merged_indexes[i] = map->old_indexes[i] = -1;
+		map->merged[i] = false;
+	}
+}
+
+/*
+ * free_partition_map
+ */
+static void
+free_partition_map(PartitionMap *map)
+{
+	pfree(map->merged_indexes);
+	pfree(map->merged);
+	pfree(map->old_indexes);
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given partitions.
+ */
+static int
+map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_index, int inner_index, int *next_index)
+{
+	int 		outer_merged_index;
+	bool 		outer_merged;
+	int 		inner_merged_index;
+	bool 		inner_merged;
+
+	Assert(outer_index >= 0 && outer_index < outer_map->nparts);
+	outer_merged_index = outer_map->merged_indexes[outer_index];
+	outer_merged = outer_map->merged[outer_index];
+	Assert(inner_index >= 0 && inner_index < inner_map->nparts);
+	inner_merged_index = inner_map->merged_indexes[inner_index];
+	inner_merged = inner_map->merged[inner_index];
+
+	/*
+	 * Handle cases where both partitions are mapped to merged partitions.
+	 */
+	if (outer_merged_index >= 0 && inner_merged_index >= 0)
+	{
+		/*
+		 * If the mereged partitions are the same, no need to do anything;
+		 * return the index of the merged partition.  Otherwise, if both
+		 * partitions are merged with dummy partitions, re-merge them; map
+		 * them to the merged partition with the smaller of the two merged
+		 * indexes and return the smaller index.  Otherwise they can't be
+		 * merged, so return -1.
+		 */
+		if (outer_merged_index == inner_merged_index)
+		{
+			Assert(outer_merged);
+			Assert(inner_merged);
+			return outer_merged_index;
+		}
+		if (!outer_merged && !inner_merged)
+		{
+			/*
+			 * Note that we will fix the larger index that have been added to
+			 * the merged_indexes list so far in fix_merged_indexes().
+			 */
+			if (outer_merged_index < inner_merged_index)
+			{
+				outer_map->merged[outer_index] = true;
+				inner_map->merged_indexes[inner_index] = outer_merged_index;
+				inner_map->merged[inner_index] = true;
+				inner_map->did_remapping = true;
+				inner_map->old_indexes[inner_index] = inner_merged_index;
+				return outer_merged_index;
+			}
+			else
+			{
+				inner_map->merged[inner_index] = true;
+				outer_map->merged_indexes[outer_index] = inner_merged_index;
+				outer_map->merged[outer_index] = true;
+				outer_map->did_remapping = true;
+				outer_map->old_indexes[outer_index] = outer_merged_index;
+				return inner_merged_index;
+			}
+		}
+		return -1;
+	}
+
+	/* At least one partition isn't mapped to a merged partition. */
+	Assert(outer_merged_index == -1 || inner_merged_index == -1);
+
+	/*
+	 * If neither of partitions isn't mapped, assign them a new merged
+	 * partition and return the index of the merged partition.  Otherwise, if
+	 * one of partitions is merged with a dummy relation (and the other isn't
+	 * merged), re-merge it with the other, with the same index, and return
+	 * the index.  Otherwise they can't be merged, so return -1.
+	 */
+	if (outer_merged_index == -1 && inner_merged_index == -1)
+	{
+		int 		merged_index = *next_index;
+
+		Assert(!outer_merged);
+		Assert(!inner_merged);
+		outer_map->merged_indexes[outer_index] = merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged_indexes[inner_index] = merged_index;
+		inner_map->merged[inner_index] = true;
+		*next_index = *next_index + 1;
+		return merged_index;
+	}
+	if (outer_merged_index >= 0 && !outer_map->merged[outer_index])
+	{
+		Assert(inner_merged_index == -1);
+		Assert(!inner_merged);
+		inner_map->merged_indexes[inner_index] = outer_merged_index;
+		inner_map->merged[inner_index] = true;
+		outer_map->merged[outer_index] = true;
+		return outer_merged_index;
+	}
+	if (inner_merged_index >= 0 && !inner_map->merged[inner_index])
+	{
+		Assert(outer_merged_index == -1);
+		Assert(!outer_merged);
+		outer_map->merged_indexes[outer_index] = inner_merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged[inner_index] = true;
+		return inner_merged_index;
+	}
+	return -1;
+}
+
+/*
+ * merge_partition_with_dummy
+ *
+ * *next_index is incremented.
+ */
+static int
+merge_partition_with_dummy(PartitionMap *map, int index, int *next_index)
+{
+	int 		merged_index = *next_index;
+
+	Assert(index >= 0 && index < map->nparts);
+	Assert(map->merged_indexes[index] == -1);
+	Assert(!map->merged[index]);
+	map->merged_indexes[index] = merged_index;
+	/* Leave the merged flag alone! */
+	*next_index = *next_index + 1;
+	return merged_index;
+}
+
+/*
+ * process_outer_partition
+ *
+ * Determine the merged partition associated with the given outer partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given outer partition.
+ */
+static bool
+process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index)
+{
+	Assert(outer_index >= 0);
+
+	/*
+	 * If the inner side has the default partition, the outer partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an outer join, in which case the outer partition has to be
+	 * scanned all the way anyway; if the outer partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the outer partition with a dummy partition.
+	 */
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0);
+
+		/*
+		 * If the outer side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (outer_has_default)
+			return false;
+
+		*merged_index = map_and_merge_partitions(outer_map, inner_map,
+												 outer_index, inner_default,
+												 next_index);
+		if (*merged_index == -1)
+			return false;
+
+		/*
+		 * If this is a FULL join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (*default_index == -1)
+				*default_index = *merged_index;
+			else
+				Assert(*merged_index == *default_index);
+			/* Don't add this index to the list of merged indexes. */
+			*merged_index = -1;
+		}
+	}
+	else
+	{
+		Assert(IS_OUTER_JOIN(jointype));
+		Assert(jointype != JOIN_RIGHT);
+
+		/*
+		 * In range partitioning, if the given outer partition is already
+		 * merged (eg, because we found an overlapping range earlier), we know
+		 * where it fits in the join result; nothing to do in that case.  Else
+		 * create a new merged partition.
+		 */
+		if (outer_map->merged_indexes[outer_index] >= 0)
+		{
+			if (strategy == PARTITION_STRATEGY_LIST)
+				*merged_index = outer_map->merged_indexes[outer_index];
+			else
+			{
+				Assert(strategy == PARTITION_STRATEGY_RANGE);
+				*merged_index = -1;
+			}
+		}
+		else
+			*merged_index = merge_partition_with_dummy(outer_map, outer_index,
+													   next_index);
+	}
+	return true;
+}
+
+/*
+ * process_inner_partition
+ *
+ * Determine the merged partition associated with the given inner partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given inner partition.
+ */
+static bool
+process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index)
+{
+	Assert(inner_index >= 0);
+
+	/*
+	 * If the outer side has the default partition, the inner partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an FULL join, in which case the inner partition has to be
+	 * scanned all the way anyway; if the inner partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the inner partition with a dummy partition.
+	 */
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0);
+
+		/*
+		 * If the inner side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (inner_has_default)
+			return false;
+
+		*merged_index = map_and_merge_partitions(outer_map, inner_map,
+												 outer_default, inner_index,
+												 next_index);
+		if (*merged_index == -1)
+			return false;
+
+		/*
+		 * If this is an outer join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (jointype == JOIN_LEFT || jointype == JOIN_ANTI ||
+			jointype == JOIN_FULL)
+		{
+			if (*default_index == -1)
+				*default_index = *merged_index;
+			else
+				Assert(*merged_index == *default_index);
+			/* Don't add this index to the list of merged indexes. */
+			*merged_index = -1;
+		}
+	}
+	else
+	{
+		Assert(jointype == JOIN_FULL);
+
+		/*
+		 * In range partitioning, if the given inner partition is already
+		 * merged (eg, because we found an overlapping range earlier), we know
+		 * where it fits in the join result; nothing to do in that case.  Else
+		 * create a new merged partition.
+		 */
+		if (inner_map->merged_indexes[inner_index] >= 0)
+		{
+			if (strategy == PARTITION_STRATEGY_LIST)
+				*merged_index = inner_map->merged_indexes[inner_index];
+			else
+			{
+				Assert(strategy == PARTITION_STRATEGY_RANGE);
+				*merged_index = -1;
+			}
+		}
+		else
+			*merged_index = merge_partition_with_dummy(inner_map, inner_index,
+													   next_index);
+	}
+	return true;
+}
+
+/*
+ * fix_merged_indexes
+ */
+static void
+fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes)
+{
+	int		   *new_indexes;
+	int			merged_index;
+	int			i;
+	ListCell   *lc;
+
+	new_indexes = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		new_indexes[i] = -1;
+
+	/* Build the mapping of old merged indexes to new merged indexes. */
+	if (outer_map->did_remapping)
+	{
+		for (i = 0; i < outer_map->nparts; i++)
+		{
+			merged_index = outer_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = outer_map->merged_indexes[i];
+		}
+	}
+	if (inner_map->did_remapping)
+	{
+		for (i = 0; i < inner_map->nparts; i++)
+		{
+			merged_index = inner_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = inner_map->merged_indexes[i];
+		}
+	}
+
+	/* Fix the merged_indexes list using the mapping. */
+	foreach(lc, merged_indexes)
+	{
+		merged_index = lfirst_int(lc);
+		Assert(merged_index >= 0);
+		if (new_indexes[merged_index] >= 0)
+			lfirst_int(lc) = new_indexes[merged_index];
+	}
+
+	pfree(new_indexes);
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * This function produces the list pairs of partitions which produce merged
+ * partitions in the order of merged partition indexes.
+ *
+ * nmerged is the number of merged partitions.
+ *
+ * If successful, the list pairs are returned as two separate lists,
+ * outer_part_list and inner_part_list, one for each side.  Otherwise, those
+ * lists will be set to NIL.
+ */
+static void
+generate_matching_part_pairs(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 PartitionMap *outer_map, PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_part_list, List **inner_part_list)
+{
+	int			outer_nparts = outer_map->nparts;
+	int			inner_nparts = inner_map->nparts;
+	int		   *outer_part_array;
+	int		   *inner_part_array;
+	int			max_nparts;
+	int 		i;
+
+	Assert(outer_nparts == outer_rel->nparts);
+	Assert(inner_nparts == inner_rel->nparts);
+
+	outer_part_array = (int *) palloc(sizeof(int) * nmerged);
+	inner_part_array = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		outer_part_array[i] = inner_part_array[i] = -1;
+
+	/* Set pairs of matching partitions. */
+	max_nparts = Max(outer_nparts, inner_nparts);
+	for (i = 0; i < max_nparts; i++)
+	{
+		if (i < outer_nparts)
+		{
+			int 		merged_index = outer_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				outer_part_array[merged_index] = i;
+			}
+		}
+		if (i < inner_nparts)
+		{
+			int 		merged_index = inner_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				inner_part_array[merged_index] = i;
+			}
+		}
+	}
+
+	*outer_part_list = NIL;
+	*inner_part_list = NIL;
+
+	/* Build the list pairs. */
+	for (i = 0; i < nmerged; i++)
+	{
+		int			outer_part = outer_part_array[i];
+		int			inner_part = inner_part_array[i];
+
+		/*
+		 * If both partitions are dummy, it means the merged partition was
+		 * removed by partition re-merging in map_and_merge_partitions();
+		 * ignore the merged partition
+		 */
+		if (outer_part == -1 && inner_part == -1)
+			continue;
+
+		*outer_part_list = lappend(*outer_part_list, outer_part >= 0 ?
+								   outer_rel->part_rels[outer_part] : NULL);
+		*inner_part_list = lappend(*inner_part_list, inner_part >= 0 ?
+								   inner_rel->part_rels[inner_part] : NULL);
+	}
+
+	pfree(outer_part_array);
+	pfree(inner_part_array);
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * Merge default partitions from both sides, if any, and assign the default
+ * partition for the join result, if necessary.
+ *
+ * If both the relations have default partitions, try mapping those to each
+ * other. If the mapping succeeds corresponding merged partition will act as
+ * the default partition of the join result.
+ *
+ * If inner side of the join has default but not the outer side, rows in it
+ * won't appear in the join result. So don't create a default partition. If
+ * outer side of the join has default but not the inner side, rows in it will
+ * appear in the join result, so create a default merged partition.
+ */
+static bool
+merge_default_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 bool outer_has_default, int outer_default,
+						 bool inner_has_default, int inner_default,
+						 JoinType jointype, int *next_index,
+						 int *default_index)
+{
+	Assert(outer_has_default || inner_has_default);
+
+	if (outer_has_default && !inner_has_default)
+	{
+		if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+			jointype == JOIN_ANTI)
+		{
+			int			merged_index;
+
+			Assert(outer_default >= 0 && outer_default < outer_map->nparts);
+			merged_index = outer_map->merged_indexes[outer_default];
+			if (merged_index == -1)
+			{
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(outer_map,
+															outer_default,
+															next_index);
+			}
+			else
+				Assert(*default_index == merged_index);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else if (!outer_has_default && inner_has_default)
+	{
+		if (jointype == JOIN_FULL)
+		{
+			int			merged_index;
+
+			Assert(inner_default >= 0 && inner_default < inner_map->nparts);
+			merged_index = inner_map->merged_indexes[inner_default];
+			if (merged_index == -1)
+			{
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(inner_map,
+															inner_default,
+															next_index);
+			}
+			else
+				Assert(*default_index == merged_index);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		*default_index = map_and_merge_partitions(outer_map,
+												  inner_map,
+												  outer_default,
+												  inner_default,
+												  next_index);
+		if (*default_index == -1)
+			return false;
+	}
+
+	return true;
+}
+
+/*
+ * merge_null_partitions
+ *
+ * Merge NULL partitions, i.e. a partition that can hold NULL values for a list
+ * partitioned table, if any. Find the index of merged partition to which the
+ * NULL values would belong in the join result. If one joining relation has a
+ * NULL partition but not the other, try matching it with the default partition
+ * from the other relation since the default partition may have rows with NULL
+ * partition key. We can eliminate a NULL partition when it appears only on the
+ * inner side of the join and the outer side doesn't have a default partition.
+ *
+ * When the equality operator used for join is strict, two NULL values will not
+ * be considered as equal, and thus a NULL partition can be eliminated for an
+ * inner join. But we don't check the strictness operator here.
+ */
+static bool
+merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_map, PartitionMap *inner_map,
+					  bool outer_has_default, bool inner_has_default,
+					  bool outer_has_null, bool inner_has_null,
+					  JoinType jointype, int *next_index,
+					  int *default_index, int *null_index)
+{
+	Assert(outer_has_null || inner_has_null);
+	Assert(*null_index == -1);
+
+	if (outer_has_null && !inner_has_null)
+	{
+		int			merged_index = -1;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join to which the NULL partition matches will
+		 * contain the NULL values and thus become the NULL partition of the
+		 * the join.
+		 */
+		if (inner_has_default || IS_OUTER_JOIN(jointype))
+		{
+			if (!process_outer_partition(outer_map,
+										 inner_map,
+										 outer_has_default,
+										 inner_has_default,
+										 outer_bi->null_index,
+										 inner_bi->default_index,
+										 jointype,
+										 outer_bi->strategy,
+										 next_index,
+										 default_index,
+										 &merged_index))
+				return false;
+		}
+		*null_index = merged_index;
+	}
+	else if (!outer_has_null && inner_has_null)
+	{
+		int			merged_index = -1;
+
+		/*
+		 * If the NULL partition was missing from the outer side of the join,
+		 * the partition of the join to which the NULL partition matches will
+		 * contain the NULL values and thus become the NULL partition of the
+		 * the join.
+		 */
+		if (outer_has_default || jointype == JOIN_FULL)
+		{
+			if (!process_inner_partition(outer_map,
+										 inner_map,
+										 outer_has_default,
+										 inner_has_default,
+										 inner_bi->null_index,
+										 outer_bi->default_index,
+										 jointype,
+										 outer_bi->strategy,
+										 next_index,
+										 default_index,
+										 &merged_index))
+				return false;
+		}
+		*null_index = merged_index;
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		*null_index = map_and_merge_partitions(outer_map,
+											   inner_map,
+											   outer_bi->null_index,
+											   inner_bi->null_index,
+											   next_index);
+		if (*null_index == -1)
+			return false;
+	}
+
+	return true;
+}
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index 23a06d718e3..e06eb9aaae5 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -577,8 +577,10 @@ typedef struct PartitionSchemeData *PartitionScheme;
  *		part_scheme - Partitioning scheme of the relation
  *		nparts - Number of partitions
  *		boundinfo - Partition bounds
+ *		merged - true if partition bounds are merged ones
  *		partition_qual - Partition constraint if not the root
  *		part_rels - RelOptInfos for each partition
+ *		all_partrels - Relids set of all partition relids
  *		partexprs, nullable_partexprs - Partition key expressions
  *		partitioned_child_rels - RT indexes of unpruned partitions of
  *								 this relation that are partitioned tables
@@ -718,9 +720,12 @@ typedef struct RelOptInfo
 	PartitionScheme part_scheme;	/* Partitioning scheme. */
 	int			nparts;			/* number of partitions */
 	struct PartitionBoundInfoData *boundinfo;	/* Partition bounds */
+	bool		merged;			/* true if partition bounds were created by
+								 * partition_bounds_merge() */
 	List	   *partition_qual; /* partition constraint */
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of partitions,
 									 * stored in the same order of bounds */
+	Relids		all_partrels;	/* Relids set of all partition relids */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 	List	   *partitioned_child_rels; /* List of RT indexes. */
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index 0d0fd42b181..9292aa11e5e 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -16,6 +16,7 @@
 #include "nodes/pg_list.h"
 #include "partitioning/partdefs.h"
 #include "utils/relcache.h"
+struct RelOptInfo;				/* avoid including pathnodes.h here */
 
 
 /*
@@ -108,5 +109,11 @@ extern int	partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 										  int nvalues, Datum *values, bool *is_equal);
 extern int	partition_hash_bsearch(PartitionBoundInfo boundinfo,
 								   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   struct RelOptInfo *outer_rel, struct RelOptInfo *inner_rel,
+					   JoinType jointype,
+					   List **outer_parts, List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index cad8dd591aa..89f0c6a9ee7 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -8,59 +8,86 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-(4 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+(7 rows)
 
 -- left outer join, with whole-row reference; partitionwise join does not apply
 EXPLAIN (COSTS OFF)
@@ -72,35 +99,50 @@ SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER
    ->  Hash Right Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(22 rows)
 
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-      t1      |      t2      
---------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
- (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
- (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
- (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
- (550,0,0550) | 
-(12 rows)
+       t1       |       t2       
+----------------+----------------
+ (-250,0,-0250) | 
+ (-200,0,-0200) | 
+ (-150,0,-0150) | (0,-150,-0150)
+ (-100,0,-0100) | 
+ (-50,0,-0050)  | 
+ (0,0,0000)     | (0,0,0000)
+ (50,0,0050)    | 
+ (100,0,0100)   | 
+ (150,0,0150)   | (0,150,0150)
+ (200,0,0200)   | 
+ (250,0,0250)   | 
+ (300,0,0300)   | (0,300,0300)
+ (350,0,0350)   | 
+ (400,0,0400)   | 
+ (450,0,0450)   | (0,450,0450)
+ (500,0,0500)   | 
+ (550,0,0550)   | 
+ (600,0,0600)   | (0,600,0600)
+ (650,0,0650)   | 
+ (700,0,0700)   | 
+ (750,0,0750)   | (0,750,0750)
+(21 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
@@ -112,35 +154,53 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
    ->  Append
          ->  Hash Right Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Right Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t2_2.b)
-(20 rows)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+      |       | -225 | -0225
+      |       |  -75 | -0075
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(14 rows)
 
 -- full outer join, with placeholder vars
 EXPLAIN (COSTS OFF)
@@ -148,8 +208,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                             QUERY PLAN                            
 ------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               Filter: (((50) = prt1_p0.a) OR ((75) = prt2_p0.b))
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
@@ -174,7 +242,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               Filter: (((50) = prt1_p4.a) OR ((75) = prt2_p4.b))
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
@@ -211,35 +287,44 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Hash Right Join
-         Hash Cond: (prt2_p2.b = prt1_p1.a)
+         Hash Cond: (prt2_p2.b = prt1_p0.a)
          ->  Append
                ->  Seq Scan on prt2_p2
                      Filter: (b > 250)
                ->  Seq Scan on prt2_p3
                      Filter: (b > 250)
+               ->  Seq Scan on prt2_p4
+                     Filter: (b > 250)
          ->  Hash
                ->  Append
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p1
                            Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p2
                            Filter: ((a < 450) AND (b = 0))
-(15 rows)
+(19 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-(9 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+(14 rows)
 
 -- Currently we can't do partitioned join if nullable-side partitions are pruned
 EXPLAIN (COSTS OFF)
@@ -247,11 +332,13 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                       QUERY PLAN                      
 ------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Hash Full Join
-         Hash Cond: (prt1_p1.a = prt2_p2.b)
-         Filter: ((prt1_p1.b = 0) OR (prt2_p2.a = 0))
+         Hash Cond: (prt1_p0.a = prt2_p2.b)
+         Filter: ((prt1_p0.b = 0) OR (prt2_p2.a = 0))
          ->  Append
+               ->  Seq Scan on prt1_p0
+                     Filter: (a < 450)
                ->  Seq Scan on prt1_p1
                      Filter: (a < 450)
                ->  Seq Scan on prt1_p2
@@ -262,64 +349,147 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                            Filter: (b > 250)
                      ->  Seq Scan on prt2_p3
                            Filter: (b > 250)
-(16 rows)
+                     ->  Seq Scan on prt2_p4
+                           Filter: (b > 250)
+(20 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
-(12 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+      |       | 375 | 0375
+      |       | 450 | 0450
+      |       | 525 | 0525
+      |       | 600 | 0600
+      |       | 675 | 0675
+      |       | 750 | 0750
+(20 rows)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Semi Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Semi Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                            Filter: (a = 0)
-         ->  Nested Loop Semi Join
-               Join Filter: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
-               ->  Materialize
-                     ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
-(24 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt1_p3 t2_3
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+                           Filter: (b = 0)
+(37 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
 
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
@@ -330,27 +500,82 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
    ->  Append
          ->  Hash Anti Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Hash Anti Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Hash Anti Join
                Hash Cond: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
-(17 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(27 rows)
 
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
-  sum  |         avg          | sum  |         avg         
--------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+  sum  |         avg          | sum  |        avg         
+-------+----------------------+------+--------------------
+ 95550 | 273.0000000000000000 | 2200 | 6.2857142857142857
 (1 row)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Append
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p0 t1
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+               Index Cond: (a = t1.b)
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.b = t2_1.a)
+         ->  Seq Scan on prt2_p1 t1_1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p1 t2_1
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p2 t1_2
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+               Index Cond: (a = t1_2.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p3 t1_3
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+               Index Cond: (a = t1_3.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p4 t1_4
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+               Index Cond: (a = t1_4.b)
+(27 rows)
+
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+  b   |   c   
+------+-------
+ -225 | -0225
+  -75 | -0075
+   75 | 0075
+  225 | 0225
+  375 | 0375
+  525 | 0525
+  675 | 0675
+(7 rows)
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -362,49 +587,74 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
+                     ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
                            Index Cond: (a = t1.a)
-                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t3
                            Index Cond: (b = t2.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_1
+                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
                            Index Cond: (a = t1_1.a)
-                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3_1
                            Index Cond: (b = t2_1.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_2
+                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
                            Index Cond: (a = t1_2.a)
-                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_2
                            Index Cond: (b = t2_2.a)
-(27 rows)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                           Index Cond: (a = t1_3.a)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_3
+                           Index Cond: (b = t2_3.a)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                           Index Cond: (a = t1_4.a)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t3_4
+                           Index Cond: (b = t2_4.a)
+(43 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   | t2a | t3a | least 
------+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
- 100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
- 250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
- 400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
- 550 | 0 | 0550 |     |     |      
-(12 rows)
+  a   | b |   c   | t2a  | t3a | least 
+------+---+-------+------+-----+-------
+ -250 | 0 | -0250 |      |     |      
+ -200 | 0 | -0200 |      |     |      
+ -150 | 0 | -0150 | -150 |   0 |  -150
+ -100 | 0 | -0100 |      |     |      
+  -50 | 0 | -0050 |      |     |      
+    0 | 0 | 0000  |    0 |   0 |     0
+   50 | 0 | 0050  |      |     |      
+  100 | 0 | 0100  |      |     |      
+  150 | 0 | 0150  |  150 |   0 |   150
+  200 | 0 | 0200  |      |     |      
+  250 | 0 | 0250  |      |     |      
+  300 | 0 | 0300  |  300 |   0 |   300
+  350 | 0 | 0350  |      |     |      
+  400 | 0 | 0400  |      |     |      
+  450 | 0 | 0450  |  450 |   0 |   450
+  500 | 0 | 0500  |      |     |      
+  550 | 0 | 0550  |      |     |      
+  600 | 0 | 0600  |  600 |   0 |   600
+  650 | 0 | 0650  |      |     |      
+  700 | 0 | 0700  |      |     |      
+  750 | 0 | 0750  |  750 |   0 |   750
+(21 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
@@ -418,64 +668,95 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
          Hash Cond: ((t1.c)::text = (t2.c)::text)
          Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Seq Scan on prt1_p1 t1
-               ->  Seq Scan on prt1_p2 t1_1
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
          ->  Hash
                ->  Append
                      ->  Hash Join
                            Hash Cond: (t2.a = t3.b)
-                           ->  Seq Scan on prt1_p1 t2
+                           ->  Seq Scan on prt1_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t3
+                                 ->  Seq Scan on prt2_p0 t3
                      ->  Hash Join
                            Hash Cond: (t2_1.a = t3_1.b)
-                           ->  Seq Scan on prt1_p2 t2_1
+                           ->  Seq Scan on prt1_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t3_1
+                                 ->  Seq Scan on prt2_p1 t3_1
                      ->  Hash Join
                            Hash Cond: (t2_2.a = t3_2.b)
-                           ->  Seq Scan on prt1_p3 t2_2
+                           ->  Seq Scan on prt1_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p3 t3_2
-(26 rows)
+                                 ->  Seq Scan on prt2_p2 t3_2
+                     ->  Hash Join
+                           Hash Cond: (t2_3.a = t3_3.b)
+                           ->  Seq Scan on prt1_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t3_3
+                     ->  Hash Join
+                           Hash Cond: (t2_4.a = t3_4.b)
+                           ->  Seq Scan on prt1_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t3_4
+(38 rows)
 
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, t2.b t2b, t2.c t2c, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
-  a  | t2a | t2c  
------+-----+------
-   0 |   0 | 0000
-  50 |     | 
- 100 |     | 
- 150 | 150 | 0150
- 200 |     | 
- 250 |     | 
- 300 | 300 | 0300
- 350 |     | 
- 400 |     | 
- 450 | 450 | 0450
- 500 |     | 
- 550 |     | 
-(12 rows)
+  a   | t2a  |  t2c  
+------+------+-------
+ -250 |      | 
+ -200 |      | 
+ -150 | -150 | -0150
+ -100 |      | 
+  -50 |      | 
+    0 |    0 | 0000
+   50 |      | 
+  100 |      | 
+  150 |  150 | 0150
+  200 |      | 
+  250 |      | 
+  300 |  300 | 0300
+  350 |      | 
+  400 |      | 
+  450 |  450 | 0450
+  500 |      | 
+  550 |      | 
+  600 |  600 | 0600
+  650 |      | 
+  700 |      | 
+  750 |  750 | 0750
+(21 rows)
 
 --
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
@@ -486,32 +767,49 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
    ->  Append
          ->  Hash Join
                Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
-               ->  Seq Scan on prt2_e_p1 t2
+               ->  Seq Scan on prt2_e_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 t1
+                     ->  Seq Scan on prt1_e_p0 t1
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
-               ->  Seq Scan on prt2_e_p2 t2_1
+               Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+               ->  Seq Scan on prt1_e_p1 t1_1
+                     Filter: (c = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 t1_1
-                           Filter: (c = 0)
+                     ->  Seq Scan on prt2_e_p1 t2_1
          ->  Hash Join
                Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
-               ->  Seq Scan on prt2_e_p3 t2_2
+               ->  Seq Scan on prt2_e_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 t1_2
+                     ->  Seq Scan on prt1_e_p2 t1_2
                            Filter: (c = 0)
-(21 rows)
+         ->  Hash Join
+               Hash Cond: (((t2_3.b + t2_3.a) / 2) = ((t1_3.a + t1_3.b) / 2))
+               ->  Seq Scan on prt2_e_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p3 t1_3
+                           Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (((t2_4.b + t2_4.a) / 2) = ((t1_4.a + t1_4.b) / 2))
+               ->  Seq Scan on prt2_e_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4 t1_4
+                           Filter: (c = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
- 150 | 0 | 150 | 0
- 300 | 0 | 300 | 0
- 450 | 0 | 450 | 0
-(4 rows)
+  a   | c |  b   | c 
+------+---+------+---
+ -250 | 0 | -250 | 0
+ -100 | 0 | -100 | 0
+    0 | 0 |    0 | 0
+    0 | 0 |    0 | 0
+  150 | 0 |  150 | 0
+  300 | 0 |  300 | 0
+  450 | 0 |  450 | 0
+  600 | 0 |  600 | 0
+  750 | 0 |  750 | 0
+(9 rows)
 
 --
 -- N-way join
@@ -524,154 +822,232 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = ((t3.a + t3.b) / 2))
+               Join Filter: (t1.a = t2.b)
                ->  Hash Join
-                     Hash Cond: (t2.b = t1.a)
-                     ->  Seq Scan on prt2_p1 t2
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on prt1_e_p0 t3
                      ->  Hash
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                     Index Cond: (((a + b) / 2) = t2.b)
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
+                     Index Cond: (b = ((t3.a + t3.b) / 2))
          ->  Nested Loop
                Join Filter: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_1.b = t1_1.a)
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
+               ->  Index Scan using iprt1_e_p4_ab2 on prt1_e_p1 t3_1
                      Index Cond: (((a + b) / 2) = t2_1.b)
          ->  Nested Loop
                Join Filter: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_2.b = t1_2.a)
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
+               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_2
                      Index Cond: (((a + b) / 2) = t2_2.b)
-(33 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = ((t3_3.a + t3_3.b) / 2))
+               ->  Nested Loop
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t2_3.b)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t2_4.b)
+               ->  Hash Join
+                     Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                     ->  Seq Scan on prt1_e_p4 t3_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_p4 t1_4
+                                 Filter: (b = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = ((t3_4.a + t3_4.b) / 2))
+(52 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
-(4 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(8 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                          QUERY PLAN                          
---------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Hash Right Join
                Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
-               ->  Seq Scan on prt1_e_p1 t3
+               ->  Seq Scan on prt1_e_p0 t3
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2.b = t1.a)
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_1.a)
-               ->  Seq Scan on prt1_e_p2 t3_1
+               ->  Seq Scan on prt1_e_p1 t3_1
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_1.b = t1_1.a)
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_2.a)
-               ->  Seq Scan on prt1_e_p3 t3_2
+               ->  Seq Scan on prt1_e_p2 t3_2
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_2.b = t1_2.a)
-                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                                        Filter: (b = 0)
-(33 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+               ->  Seq Scan on prt1_e_p4 t3_4
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_4.b = t1_4.a)
+                           ->  Seq Scan on prt2_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_p4 t1_4
+                                       Filter: (b = 0)
+(51 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                            QUERY PLAN                             
--------------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p1 t3
+                           ->  Seq Scan on prt1_e_p0 t3
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p1_b on prt2_p1 t2
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
                      Index Cond: (b = t1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p2 t3_1
+                           ->  Seq Scan on prt1_e_p1 t3_1
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
+               ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
                      Index Cond: (b = t1_1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p3 t3_2
+                           ->  Seq Scan on prt1_e_p2 t3_2
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
                      Index Cond: (b = t1_2.a)
-(30 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_e_p3 t3_3
+                           Filter: (c = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                           Index Cond: (a = ((t3_3.a + t3_3.b) / 2))
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Nested Loop Left Join
+               ->  Hash Right Join
+                     Hash Cond: (t1_4.a = ((t3_4.a + t3_4.b) / 2))
+                     ->  Seq Scan on prt1_p4 t1_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p4 t3_4
+                                 Filter: (c = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = t1_4.a)
+(47 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -680,21 +1056,34 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                                                    QUERY PLAN                                                   
 ----------------------------------------------------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   Sort Key: prt1_p0.a, prt2_p0.b, ((prt1_e_p0.a + prt1_e_p0.b))
    ->  Append
          ->  Hash Full Join
-               Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
-               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               Hash Cond: (prt1_p0.a = ((prt1_e_p0.a + prt1_e_p0.b) / 2))
+               Filter: ((prt1_p0.a = (50)) OR (prt2_p0.b = (75)) OR (((prt1_e_p0.a + prt1_e_p0.b) / 2) = (50)))
                ->  Hash Full Join
-                     Hash Cond: (prt1_p1.a = prt2_p1.b)
-                     ->  Seq Scan on prt1_p1
+                     Hash Cond: (prt1_p0.a = prt2_p0.b)
+                     ->  Seq Scan on prt1_p0
                            Filter: (b = 0)
                      ->  Hash
-                           ->  Seq Scan on prt2_p1
+                           ->  Seq Scan on prt2_p0
                                  Filter: (a = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1
+                     ->  Seq Scan on prt1_e_p0
                            Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = prt1_p1.a)
+               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               ->  Seq Scan on prt1_e_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on prt1_p1
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1
+                                       Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
                Filter: ((prt1_p2.a = (50)) OR (prt2_p2.b = (75)) OR (((prt1_e_p2.a + prt1_e_p2.b) / 2) = (50)))
@@ -721,7 +1110,20 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                ->  Hash
                      ->  Seq Scan on prt1_e_p3
                            Filter: (c = 0)
-(42 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = ((prt1_e_p4.a + prt1_e_p4.b) / 2))
+               Filter: ((prt1_p4.a = (50)) OR (prt2_p4.b = (75)) OR (((prt1_e_p4.a + prt1_e_p4.b) / 2) = (50)))
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = prt2_p4.b)
+                     ->  Seq Scan on prt1_p4
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4
+                                 Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4
+                           Filter: (c = 0)
+(68 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
@@ -739,172 +1141,260 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = t1_3.b)
+               Join Filter: (t1.a = t1_5.b)
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Join
-                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
-                           ->  Seq Scan on prt1_e_p1 t2
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t1_3
+                                 ->  Seq Scan on prt2_p0 t1_5
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
                      Index Cond: (a = ((t2.a + t2.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_1.a = t1_4.b)
+               Join Filter: (t1_1.a = t1_6.b)
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Join
-                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
-                           ->  Seq Scan on prt1_e_p2 t2_1
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_6.b)
+                           ->  Seq Scan on prt1_e_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t1_4
+                                 ->  Seq Scan on prt2_p1 t1_6
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
                      Index Cond: (a = ((t2_1.a + t2_1.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_2.a = t1_5.b)
+               Join Filter: (t1_2.a = t1_7.b)
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
+                           ->  Seq Scan on prt2_p2 t1_7
                                  Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
+                           ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t2_2
+                                 Index Cond: (((a + b) / 2) = t1_7.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t1_8.b)
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Nested Loop
+                           ->  Seq Scan on prt2_p3 t1_8
+                                 Filter: (a = 0)
+                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_3
+                                 Index Cond: (((a + b) / 2) = t1_8.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = ((t2_3.a + t2_3.b) / 2))
+                     Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t1_9.b)
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Join
+                           Hash Cond: (((t2_4.a + t2_4.b) / 2) = t1_9.b)
+                           ->  Seq Scan on prt1_e_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t1_9
+                                       Filter: (a = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = ((t2_4.a + t2_4.b) / 2))
+                     Filter: (b = 0)
+(66 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                               QUERY PLAN                                
--------------------------------------------------------------------------
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
-                           ->  Seq Scan on prt2_p1 t1_3
+                           Hash Cond: (t1_5.b = ((t1_10.a + t1_10.b) / 2))
+                           ->  Seq Scan on prt2_p0 t1_5
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t1_6
+                                 ->  Seq Scan on prt1_e_p0 t1_10
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
-                     Index Cond: (a = t1_3.b)
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t1_5.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
-                           ->  Seq Scan on prt2_p2 t1_4
+                           Hash Cond: (t1_6.b = ((t1_11.a + t1_11.b) / 2))
+                           ->  Seq Scan on prt2_p1 t1_6
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t1_7
+                                 ->  Seq Scan on prt1_e_p1 t1_11
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
-                     Index Cond: (a = t1_4.b)
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
+                     Index Cond: (a = t1_6.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
-                           ->  Seq Scan on prt2_p3 t1_5
+                           Hash Cond: (t1_7.b = ((t1_12.a + t1_12.b) / 2))
+                           ->  Seq Scan on prt2_p2 t1_7
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p3 t1_8
+                                 ->  Seq Scan on prt1_e_p2 t1_12
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t1_5.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t1_7.b)
                      Filter: (b = 0)
-(39 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_8.b = ((t1_13.a + t1_13.b) / 2))
+                           ->  Seq Scan on prt2_p3 t1_8
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p3 t1_13
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t1_8.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_9.b = ((t1_14.a + t1_14.b) / 2))
+                           ->  Seq Scan on prt2_p4 t1_9
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t1_14
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = t1_9.b)
+                     Filter: (b = 0)
+(63 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 -- test merge joins
 SET enable_hashjoin TO off;
 SET enable_nestloop TO off;
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                           QUERY PLAN                           
-----------------------------------------------------------------
+                            QUERY PLAN                            
+------------------------------------------------------------------
  Merge Append
    Sort Key: t1.a
    ->  Merge Semi Join
-         Merge Cond: (t1.a = t1_3.b)
+         Merge Cond: (t1.a = t1_5.b)
          ->  Sort
                Sort Key: t1.a
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+               Merge Cond: (t1_5.b = (((t1_10.a + t1_10.b) / 2)))
                ->  Sort
-                     Sort Key: t1_3.b
-                     ->  Seq Scan on prt2_p1 t1_3
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on prt2_p0 t1_5
                ->  Sort
-                     Sort Key: (((t1_6.a + t1_6.b) / 2))
-                     ->  Seq Scan on prt1_e_p1 t1_6
+                     Sort Key: (((t1_10.a + t1_10.b) / 2))
+                     ->  Seq Scan on prt1_e_p0 t1_10
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_1.a = t1_4.b)
+         Merge Cond: (t1_1.a = t1_6.b)
          ->  Sort
                Sort Key: t1_1.a
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+               Merge Cond: (t1_6.b = (((t1_11.a + t1_11.b) / 2)))
                ->  Sort
-                     Sort Key: t1_4.b
-                     ->  Seq Scan on prt2_p2 t1_4
+                     Sort Key: t1_6.b
+                     ->  Seq Scan on prt2_p1 t1_6
                ->  Sort
-                     Sort Key: (((t1_7.a + t1_7.b) / 2))
-                     ->  Seq Scan on prt1_e_p2 t1_7
+                     Sort Key: (((t1_11.a + t1_11.b) / 2))
+                     ->  Seq Scan on prt1_e_p1 t1_11
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_2.a = t1_5.b)
+         Merge Cond: (t1_2.a = t1_7.b)
          ->  Sort
                Sort Key: t1_2.a
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+               Merge Cond: (t1_7.b = (((t1_12.a + t1_12.b) / 2)))
                ->  Sort
-                     Sort Key: t1_5.b
-                     ->  Seq Scan on prt2_p3 t1_5
+                     Sort Key: t1_7.b
+                     ->  Seq Scan on prt2_p2 t1_7
                ->  Sort
-                     Sort Key: (((t1_8.a + t1_8.b) / 2))
-                     ->  Seq Scan on prt1_e_p3 t1_8
+                     Sort Key: (((t1_12.a + t1_12.b) / 2))
+                     ->  Seq Scan on prt1_e_p2 t1_12
                            Filter: (c = 0)
-(47 rows)
+   ->  Merge Semi Join
+         Merge Cond: (t1_3.a = t1_8.b)
+         ->  Sort
+               Sort Key: t1_3.a
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_8.b = (((t1_13.a + t1_13.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_8.b
+                     ->  Seq Scan on prt2_p3 t1_8
+               ->  Sort
+                     Sort Key: (((t1_13.a + t1_13.b) / 2))
+                     ->  Seq Scan on prt1_e_p3 t1_13
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_4.a = t1_9.b)
+         ->  Sort
+               Sort Key: t1_4.a
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_9.b = (((t1_14.a + t1_14.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_9.b
+                     ->  Seq Scan on prt2_p4 t1_9
+               ->  Sort
+                     Sort Key: (((t1_14.a + t1_14.b) / 2))
+                     ->  Seq Scan on prt1_e_p4 t1_14
+                           Filter: (c = 0)
+(77 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
@@ -921,14 +1411,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
                            ->  Sort
                                  Sort Key: (((t3.a + t3.b) / 2))
-                                 ->  Seq Scan on prt1_e_p1 t3
+                                 ->  Seq Scan on prt1_e_p0 t3
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1.a
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                ->  Sort
                      Sort Key: t2.b
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Merge Left Join
                Merge Cond: (t1_1.a = t2_1.b)
                ->  Sort
@@ -937,14 +1427,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
                            ->  Sort
                                  Sort Key: (((t3_1.a + t3_1.b) / 2))
-                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                 ->  Seq Scan on prt1_e_p1 t3_1
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_1.a
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                ->  Sort
                      Sort Key: t2_1.b
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Merge Left Join
                Merge Cond: (t1_2.a = t2_2.b)
                ->  Sort
@@ -953,32 +1443,74 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
                            ->  Sort
                                  Sort Key: (((t3_2.a + t3_2.b) / 2))
-                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                 ->  Seq Scan on prt1_e_p2 t3_2
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_2.a
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                ->  Sort
                      Sort Key: t2_2.b
-                     ->  Seq Scan on prt2_p3 t2_2
-(51 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Merge Left Join
+               Merge Cond: (t1_3.a = t2_3.b)
+               ->  Sort
+                     Sort Key: t1_3.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_3.a + t3_3.b) / 2)) = t1_3.a)
+                           ->  Sort
+                                 Sort Key: (((t3_3.a + t3_3.b) / 2))
+                                 ->  Seq Scan on prt1_e_p3 t3_3
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_3.a
+                                 ->  Seq Scan on prt1_p3 t1_3
+               ->  Sort
+                     Sort Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Merge Left Join
+               Merge Cond: (t1_4.a = t2_4.b)
+               ->  Sort
+                     Sort Key: t1_4.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_4.a + t3_4.b) / 2)) = t1_4.a)
+                           ->  Sort
+                                 Sort Key: (((t3_4.a + t3_4.b) / 2))
+                                 ->  Seq Scan on prt1_e_p4 t3_4
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_4.a
+                                 ->  Seq Scan on prt1_p4 t1_4
+               ->  Sort
+                     Sort Key: t2_4.b
+                     ->  Seq Scan on prt2_p4 t2_4
+(83 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- MergeAppend on nullable column
 -- This should generate a partitionwise join, but currently fails to
@@ -987,12 +1519,14 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Merge Left Join
-         Merge Cond: (prt1_p1.a = prt2_p2.b)
+         Merge Cond: (prt1_p0.a = prt2_p2.b)
          ->  Sort
-               Sort Key: prt1_p1.a
+               Sort Key: prt1_p0.a
                ->  Append
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p1
                            Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p2
@@ -1004,21 +1538,28 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                            Filter: (b > 250)
                      ->  Seq Scan on prt2_p3
                            Filter: (b > 250)
-(18 rows)
+                     ->  Seq Scan on prt2_p4
+                           Filter: (b > 250)
+(22 rows)
 
 SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  b  
------+-----
-   0 |    
-  50 |    
- 100 |    
- 150 |    
- 200 |    
- 250 |    
- 300 | 300
- 350 |    
- 400 |    
-(9 rows)
+  a   |  b  
+------+-----
+ -250 |    
+ -200 |    
+ -150 |    
+ -100 |    
+  -50 |    
+    0 |    
+   50 |    
+  100 |    
+  150 |    
+  200 |    
+  250 |    
+  300 | 300
+  350 |    
+  400 |    
+(14 rows)
 
 -- merge join when expression with whole-row reference needs to be sorted;
 -- partitionwise join does not apply
@@ -1032,30 +1573,797 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
          Sort Key: t1.a, ((((t1.*)::prt1))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
    ->  Sort
          Sort Key: t2.b, ((((t2.*)::prt2))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt2_p1 t2
-                     ->  Seq Scan on prt2_p2 t2_1
-                     ->  Seq Scan on prt2_p3 t2_2
-(16 rows)
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+(20 rows)
 
 SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.b ORDER BY t1.a;
- a  | b  
-----+----
-  0 |  0
-  6 |  6
- 12 | 12
- 18 | 18
- 24 | 24
-(5 rows)
+  a  |  b  
+-----+-----
+ -24 | -24
+ -18 | -18
+ -12 | -12
+  -6 |  -6
+   0 |   0
+   6 |   6
+  12 |  12
+  18 |  18
+  24 |  24
+(9 rows)
 
 RESET enable_hashjoin;
 RESET enable_nestloop;
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p4 t2_3
+               ->  Seq Scan on prt2_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_4
+                           Filter: (b = 0)
+(22 rows)
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p3 t1_2
+                           Filter: (b = 0)
+(21 rows)
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -150 | -0150 | 0 | -0150
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+(7 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 |   | 
+ -200 | -0200 |   | 
+ -150 | -0150 | 0 | -0150
+ -100 | -0100 |   | 
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+(21 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -200 | 0 | -0200
+ -100 | 0 | -0100
+  -50 | 0 | -0050
+   50 | 0 | 0050
+  100 | 0 | 0100
+  200 | 0 | 0200
+  250 | 0 | 0250
+  350 | 0 | 0350
+  400 | 0 | 0400
+  500 | 0 | 0500
+  550 | 0 | 0550
+  650 | 0 | 0650
+  700 | 0 | 0700
+(14 rows)
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+                             QUERY PLAN                              
+---------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t3.c
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p0_a on prt1_p0 t3
+                           Index Cond: (a = t2.b)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t2.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_1.a = t2_1.b)
+                           ->  Seq Scan on prt1_p1 t3_1
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1 t2_1
+                                       Filter: (a = 0)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p2_a on prt1_p2 t3_2
+                           Index Cond: (a = t2_2.b)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t3_3
+                           Index Cond: (a = t2_3.b)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_4.a = t2_4.b)
+                           ->  Seq Scan on prt1_p4 t3_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t2_4
+                                       Filter: (a = 0)
+(47 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+  a   | a |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.b
+   ->  Hash Right Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p5 t2_5
+                           Filter: (a = 0)
+(24 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: (t1.a = t2.b)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.a, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
+                     Filter: (a = 0)
+(22 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
 --
 -- partitioned by multiple columns
 --
@@ -1125,82 +2433,1552 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 -- test partition matching with N-way join
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                   QUERY PLAN                                   
---------------------------------------------------------------------------------
+                                           QUERY PLAN                                           
+------------------------------------------------------------------------------------------------
  GroupAggregate
    Group Key: t1.c, t2.c, t3.c
    ->  Sort
          Sort Key: t1.c, t3.c
          ->  Append
                ->  Hash Join
-                     Hash Cond: (t1.c = ltrim(t3.c, 'A'::text))
+                     Hash Cond: ((t1.c)::text = ltrim(t3.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1.b = t2.b) AND (t1.c = t2.c))
-                           ->  Seq Scan on plt1_p1 t1
+                           Hash Cond: ((t2.b = t1.b) AND ((t2.c)::text = (t1.c)::text))
+                           ->  Seq Scan on plt2_p4 t2
                            ->  Hash
-                                 ->  Seq Scan on plt2_p1 t2
+                                 ->  Seq Scan on plt1_p4 t1
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p1 t3
+                           ->  Seq Scan on plt1_e_p4 t3
                ->  Hash Join
-                     Hash Cond: (t1_1.c = ltrim(t3_1.c, 'A'::text))
+                     Hash Cond: ((t1_1.c)::text = ltrim(t3_1.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
-                           ->  Seq Scan on plt1_p2 t1_1
+                           Hash Cond: ((t1_1.b = t2_1.b) AND ((t1_1.c)::text = (t2_1.c)::text))
+                           ->  Seq Scan on plt1_p1 t1_1
                            ->  Hash
-                                 ->  Seq Scan on plt2_p2 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p2 t3_1
+                           ->  Seq Scan on plt1_e_p1 t3_1
                ->  Hash Join
-                     Hash Cond: (t1_2.c = ltrim(t3_2.c, 'A'::text))
+                     Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
-                           ->  Seq Scan on plt1_p3 t1_2
+                           Hash Cond: ((t1_2.b = t2_2.b) AND ((t1_2.c)::text = (t2_2.c)::text))
+                           ->  Seq Scan on plt1_p2 t1_2
                            ->  Hash
-                                 ->  Seq Scan on plt2_p3 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p3 t3_2
-(32 rows)
+                           ->  Seq Scan on plt1_e_p2 t3_2
+               ->  Hash Join
+                     Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2_3.b = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text))
+                           ->  Seq Scan on plt2_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p3 t3_3
+(41 rows)
 
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  75.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 123.0000000000000000 | 123.0000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 225.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 273.0000000000000000 | 273.0000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 375.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 423.0000000000000000 | 423.0000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 525.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 573.0000000000000000 | 573.0000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p5 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_3
+                                 ->  Seq Scan on plt2_p3 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Seq Scan on plt2_p2 t2_2
+                                 ->  Seq Scan on plt2_p3 t2_3
+(22 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1225,22 +4003,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
 --------------------------------------------------
  Hash Left Join
    Hash Cond: (t2.b = a)
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t3.a = t2.b)
-               ->  Seq Scan on prt1_p1 t3
-               ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
-         ->  Hash Join
-               Hash Cond: (t3_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t3_1
-               ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
-         ->  Hash Join
-               Hash Cond: (t3_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t3_2
-               ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
+   ->  Hash Join
+         Hash Cond: (t3.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t3
+               ->  Seq Scan on prt1_p1 t3_1
+               ->  Seq Scan on prt1_p2 t3_2
+               ->  Seq Scan on prt1_p3 t3_3
+               ->  Seq Scan on prt1_p4 t3_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
    ->  Hash
          ->  Result
                One-Time Filter: false
@@ -1255,16 +4033,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 --
 -- tests for hash partitioned tables.
@@ -1340,41 +4124,9 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  273.0000000000000000 | 273.0000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
 (6 rows)
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
- Sort
-   Sort Key: t1.a
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
-               ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
-               ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
-                           Filter: (b = 0)
-(21 rows)
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
@@ -1389,26 +4141,24 @@ SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c
    Sort Key: t1.c
    ->  HashAggregate
          Group Key: t1.c, t2.c
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t2.c = t1.c)
-                     ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p1 t1
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on plt1_p4 t1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_1.c = t1_1.c)
-                     ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_p2 t1_1
+                           ->  Seq Scan on plt1_p1 t1_1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_2.c = t1_2.c)
-                     ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Seq Scan on plt1_p2 t1_2
                                  Filter: ((a % 25) = 0)
-(23 rows)
+                           ->  Seq Scan on plt1_p3 t1_3
+                                 Filter: ((a % 25) = 0)
+(21 rows)
 
 --
 -- multiple levels of partitioning
@@ -1804,64 +4554,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
  Hash Join
    Hash Cond: (t1.a = t2.a)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt4_n_p1 t2
                ->  Seq Scan on prt4_n_p2 t2_1
                ->  Seq Scan on prt4_n_p3 t2_2
-(11 rows)
+(13 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a and t1.a = t3.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                        QUERY PLAN                        
+----------------------------------------------------------
  Hash Join
-   Hash Cond: (t2.a = t1.a)
+   Hash Cond: (t3.b = t1.a)
    ->  Append
-         ->  Seq Scan on prt4_n_p1 t2
-         ->  Seq Scan on prt4_n_p2 t2_1
-         ->  Seq Scan on prt4_n_p3 t2_2
+         ->  Seq Scan on prt2_p0 t3
+         ->  Seq Scan on prt2_p1 t3_1
+         ->  Seq Scan on prt2_p2 t3_2
+         ->  Seq Scan on prt2_p3 t3_3
+         ->  Seq Scan on prt2_p4 t3_4
+         ->  Seq Scan on prt2_p5 t3_5
    ->  Hash
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.a = t3_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_2.b)
-                     ->  Seq Scan on prt1_p3 t1_2
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3 t3_2
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt4_n_p1 t2
+                           ->  Seq Scan on prt4_n_p2 t2_1
+                           ->  Seq Scan on prt4_n_p3 t2_2
 (23 rows)
 
 -- partitionwise join can not be applied if there are no equi-join conditions
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (b > t1.a)
-(12 rows)
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+(16 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partitionwise join
@@ -1947,16 +4703,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
-                     ->  Seq Scan on plt1_p1 t3
-                     ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3
+                     ->  Seq Scan on plt1_p1 t3_1
+                     ->  Seq Scan on plt1_p2 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partitionwise join can not be applied for a join between list and range
 -- partitioned tables
@@ -1967,14 +4724,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
  Hash Full Join
    Hash Cond: ((t2.c)::text = (t1.c)::text)
    ->  Append
-         ->  Seq Scan on prt1_p1 t2
-         ->  Seq Scan on prt1_p2 t2_1
-         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p0 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p2 t2_2
+         ->  Seq Scan on prt1_p3 t2_3
+         ->  Seq Scan on prt1_p4 t2_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt1_n_p1 t1
                ->  Seq Scan on prt1_n_p2 t1_1
-(10 rows)
+(12 rows)
 
 -- partitionwise join can not be applied if only one of joining tables has
 -- default partition
@@ -1990,16 +4749,202 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
    ->  Hash Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
+ANALYZE plt1;
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+ANALYZE plt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.c = t2.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt1_p1 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t2_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_1
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+  0 | 0000 |    | 
+  5 | 0000 |    | 
+ 10 | 0000 |    | 
+ 15 | 0000 |    | 
+ 20 | 0000 |    | 
+  2 | 0002 |  2 | 0002
+  2 | 0002 |  7 | 0002
+  2 | 0002 | 12 | 0002
+  2 | 0002 | 17 | 0002
+  2 | 0002 | 22 | 0002
+  7 | 0002 |  2 | 0002
+  7 | 0002 |  7 | 0002
+  7 | 0002 | 12 | 0002
+  7 | 0002 | 17 | 0002
+  7 | 0002 | 22 | 0002
+ 12 | 0002 |  2 | 0002
+ 12 | 0002 |  7 | 0002
+ 12 | 0002 | 12 | 0002
+ 12 | 0002 | 17 | 0002
+ 12 | 0002 | 22 | 0002
+ 17 | 0002 |  2 | 0002
+ 17 | 0002 |  7 | 0002
+ 17 | 0002 | 12 | 0002
+ 17 | 0002 | 17 | 0002
+ 17 | 0002 | 22 | 0002
+ 22 | 0002 |  2 | 0002
+ 22 | 0002 |  7 | 0002
+ 22 | 0002 | 12 | 0002
+ 22 | 0002 | 17 | 0002
+ 22 | 0002 | 22 | 0002
+    |      |  1 | 0001
+    |      |  6 | 0001
+    |      | 11 | 0001
+    |      | 16 | 0001
+    |      | 21 | 0001
+(35 rows)
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1;
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2;
+CREATE TABLE plt3 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt3_p1 PARTITION OF plt3 FOR VALUES IN ('0001');
+CREATE TABLE plt3_p2 PARTITION OF plt3 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2.a, t3.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.c = t3.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1.c = t2.c)
+                     ->  Seq Scan on plt1_p1 t1
+                     ->  Hash
+                           ->  Seq Scan on plt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on plt3_p1 t3
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t3_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_1.c = t2_1.c)
+                     ->  Seq Scan on plt1_p2 t1_1
+                     ->  Hash
+                           ->  Seq Scan on plt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt3_p2 t3_1
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+ a  |  c   | a  |  c   | a  |  c   
+----+------+----+------+----+------
+  0 | 0000 |    |      |    | 
+  5 | 0000 |    |      |    | 
+ 10 | 0000 |    |      |    | 
+ 15 | 0000 |    |      |    | 
+ 20 | 0000 |    |      |    | 
+  1 | 0001 |    |      |  1 | 0001
+  1 | 0001 |    |      |  6 | 0001
+  1 | 0001 |    |      | 11 | 0001
+  1 | 0001 |    |      | 16 | 0001
+  1 | 0001 |    |      | 21 | 0001
+  6 | 0001 |    |      |  1 | 0001
+  6 | 0001 |    |      |  6 | 0001
+  6 | 0001 |    |      | 11 | 0001
+  6 | 0001 |    |      | 16 | 0001
+  6 | 0001 |    |      | 21 | 0001
+ 11 | 0001 |    |      |  1 | 0001
+ 11 | 0001 |    |      |  6 | 0001
+ 11 | 0001 |    |      | 11 | 0001
+ 11 | 0001 |    |      | 16 | 0001
+ 11 | 0001 |    |      | 21 | 0001
+ 16 | 0001 |    |      |  1 | 0001
+ 16 | 0001 |    |      |  6 | 0001
+ 16 | 0001 |    |      | 11 | 0001
+ 16 | 0001 |    |      | 16 | 0001
+ 16 | 0001 |    |      | 21 | 0001
+ 21 | 0001 |    |      |  1 | 0001
+ 21 | 0001 |    |      |  6 | 0001
+ 21 | 0001 |    |      | 11 | 0001
+ 21 | 0001 |    |      | 16 | 0001
+ 21 | 0001 |    |      | 21 | 0001
+  2 | 0002 |  2 | 0002 |    | 
+  2 | 0002 |  7 | 0002 |    | 
+  2 | 0002 | 12 | 0002 |    | 
+  2 | 0002 | 17 | 0002 |    | 
+  2 | 0002 | 22 | 0002 |    | 
+  7 | 0002 |  2 | 0002 |    | 
+  7 | 0002 |  7 | 0002 |    | 
+  7 | 0002 | 12 | 0002 |    | 
+  7 | 0002 | 17 | 0002 |    | 
+  7 | 0002 | 22 | 0002 |    | 
+ 12 | 0002 |  2 | 0002 |    | 
+ 12 | 0002 |  7 | 0002 |    | 
+ 12 | 0002 | 12 | 0002 |    | 
+ 12 | 0002 | 17 | 0002 |    | 
+ 12 | 0002 | 22 | 0002 |    | 
+ 17 | 0002 |  2 | 0002 |    | 
+ 17 | 0002 |  7 | 0002 |    | 
+ 17 | 0002 | 12 | 0002 |    | 
+ 17 | 0002 | 17 | 0002 |    | 
+ 17 | 0002 | 22 | 0002 |    | 
+ 22 | 0002 |  2 | 0002 |    | 
+ 22 | 0002 |  7 | 0002 |    | 
+ 22 | 0002 | 12 | 0002 |    | 
+ 22 | 0002 | 17 | 0002 |    | 
+ 22 | 0002 | 22 | 0002 |    | 
+(55 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index fb3ba18a26f..b0ab002eeec 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -10,25 +10,39 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -69,11 +83,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -95,20 +117,30 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 
 EXPLAIN (COSTS OFF)
@@ -172,6 +204,128 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -196,28 +350,79 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
 -- test partition matching with N-way join
@@ -225,6 +430,175 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
@@ -270,27 +644,18 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
 ALTER TABLE plt2 DETACH PARTITION plt2_p3;
 ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
 ANALYZE plt2;
-
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.a % 25 = 0 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
 --
 -- multiple levels of partitioning
 --
@@ -435,3 +800,47 @@ ANALYZE prt2;
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
+ANALYZE plt1;
+
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+ANALYZE plt2;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1;
+
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2;
+
+CREATE TABLE plt3 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt3_p1 PARTITION OF plt3 FOR VALUES IN ('0001');
+CREATE TABLE plt3_p2 PARTITION OF plt3 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
-- 
2.18.0

0002-Few-more-tests-and-adjustments.patchapplication/octet-stream; name=0002-Few-more-tests-and-adjustments.patchDownload
From 39777663e4c4156042ff9d51483c33e7663d2ab9 Mon Sep 17 00:00:00 2001
From: Amul Sul <amul.sul@enterprisedb.com>
Date: Tue, 5 Nov 2019 07:26:18 -0500
Subject: [PATCH 2/2] Few more tests and adjustments

---
 src/test/regress/expected/partition_join.out | 209 +++++++++++++++----
 src/test/regress/sql/partition_join.sql      |  49 +++--
 2 files changed, 199 insertions(+), 59 deletions(-)

diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 89f0c6a9ee7..31e4aa60280 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -1607,15 +1607,11 @@ RESET enable_hashjoin;
 RESET enable_nestloop;
 -- test default partition behavior for range, partition-wise join is not
 -- possible since more than one partition on one side matches default partition
--- on the other side. Default partition from prt1 matches default partition and
--- prt2_p4 from prt2 and default partition from prt2 matches default partition
--- and prt1_p0 from prt1
+-- on the other side. Default partition from prt1 matches prt2_p3 and
+-- prt2_p4 partition from prt2.
 ALTER TABLE prt1 DETACH PARTITION prt1_p3;
 ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
 ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
                     QUERY PLAN                    
@@ -1628,8 +1624,8 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
                ->  Seq Scan on prt2_p0 t2
                ->  Seq Scan on prt2_p1 t2_1
                ->  Seq Scan on prt2_p2 t2_2
-               ->  Seq Scan on prt2_p4 t2_3
-               ->  Seq Scan on prt2_p3 t2_4
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
          ->  Hash
                ->  Append
                      ->  Seq Scan on prt1_p0 t1
@@ -1644,51 +1640,46 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
                            Filter: (b = 0)
 (22 rows)
 
--- partition-wise join should be possible when we drop the first and last
--- partitions from both sides
-ALTER TABLE prt1 DETACH PARTITION prt1_p0;
-ALTER TABLE prt1 DETACH PARTITION prt1_p4;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+-- partition-wise join should be possible when we drop the prt2_p4 from prt2.
 ALTER TABLE prt2 DETACH PARTITION prt2_p4;
 ANALYZE prt2;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+(26 rows)
 
 -- restore the partitioned tables for rest of the tests
-ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
-ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
 ALTER TABLE prt1 DETACH PARTITION prt1_p3;
 ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
 ANALYZE prt1;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
 ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
 ANALYZE prt2;
 -- Add an extra partition to prt2 , Partition-wise join is possible with
 -- extra partitions on inner side are allowed
@@ -4774,12 +4765,14 @@ DROP TABLE plt2;
 CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
 CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0002');
 CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
-INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007');
+INSERT INTO plt1 SELECT i, i, to_char(i % 8, 'FM0000') FROM generate_series(0, 24) i WHERE i % 8 IN (0, 2, 3, 4, 6, 7);
 ANALYZE plt1;
 CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
 CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0001', '0002');
 CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
-INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0005', '0007');
+INSERT INTO plt2 SELECT i, i, to_char(i % 8, 'FM0000') FROM generate_series(0, 24) i WHERE i % 8 IN (1, 2, 3, 4, 5, 7);
 ANALYZE plt2;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
@@ -4800,16 +4793,91 @@ SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WH
                ->  Seq Scan on plt1_p2 t1_1
                ->  Hash
                      ->  Seq Scan on plt2_p2 t2_1
-(15 rows)
+         ->  Hash Full Join
+               Hash Cond: (t1_2.c = t2_2.c)
+               Filter: (((COALESCE(t1_2.a, 0) % 5) <> 3) AND ((COALESCE(t1_2.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt1_p3 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_2
+(21 rows)
 
 SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
  a  |  c   | a  |  c   
 ----+------+----+------
   0 | 0000 |    | 
-  5 | 0000 |    | 
- 10 | 0000 |    | 
- 15 | 0000 |    | 
- 20 | 0000 |    | 
+ 16 | 0000 |    | 
+  2 | 0002 |  2 | 0002
+  2 | 0002 | 10 | 0002
+  2 | 0002 | 18 | 0002
+ 10 | 0002 |  2 | 0002
+ 10 | 0002 | 10 | 0002
+ 10 | 0002 | 18 | 0002
+ 11 | 0003 |  3 | 0003
+ 11 | 0003 | 11 | 0003
+ 11 | 0003 | 19 | 0003
+ 12 | 0004 |  4 | 0004
+ 12 | 0004 | 12 | 0004
+ 12 | 0004 | 20 | 0004
+ 20 | 0004 |  4 | 0004
+ 20 | 0004 | 12 | 0004
+ 20 | 0004 | 20 | 0004
+  6 | 0006 |    | 
+ 22 | 0006 |    | 
+  7 | 0007 |  7 | 0007
+  7 | 0007 | 15 | 0007
+  7 | 0007 | 23 | 0007
+ 15 | 0007 |  7 | 0007
+ 15 | 0007 | 15 | 0007
+ 15 | 0007 | 23 | 0007
+    |      |  1 | 0001
+    |      |  5 | 0005
+    |      |  9 | 0001
+    |      | 13 | 0005
+    |      | 17 | 0001
+    |      | 21 | 0005
+(31 rows)
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN (NULL, '0001', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+ANALYZE plt1;
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 DEFAULT;
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
+ANALYZE plt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2_1.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.c = t2_1.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt1_p1 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t2.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+  1 | 0001 |    | 
+  6 | 0001 |    | 
+ 11 | 0001 |    | 
+ 16 | 0001 |    | 
+ 21 | 0001 |    | 
   2 | 0002 |  2 | 0002
   2 | 0002 |  7 | 0002
   2 | 0002 | 12 | 0002
@@ -4835,11 +4903,72 @@ SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WH
  22 | 0002 | 12 | 0002
  22 | 0002 | 17 | 0002
  22 | 0002 | 22 | 0002
-    |      |  1 | 0001
-    |      |  6 | 0001
-    |      | 11 | 0001
-    |      | 16 | 0001
-    |      | 21 | 0001
+    |      |  0 | 0000
+    |      |  5 | 0000
+    |      | 10 | 0000
+    |      | 15 | 0000
+    |      | 20 | 0000
+(35 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2 t2 FULL JOIN plt1 t1 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2_1.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t2_1.c = t1.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1
+         ->  Hash Full Join
+               Hash Cond: (t2.c = t1_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt2_p2 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_1
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2 t2 FULL JOIN plt1 t1 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+  1 | 0001 |    | 
+  6 | 0001 |    | 
+ 11 | 0001 |    | 
+ 16 | 0001 |    | 
+ 21 | 0001 |    | 
+  2 | 0002 |  2 | 0002
+  2 | 0002 |  7 | 0002
+  2 | 0002 | 12 | 0002
+  2 | 0002 | 17 | 0002
+  2 | 0002 | 22 | 0002
+  7 | 0002 |  2 | 0002
+  7 | 0002 |  7 | 0002
+  7 | 0002 | 12 | 0002
+  7 | 0002 | 17 | 0002
+  7 | 0002 | 22 | 0002
+ 12 | 0002 |  2 | 0002
+ 12 | 0002 |  7 | 0002
+ 12 | 0002 | 12 | 0002
+ 12 | 0002 | 17 | 0002
+ 12 | 0002 | 22 | 0002
+ 17 | 0002 |  2 | 0002
+ 17 | 0002 |  7 | 0002
+ 17 | 0002 | 12 | 0002
+ 17 | 0002 | 17 | 0002
+ 17 | 0002 | 22 | 0002
+ 22 | 0002 |  2 | 0002
+ 22 | 0002 |  7 | 0002
+ 22 | 0002 | 12 | 0002
+ 22 | 0002 | 17 | 0002
+ 22 | 0002 | 22 | 0002
+    |      |  0 | 0000
+    |      |  5 | 0000
+    |      | 10 | 0000
+    |      | 15 | 0000
+    |      | 20 | 0000
 (35 rows)
 
 DROP TABLE plt1;
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index b0ab002eeec..df3fdbeef49 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -206,39 +206,25 @@ RESET enable_nestloop;
 
 -- test default partition behavior for range, partition-wise join is not
 -- possible since more than one partition on one side matches default partition
--- on the other side. Default partition from prt1 matches default partition and
--- prt2_p4 from prt2 and default partition from prt2 matches default partition
--- and prt1_p0 from prt1
+-- on the other side. Default partition from prt1 matches prt2_p3 and
+-- prt2_p4 partition from prt2.
 ALTER TABLE prt1 DETACH PARTITION prt1_p3;
 ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
 ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
 
--- partition-wise join should be possible when we drop the first and last
--- partitions from both sides
-ALTER TABLE prt1 DETACH PARTITION prt1_p0;
-ALTER TABLE prt1 DETACH PARTITION prt1_p4;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+-- partition-wise join should be possible when we drop the prt2_p4 from prt2.
 ALTER TABLE prt2 DETACH PARTITION prt2_p4;
 ANALYZE prt2;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
 
 -- restore the partitioned tables for rest of the tests
-ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
-ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
 ALTER TABLE prt1 DETACH PARTITION prt1_p3;
 ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
 ANALYZE prt1;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
 ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
 ANALYZE prt2;
 
 -- Add an extra partition to prt2 , Partition-wise join is possible with
@@ -807,19 +793,44 @@ DROP TABLE plt2;
 CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
 CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0002');
 CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
-INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007');
+INSERT INTO plt1 SELECT i, i, to_char(i % 8, 'FM0000') FROM generate_series(0, 24) i WHERE i % 8 IN (0, 2, 3, 4, 6, 7);
 ANALYZE plt1;
 
 CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
 CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0001', '0002');
 CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
-INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0005', '0007');
+INSERT INTO plt2 SELECT i, i, to_char(i % 8, 'FM0000') FROM generate_series(0, 24) i WHERE i % 8 IN (1, 2, 3, 4, 5, 7);
+ANALYZE plt2;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN (NULL, '0001', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+ANALYZE plt1;
+
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 DEFAULT;
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
 ANALYZE plt2;
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
 SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2 t2 FULL JOIN plt1 t1 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2 t2 FULL JOIN plt1 t1 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+
 DROP TABLE plt1;
 DROP TABLE plt2;
 
-- 
2.18.0

enable-coverage-without-002-patch.outapplication/octet-stream; name=enable-coverage-without-002-patch.outDownload
enable-coverage-with-002-patch.outapplication/octet-stream; name=enable-coverage-with-002-patch.outDownload
#103amul sul
sulamul@gmail.com
In reply to: amul sul (#102)
2 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Attached is the rebase atop the latest master head(a9056cc637f).

On Tue, Nov 5, 2019 at 6:44 PM amul sul <sulamul@gmail.com> wrote:

Show quoted text

On Fri, Nov 1, 2019 at 3:58 PM Etsuro Fujita <etsuro.fujita@gmail.com>
wrote:

On Thu, Oct 31, 2019 at 6:49 PM Etsuro Fujita <etsuro.fujita@gmail.com>
wrote:

Attached is an updated version. If no objections, I'll merge this
with the main patch [1].

Done. Attached is a new version of the patch.

Other changes: in generate_matching_part_pairs(), I changed variable
names to match other functions, simplified assertions, and
adjusted/added comments a bit.

Thanks for the update version.

A query and comments for v25:

583 + * The function returns NULL if we can not find the matching pair of
584 + * partitions. This happens if 1. multiple partitions on one side
match with
585 + * one partition on the other side. 2. a given partition on the
outer side
586 + * doesn't have a matching partition on the inner side. We can not
support the
587 + * first case since we don't have a way to represent multiple
partitions as a
588 + * single relation (RelOptInfo) and then perform join using the
ganged
589 + * relation. We can not support the second case since the missing
inner
590 + * partition needs to be represented as an empty relation and we
don't have a
591 + * way to introduce empty relation during join planning after
creating paths
592 + * for all the base relations.
593 + */
594 +PartitionBoundInfo
595 +partition_bounds_merge(int partnatts,

I think the second condition mention for partition_bounds_merge() looks
outdated, do you think we should remove that or am I missing something
here?
---

1768 +
1769 + /*
1770 + * If this is an outer join, the merged partition would act
as the
1771 + * default partition of the join; record the index in
*default_index
1772 + * if not done yet.
1773 + */
1774 + if (jointype == JOIN_LEFT || jointype == JOIN_ANTI ||
1775 + jointype == JOIN_FULL)
1776 + {

As decided need to replace this list by IS_OUTER_JOIN(jointype).
---

2020 + if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
2021 + jointype == JOIN_ANTI)
2022 + {

Same as the previous.
---

I tried a coverage testing and tried to adjust & add a few tests to
improved the
code coverage for the v25 patch. Please have a look at the attached 0002 &
also
attach the coverage output with & without this patch, TWIMW.

0001 is the same v25 patch, reattaching to make CFbot happy.

Regards,
Amul

Attachments:

0002-Few-more-tests-and-adjustments-v2.patchapplication/octet-stream; name=0002-Few-more-tests-and-adjustments-v2.patchDownload
From 22b1ff214ccec331d1e5d512ccd7e647db43f3b6 Mon Sep 17 00:00:00 2001
From: Amul Sul <amul.sul@enterprisedb.com>
Date: Tue, 5 Nov 2019 07:26:18 -0500
Subject: [PATCH 2/2] Few more tests and adjustments v2

---
 src/test/regress/expected/partition_join.out | 209 +++++++++++++++----
 src/test/regress/sql/partition_join.sql      |  49 +++--
 2 files changed, 199 insertions(+), 59 deletions(-)

diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 618862ce87f..fa9e0872cbf 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -1710,15 +1710,11 @@ RESET enable_hashjoin;
 RESET enable_nestloop;
 -- test default partition behavior for range, partition-wise join is not
 -- possible since more than one partition on one side matches default partition
--- on the other side. Default partition from prt1 matches default partition and
--- prt2_p4 from prt2 and default partition from prt2 matches default partition
--- and prt1_p0 from prt1
+-- on the other side. Default partition from prt1 matches prt2_p3 and
+-- prt2_p4 partition from prt2.
 ALTER TABLE prt1 DETACH PARTITION prt1_p3;
 ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
 ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
                     QUERY PLAN                    
@@ -1731,8 +1727,8 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
                ->  Seq Scan on prt2_p0 t2
                ->  Seq Scan on prt2_p1 t2_1
                ->  Seq Scan on prt2_p2 t2_2
-               ->  Seq Scan on prt2_p4 t2_3
-               ->  Seq Scan on prt2_p3 t2_4
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
          ->  Hash
                ->  Append
                      ->  Seq Scan on prt1_p0 t1
@@ -1747,51 +1743,46 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
                            Filter: (b = 0)
 (22 rows)
 
--- partition-wise join should be possible when we drop the first and last
--- partitions from both sides
-ALTER TABLE prt1 DETACH PARTITION prt1_p0;
-ALTER TABLE prt1 DETACH PARTITION prt1_p4;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+-- partition-wise join should be possible when we drop the prt2_p4 from prt2.
 ALTER TABLE prt2 DETACH PARTITION prt2_p4;
 ANALYZE prt2;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+(26 rows)
 
 -- restore the partitioned tables for rest of the tests
-ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
-ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
 ALTER TABLE prt1 DETACH PARTITION prt1_p3;
 ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
 ANALYZE prt1;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
 ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
 ANALYZE prt2;
 -- Add an extra partition to prt2 , Partition-wise join is possible with
 -- extra partitions on inner side are allowed
@@ -4877,12 +4868,14 @@ DROP TABLE plt2;
 CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
 CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0002');
 CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
-INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007');
+INSERT INTO plt1 SELECT i, i, to_char(i % 8, 'FM0000') FROM generate_series(0, 24) i WHERE i % 8 IN (0, 2, 3, 4, 6, 7);
 ANALYZE plt1;
 CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
 CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0001', '0002');
 CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
-INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0005', '0007');
+INSERT INTO plt2 SELECT i, i, to_char(i % 8, 'FM0000') FROM generate_series(0, 24) i WHERE i % 8 IN (1, 2, 3, 4, 5, 7);
 ANALYZE plt2;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
@@ -4903,16 +4896,91 @@ SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WH
                ->  Seq Scan on plt1_p2 t1_1
                ->  Hash
                      ->  Seq Scan on plt2_p2 t2_1
-(15 rows)
+         ->  Hash Full Join
+               Hash Cond: (t1_2.c = t2_2.c)
+               Filter: (((COALESCE(t1_2.a, 0) % 5) <> 3) AND ((COALESCE(t1_2.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt1_p3 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_2
+(21 rows)
 
 SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
  a  |  c   | a  |  c   
 ----+------+----+------
   0 | 0000 |    | 
-  5 | 0000 |    | 
- 10 | 0000 |    | 
- 15 | 0000 |    | 
- 20 | 0000 |    | 
+ 16 | 0000 |    | 
+  2 | 0002 |  2 | 0002
+  2 | 0002 | 10 | 0002
+  2 | 0002 | 18 | 0002
+ 10 | 0002 |  2 | 0002
+ 10 | 0002 | 10 | 0002
+ 10 | 0002 | 18 | 0002
+ 11 | 0003 |  3 | 0003
+ 11 | 0003 | 11 | 0003
+ 11 | 0003 | 19 | 0003
+ 12 | 0004 |  4 | 0004
+ 12 | 0004 | 12 | 0004
+ 12 | 0004 | 20 | 0004
+ 20 | 0004 |  4 | 0004
+ 20 | 0004 | 12 | 0004
+ 20 | 0004 | 20 | 0004
+  6 | 0006 |    | 
+ 22 | 0006 |    | 
+  7 | 0007 |  7 | 0007
+  7 | 0007 | 15 | 0007
+  7 | 0007 | 23 | 0007
+ 15 | 0007 |  7 | 0007
+ 15 | 0007 | 15 | 0007
+ 15 | 0007 | 23 | 0007
+    |      |  1 | 0001
+    |      |  5 | 0005
+    |      |  9 | 0001
+    |      | 13 | 0005
+    |      | 17 | 0001
+    |      | 21 | 0005
+(31 rows)
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN (NULL, '0001', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+ANALYZE plt1;
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 DEFAULT;
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
+ANALYZE plt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2_1.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.c = t2_1.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt1_p1 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t2.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+  1 | 0001 |    | 
+  6 | 0001 |    | 
+ 11 | 0001 |    | 
+ 16 | 0001 |    | 
+ 21 | 0001 |    | 
   2 | 0002 |  2 | 0002
   2 | 0002 |  7 | 0002
   2 | 0002 | 12 | 0002
@@ -4938,11 +5006,72 @@ SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WH
  22 | 0002 | 12 | 0002
  22 | 0002 | 17 | 0002
  22 | 0002 | 22 | 0002
-    |      |  1 | 0001
-    |      |  6 | 0001
-    |      | 11 | 0001
-    |      | 16 | 0001
-    |      | 21 | 0001
+    |      |  0 | 0000
+    |      |  5 | 0000
+    |      | 10 | 0000
+    |      | 15 | 0000
+    |      | 20 | 0000
+(35 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2 t2 FULL JOIN plt1 t1 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2_1.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t2_1.c = t1.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1
+         ->  Hash Full Join
+               Hash Cond: (t2.c = t1_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt2_p2 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_1
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2 t2 FULL JOIN plt1 t1 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+  1 | 0001 |    | 
+  6 | 0001 |    | 
+ 11 | 0001 |    | 
+ 16 | 0001 |    | 
+ 21 | 0001 |    | 
+  2 | 0002 |  2 | 0002
+  2 | 0002 |  7 | 0002
+  2 | 0002 | 12 | 0002
+  2 | 0002 | 17 | 0002
+  2 | 0002 | 22 | 0002
+  7 | 0002 |  2 | 0002
+  7 | 0002 |  7 | 0002
+  7 | 0002 | 12 | 0002
+  7 | 0002 | 17 | 0002
+  7 | 0002 | 22 | 0002
+ 12 | 0002 |  2 | 0002
+ 12 | 0002 |  7 | 0002
+ 12 | 0002 | 12 | 0002
+ 12 | 0002 | 17 | 0002
+ 12 | 0002 | 22 | 0002
+ 17 | 0002 |  2 | 0002
+ 17 | 0002 |  7 | 0002
+ 17 | 0002 | 12 | 0002
+ 17 | 0002 | 17 | 0002
+ 17 | 0002 | 22 | 0002
+ 22 | 0002 |  2 | 0002
+ 22 | 0002 |  7 | 0002
+ 22 | 0002 | 12 | 0002
+ 22 | 0002 | 17 | 0002
+ 22 | 0002 | 22 | 0002
+    |      |  0 | 0000
+    |      |  5 | 0000
+    |      | 10 | 0000
+    |      | 15 | 0000
+    |      | 20 | 0000
 (35 rows)
 
 DROP TABLE plt1;
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index b7f53388d22..a54ef5ccada 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -221,39 +221,25 @@ RESET enable_nestloop;
 
 -- test default partition behavior for range, partition-wise join is not
 -- possible since more than one partition on one side matches default partition
--- on the other side. Default partition from prt1 matches default partition and
--- prt2_p4 from prt2 and default partition from prt2 matches default partition
--- and prt1_p0 from prt1
+-- on the other side. Default partition from prt1 matches prt2_p3 and
+-- prt2_p4 partition from prt2.
 ALTER TABLE prt1 DETACH PARTITION prt1_p3;
 ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
 ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
 
--- partition-wise join should be possible when we drop the first and last
--- partitions from both sides
-ALTER TABLE prt1 DETACH PARTITION prt1_p0;
-ALTER TABLE prt1 DETACH PARTITION prt1_p4;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+-- partition-wise join should be possible when we drop the prt2_p4 from prt2.
 ALTER TABLE prt2 DETACH PARTITION prt2_p4;
 ANALYZE prt2;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
 
 -- restore the partitioned tables for rest of the tests
-ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
-ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
 ALTER TABLE prt1 DETACH PARTITION prt1_p3;
 ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
 ANALYZE prt1;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
 ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
 ANALYZE prt2;
 
 -- Add an extra partition to prt2 , Partition-wise join is possible with
@@ -822,19 +808,44 @@ DROP TABLE plt2;
 CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
 CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0002');
 CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
-INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007');
+INSERT INTO plt1 SELECT i, i, to_char(i % 8, 'FM0000') FROM generate_series(0, 24) i WHERE i % 8 IN (0, 2, 3, 4, 6, 7);
 ANALYZE plt1;
 
 CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
 CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0001', '0002');
 CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
-INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0005', '0007');
+INSERT INTO plt2 SELECT i, i, to_char(i % 8, 'FM0000') FROM generate_series(0, 24) i WHERE i % 8 IN (1, 2, 3, 4, 5, 7);
+ANALYZE plt2;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN (NULL, '0001', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+ANALYZE plt1;
+
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 DEFAULT;
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
 ANALYZE plt2;
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
 SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2 t2 FULL JOIN plt1 t1 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2 t2 FULL JOIN plt1 t1 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+
 DROP TABLE plt1;
 DROP TABLE plt2;
 
-- 
2.18.0

001-Improve-partition-matching-for-partitionwise-joins-v26.patchapplication/octet-stream; name=001-Improve-partition-matching-for-partitionwise-joins-v26.patchDownload
From e3aa2201d19bd3a8b3f6e618d250535ba1562962 Mon Sep 17 00:00:00 2001
From: Amul Sul <amul.sul@enterprisedb.com>
Date: Wed, 6 Nov 2019 00:53:18 -0500
Subject: [PATCH 1/2] Improve-partition-matching-for-partitionwise-joins-v26

---
 src/backend/nodes/outfuncs.c                 |    2 +
 src/backend/optimizer/path/joinrels.c        |  246 +-
 src/backend/optimizer/util/inherit.c         |    2 +
 src/backend/optimizer/util/relnode.c         |   45 +-
 src/backend/partitioning/partbounds.c        | 1691 +++++++
 src/include/nodes/pathnodes.h                |    5 +
 src/include/partitioning/partbounds.h        |    7 +
 src/test/regress/expected/partition_join.out | 4191 +++++++++++++++---
 src/test/regress/sql/partition_join.sql      |  471 +-
 9 files changed, 5967 insertions(+), 693 deletions(-)

diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index b0dcd02ff68..b8151574024 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -2281,6 +2281,8 @@ _outRelOptInfo(StringInfo str, const RelOptInfo *node)
 	WRITE_BOOL_FIELD(has_eclass_joins);
 	WRITE_BOOL_FIELD(consider_partitionwise_join);
 	WRITE_BITMAPSET_FIELD(top_parent_relids);
+	WRITE_BOOL_FIELD(merged);
+	WRITE_BITMAPSET_FIELD(all_partrels);
 	WRITE_NODE_FIELD(partitioned_child_rels);
 }
 
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 6a480ab7644..ed7bc23c7be 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -48,6 +48,9 @@ static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 												Relids left_relids, Relids right_relids);
 static int	match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel,
 										 bool strict_op);
+static void get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2);
 
 
 /*
@@ -1357,25 +1360,30 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	bool		merged = false;
+	List	   *parts1 = NIL;
+	List	   *parts2 = NIL;
+	ListCell   *lcr1 = NULL;
+	ListCell   *lcr2 = NULL;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1390,34 +1398,107 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		   joinrel->part_scheme == rel2->part_scheme);
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * If we don't have the partition bounds for the join rel yet, try to
+	 * create it along with pairs of partitions to be joined; else generate
+	 * those using the partitioning info for the join rel we already have.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	if (joinrel->nparts == -1)
+	{
+		PartitionScheme part_scheme = joinrel->part_scheme;
+		PartitionBoundInfo boundinfo = NULL;
+		int			nparts = 0;
+
+		Assert(joinrel->boundinfo == NULL);
+		Assert(joinrel->part_rels == NULL);
+
+		/*
+		 * See if the partition bounds for inputs are exactly the same, in
+		 * which case we don't need to work hard: partitions with the same
+		 * partition indexes will form join pairs, and the join rel will have
+		 * the same partition bounds as inputs; otherwise try to merge the
+		 * partition bounds along with generating join pairs.
+		 *
+		 * Even if one or both inputs have merged partition bounds, it'd be
+		 * possible for the partition bounds to be exactly the same, but it
+		 * seems unlikely to be worth the cycles to check; do this check only
+		 * if both inputs have non-merged partition bounds.
+		 */
+		if (!rel1->merged &&
+			!rel2->merged &&
+			rel1->nparts == rel2->nparts &&
+			partition_bounds_equal(part_scheme->partnatts,
+								   part_scheme->parttyplen,
+								   part_scheme->parttypbyval,
+								   rel1->boundinfo, rel2->boundinfo))
+		{
+			boundinfo = rel1->boundinfo;
+			nparts = rel1->nparts;
+		}
+		else
+		{
+			boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											   part_scheme->parttyplen,
+											   part_scheme->parttypbyval,
+											   part_scheme->partsupfunc,
+											   part_scheme->partcollation,
+											   rel1, rel2,
+											   parent_sjinfo->jointype,
+											   &parts1, &parts2);
+			if (boundinfo == NULL)
+			{
+				joinrel->nparts = 0;
+				return;
+			}
+			nparts = list_length(parts1);
+			merged = true;
+		}
+
+		Assert(nparts > 0);
+		joinrel->boundinfo = boundinfo;
+		joinrel->merged = merged;
+		joinrel->nparts = nparts;
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+	}
+	else
+	{
+		Assert(joinrel->nparts > 0);
+		Assert(joinrel->boundinfo);
+		Assert(joinrel->part_rels);
+
+		/*
+		 * If the partition bounds for the join rel are not merged ones,
+		 * inputs are guaranteed to have the same partition bounds, so
+		 * partitions with the same partition indexes will form join pairs;
+		 * else let get_matching_part_pairs() do the work.
+		 */
+		if (joinrel->merged)
+		{
+			get_matching_part_pairs(root, joinrel, rel1, rel2,
+									&parts1, &parts2);
+			Assert(list_length(parts1) == joinrel->nparts);
+			Assert(list_length(parts2) == joinrel->nparts);
+			merged = true;
+		}
+	}
 
-	nparts = joinrel->nparts;
+	if (merged)
+	{
+		lcr1 = list_head(parts1);
+		lcr2 = list_head(parts2);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
-		bool		rel1_empty = (child_rel1 == NULL ||
-								  IS_DUMMY_REL(child_rel1));
-		bool		rel2_empty = (child_rel2 == NULL ||
-								  IS_DUMMY_REL(child_rel2));
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		bool		rel1_empty;
+		bool		rel2_empty;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1425,6 +1506,22 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		if (merged)
+		{
+			child_rel1 = lfirst_node(RelOptInfo, lcr1);
+			child_rel2 = lfirst_node(RelOptInfo, lcr2);
+			lcr1 = lnext(parts1, lcr1);
+			lcr2 = lnext(parts2, lcr2);
+		}
+		else
+		{
+			child_rel1 = rel1->part_rels[cnt_parts];
+			child_rel2 = rel2->part_rels[cnt_parts];
+		}
+
+		rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1));
+		rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2));
+
 		/*
 		 * Check for cases where we can prove that this segment of the join
 		 * returns no rows, due to one or both inputs being empty (including
@@ -1522,6 +1619,8 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 												 child_sjinfo,
 												 child_sjinfo->jointype);
 			joinrel->part_rels[cnt_parts] = child_joinrel;
+			joinrel->all_partrels = bms_add_members(joinrel->all_partrels,
+													child_joinrel->relids);
 		}
 
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
@@ -1529,6 +1628,7 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
+
 	}
 }
 
@@ -1738,3 +1838,99 @@ match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op)
 
 	return -1;
 }
+
+/*
+ * get_matching_part_pairs
+ *		Generate join pairs of partitions for the two inputs
+ */
+static void
+get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2)
+{
+	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
+	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
+	int 		cnt_parts;
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
+	{
+		RelOptInfo *child_joinrel = joinrel->part_rels[cnt_parts];
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		Relids		child_relids1;
+		Relids		child_relids2;
+
+		/*
+		 * If this segment of the join is empty, it means that this segment
+		 * was ignored when previously creating child-join paths for it in
+		 * try_partitionwise_join() as it would not contribute to the join
+		 * result, due to one or both inputs being empty; add NULL to each of
+		 * the given lists so that this segment will be ignored again in that
+		 * function.
+		 */
+		if (!child_joinrel)
+		{
+			*parts1 = lappend(*parts1, NULL);
+			*parts2 = lappend(*parts2, NULL);
+			continue;
+		}
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel1 side.
+		 */
+		child_relids1 = bms_intersect(child_joinrel->relids,
+									  rel1->all_partrels);
+		Assert(bms_num_members(child_relids1) == bms_num_members(rel1->relids));
+
+		/*
+		 * Get a child rel for rel1 with the relids.  Note that we should have
+		 * the child rel even if rel1 is a join rel, because in that case the
+		 * partitions specified in the relids would have matching/overlapping
+		 * boundaries, so those partitions should be considered as ones to be
+		 * joined even when planning partitionwise joins of rel1, meaning that
+		 * the child rel would have been built by the time we get here.
+		 */
+		if (rel1_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids1);
+
+			child_rel1 = find_base_rel(root, varno);
+		}
+		else
+			child_rel1 = find_join_rel(root, child_relids1);
+		Assert(child_rel1);
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel2 side.
+		 */
+		child_relids2 = bms_intersect(child_joinrel->relids,
+									  rel2->all_partrels);
+		Assert(bms_num_members(child_relids2) == bms_num_members(rel2->relids));
+
+		/*
+		 * Get a child rel for rel2 with the relids.  See above comments.
+		 */
+		if (rel2_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids2);
+
+			child_rel2 = find_base_rel(root, varno);
+		}
+		else
+			child_rel2 = find_join_rel(root, child_relids2);
+		Assert(child_rel2);
+
+		/*
+		 * The join of rel1 and rel2 is legal, so is the join of the child
+		 * rels obtained above; add them to the given lists as a join pair
+		 * producing this join segment.
+		 */
+		*parts1 = lappend(*parts1, child_rel1);
+		*parts2 = lappend(*parts2, child_rel2);
+	}
+}
diff --git a/src/backend/optimizer/util/inherit.c b/src/backend/optimizer/util/inherit.c
index 38bc61e6878..caf6039c106 100644
--- a/src/backend/optimizer/util/inherit.c
+++ b/src/backend/optimizer/util/inherit.c
@@ -376,6 +376,8 @@ expand_partitioned_rtentry(PlannerInfo *root, RelOptInfo *relinfo,
 		/* Create the otherrel RelOptInfo too. */
 		childrelinfo = build_simple_rel(root, childRTindex, relinfo);
 		relinfo->part_rels[i] = childrelinfo;
+		relinfo->all_partrels = bms_add_members(relinfo->all_partrels,
+												childrelinfo->relids);
 
 		/* If this child is itself partitioned, recurse */
 		if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 03e02423b2e..a1cf6329328 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -240,10 +240,12 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->has_eclass_joins = false;
 	rel->consider_partitionwise_join = false;	/* might get changed later */
 	rel->part_scheme = NULL;
-	rel->nparts = 0;
+	rel->nparts = -1;
 	rel->boundinfo = NULL;
+	rel->merged = false;
 	rel->partition_qual = NIL;
 	rel->part_rels = NULL;
+	rel->all_partrels = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->partitioned_child_rels = NIL;
@@ -653,10 +655,12 @@ build_join_rel(PlannerInfo *root,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -829,10 +833,12 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -1639,7 +1645,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1652,24 +1658,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1681,17 +1669,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 4bc88104ee9..ff1965f5ab6 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -25,6 +25,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/pathnodes.h"
 #include "parser/parse_coerce.h"
 #include "partitioning/partbounds.h"
 #include "partitioning/partdesc.h"
@@ -69,6 +70,17 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+typedef struct PartitionMap
+{
+	int			nparts;			/* number of partitions */
+	int		   *merged_indexes;	/* indexes of merged partitions */
+	bool	   *merged;			/* flags to indicate whether partitions are
+								 * merged with non-dummy partitions */
+	bool		did_remapping;	/* did we remap partitions? */
+	int		   *old_indexes;	/* old indexes of merged partitions if
+								 * did_remapping */
+} PartitionMap;
+
 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);
@@ -108,6 +120,85 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 									 Expr **keyCol,
 									 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static PartitionBoundInfo partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts);
+static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							JoinType jointype,
+							List **outer_parts, List **inner_parts);
+static void init_partition_map(RelOptInfo *rel, PartitionMap *map);
+static void free_partition_map(PartitionMap *map);
+static int map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_part, int inner_part, int *next_index);
+static int merge_partition_with_dummy(PartitionMap *map, int index,
+						   int *next_index);
+static bool process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index);
+static bool process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index);
+static void fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes);
+static void generate_matching_part_pairs(RelOptInfo *outer_rel,
+							 RelOptInfo *inner_rel,
+							 PartitionMap *outer_map,
+							 PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_part_list,
+							 List **inner_part_list);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *collations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2);
+static bool partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
+						   Oid *collations, PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, int *ub_cmpval,
+						   int *lb_cmpval);
+static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes);
+static bool merge_default_partitions(PartitionMap *outer_map,
+						 PartitionMap *inner_map,
+						 bool outer_has_default,
+						 int outer_default,
+						 bool inner_has_default,
+						 int inner_default,
+						 JoinType jointype,
+						 int *next_index,
+						 int *default_index);
+static bool merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_map, PartitionMap *inner_map,
+					  bool outer_has_default, bool inner_has_default,
+					  bool outer_has_null, bool inner_has_null,
+					  JoinType jointype, int *next_index,
+					  int *default_index, int *null_index);
 
 /*
  * get_qual_from_partbound
@@ -2995,3 +3086,1603 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * The function produces the partition bounds for a join between two relations
+ * whose partition bounds are given. The function also returns two lists of
+ * partition indexes one for each of the joining relations. Both the lists
+ * contain the same number of elements. The partition indexes at the same
+ * positions in the lists indicate the pair partitions, one from each side, to
+ * be joined and the position itself corresponds to the index of partition
+ * produced by that child-join in the partitioned join.
+ *
+ * The function returns NULL if we can not find the matching pair of
+ * partitions. This happens if 1. multiple partitions on one side match with
+ * one partition on the other side. 2. a given partition on the outer side
+ * doesn't have a matching partition on the inner side. We can not support the
+ * first case since we don't have a way to represent multiple partitions as a
+ * single relation (RelOptInfo) and then perform join using the ganged
+ * relation. We can not support the second case since the missing inner
+ * partition needs to be represented as an empty relation and we don't have a
+ * way to introduce empty relation during join planning after creating paths
+ * for all the base relations.
+ */
+PartitionBoundInfo
+partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
+{
+	PartitionBoundInfo 	merged_bounds;
+	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
+						inner_binfo = inner_rel->boundinfo;
+	char				strategy = outer_binfo->strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = NULL;
+
+			break;
+
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partsupfunc,
+														partcollation,
+														outer_rel,
+														inner_rel,
+														jointype,
+														outer_parts,
+														inner_parts);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(partnatts,
+														 partsupfunc,
+														 partcollation,
+														 outer_rel,
+														 inner_rel,
+														 jointype,
+														 outer_parts,
+														 inner_parts);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	return merged_bounds;
+}
+
+/*
+ * partition_get_range_bounds
+ *
+ * Given the index of lower bound in datums array, return lower and upper
+ * bounds and the index of the partition with that lower bound.
+ */
+static int
+partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
+						   PartitionRangeBound *lower,
+						   PartitionRangeBound *upper)
+{
+	int			part_index;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The lower bound should correspond to a valid partition. */
+	part_index = bi->indexes[lb_index + 1];
+	Assert(part_index >= 0);
+
+	lower->kind = bi->kind[lb_index];
+	lower->datums = bi->datums[lb_index];
+	lower->lower = true;
+	upper->kind = bi->kind[lb_index + 1];
+	upper->datums = bi->datums[lb_index + 1];
+	upper->lower = false;
+
+	return part_index;
+}
+
+/*
+ * partition_range_get_next_lb_index
+ *
+ * Given the index of lower bound in datums array return the
+ * index of lower bound of the next partition. When the given index corresponds
+ * to the last partition, return number of datums (ndatums).
+ */
+static int
+partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
+{
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The partition index corresponding to the upper bound should be valid. */
+	Assert(bi->indexes[lb_index + 1] >= 0);
+
+	/*
+	 * If there are no bounds left beyond the upper bound, we have reached the
+	 * last partition.
+	 */
+	if (lb_index + 2 < bi->ndatums)
+	{
+		/*
+		 * If the bound next to the upper bound corresponds to no partition,
+		 * that's the next lower bound of the next partition. Otherwise, the
+		 * current upper bound is the lower bound of the next partition.
+		 */
+		if (bi->indexes[lb_index + 2] < 0)
+			return lb_index + 2;
+		else
+			return lb_index + 1;
+	}
+	else
+		return bi->ndatums;
+}
+
+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *partcollations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2)
+{
+	return partition_rbound_cmp(partnatts, partsupfunc, partcollations,
+								bound1->datums, bound1->kind, bound1->lower,
+								bound2);
+}
+
+/*
+ * partition_range_cmp
+ *
+ * Compare the bounds of two range partitions. Set ub_cmpval <, = or > 0, if the
+ * first partition's upper bound is lower than, equal to or higher than the
+ * second partition's upper bound resp. Similarly set lb_cmpval <, =  or > 0,
+ * if the first partition's lower bound is lower than, equal to or higher than
+ * the second partition's lower bound resp.
+ *
+ * Return true, if the ranges overlap, otherwise return false.
+ */
+static bool
+partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
+					PartitionRangeBound *lower_bound1,
+					PartitionRangeBound *upper_bound1,
+					PartitionRangeBound *lower_bound2,
+					PartitionRangeBound *upper_bound2, int *ub_cmpval,
+					int *lb_cmpval)
+{
+	bool		overlap;
+
+	/*
+	 * Compare upper bound of the first partition with the lower bound of the
+	 * second and vice-versa. If lower bound is higher than the upper bound,
+	 * the partitions are not overlapping. All other cases indicate overlapping
+	 * partitions.
+	 */
+	if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+								  lower_bound1, upper_bound2) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = 1;
+		*lb_cmpval = 1;
+	}
+	else if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+									   lower_bound2, upper_bound1) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = -1;
+		*lb_cmpval = -1;
+	}
+	else
+	{
+		overlap = true;
+		*ub_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, upper_bound1,
+											   upper_bound2);
+		*lb_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, lower_bound1,
+											   lower_bound2);
+	}
+
+	return overlap;
+}
+
+/*
+ * partition_range_merge
+ *
+ * Merge the partition bounds of given two partitions such that the join
+ * between the given two partitions fits merged bounds.
+ *
+ * "merged_upper" will be set to one of the given upper bounds and
+ * "merged_lower" will be set to one of the given lower bounds.
+ */
+static void
+partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
+					  Oid *partcollations, JoinType jointype,
+					  PartitionRangeBound *left_lb,
+					  PartitionRangeBound *left_ub,
+					  PartitionRangeBound *right_lb,
+					  PartitionRangeBound *right_ub,
+					  PartitionRangeBound **merged_lb,
+					  PartitionRangeBound **merged_ub)
+{
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = left_ub;
+			*merged_lb = left_lb;
+			break;
+
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) < 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) > 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		case JOIN_FULL:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) > 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) < 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+}
+
+/*
+ * Add the lower bound of the next range to the list of bounds, if the lower
+ * bound is higher or equal to the previous upper bound. If successful return
+ * true, otherwise false.
+ */
+static bool
+partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
+							  Oid *partcollations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		prev_ub.datums = llast(*merged_datums);
+		prev_ub.kind = llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  next_lb_datums, next_lb_kind, false,
+									  &prev_ub);
+	}
+
+	/*
+	 * The lower bound is lower than the last upper bound, thus does not fit
+	 * the bounds created so far and hence can not be merged with the existing
+	 * bounds.
+	 */
+	if (cmpval < 0)
+		return false;
+
+	/*
+	 * Add bounds of the new merged partition. If the next lower bound is
+	 * higher than the last upper bound, add new range with index
+	 * corresponding to the lower bound as -1. If the merged lower bound
+	 * is same as the last merged upper bound, the last upper bound will be
+	 * reused as the lower bound of the next range.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, next_lb_datums);
+		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	return true;
+}
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			next_index = 0;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_kinds = NIL;
+	List	   *merged_indexes = NIL;
+	int			outer_lb_index;
+	int			inner_lb_index;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = inner_lb_index = 0;
+	while (outer_lb_index < outer_bi->ndatums ||
+		   inner_lb_index < inner_bi->ndatums)
+	{
+		PartitionRangeBound *merged_lb = NULL;
+		PartitionRangeBound *merged_ub = NULL;
+		int			merged_index = -1;
+		PartitionRangeBound outer_lb;
+		PartitionRangeBound outer_ub;
+		PartitionRangeBound inner_lb;
+		PartitionRangeBound inner_ub;
+		int			outer_part = -1;
+		int			inner_part = -1;
+		bool		overlap;
+		int			ub_cmpval;
+		int			lb_cmpval;
+
+		/* Get the range bounds of the next pair of partitions. */
+		if (outer_lb_index < outer_bi->ndatums)
+			outer_part = partition_get_range_bounds(outer_bi, outer_lb_index,
+													&outer_lb, &outer_ub);
+		if (inner_lb_index < inner_bi->ndatums)
+			inner_part = partition_get_range_bounds(inner_bi, inner_lb_index,
+													&inner_lb, &inner_ub);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (outer_lb_index >= outer_bi->ndatums)
+		{
+			overlap = false;
+			ub_cmpval = 1;
+			lb_cmpval = 1;
+		}
+		else if (inner_lb_index >= inner_bi->ndatums)
+		{
+			overlap = false;
+			ub_cmpval = -1;
+			lb_cmpval = -1;
+		}
+		else
+			overlap = partition_range_cmp(partnatts, partsupfuncs, partcollations,
+										  &outer_lb, &outer_ub, &inner_lb,
+										  &inner_ub, &ub_cmpval, &lb_cmpval);
+
+		if (overlap)
+		{
+			/*
+			 * The rows from overlapping portion of ranges on both sides may
+			 * join, hence the corresponding pair of partitions form a joining
+			 * pair. Match them and produce the bounds of the joint partition
+			 * and its index by merging the bounds according to the type of
+			 * join.
+			 */
+			partition_range_merge(partnatts, partsupfuncs, partcollations,
+								  jointype, &outer_lb, &outer_ub, &inner_lb,
+								  &inner_ub, &merged_lb, &merged_ub);
+
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
+													outer_part, inner_part,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				/* Failed to match the partitions. */
+				return NULL;
+			}
+
+			/*
+			 * If the ranges overlap but don't exactly match, a row from
+			 * non-overlapping portion of the range from one side of join may
+			 * find its join partner in the previous or next overlapping
+			 * partition or default partition on the other side , if such a
+			 * partition exists. All those cases, if true, will cause one
+			 * partition from that side to match at least two partitions on the
+			 * other side; a case that we do not support now. Previous
+			 * partition has been delt with in the previous iteration of this
+			 * loop, next partition will be delt in the next iteration. We will
+			 * deal with the default partition here.
+			 */
+			if ((lb_cmpval < 0 && inner_has_default) ||
+				/* Non-overlapping range on the lower side of outer range. */
+				(lb_cmpval > 0 && outer_has_default) ||
+				/* Non-overlapping range on the lower side of inner range. */
+				(ub_cmpval < 0 && outer_has_default) ||
+				/* Non-overlapping range on the upper side of inner range. */
+				(ub_cmpval > 0 && inner_has_default))
+				/* Non-overlapping range on the upper side of outer range. */
+				return NULL;
+		}
+
+		if (ub_cmpval == 0)
+		{
+			/* Upper bounds of both the ranges match. */
+			Assert(overlap);
+
+			/* Move to the next pair of partitions. */
+			Assert(outer_lb_index < outer_bi->ndatums);
+			Assert(inner_lb_index < inner_bi->ndatums);
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				if (inner_has_default || IS_OUTER_JOIN(jointype))
+				{
+					if (!process_outer_partition(&outer_map,
+												 &inner_map,
+												 outer_has_default,
+												 inner_has_default,
+												 outer_part,
+												 inner_default,
+												 jointype,
+												 outer_bi->strategy,
+												 &next_index,
+												 &default_index,
+												 &merged_index))
+						return NULL;
+				}
+
+				merged_lb = &outer_lb;
+				merged_ub = &outer_ub;
+			}
+
+			/* Move to the next partition on the outer side. */
+			Assert(outer_lb_index < outer_bi->ndatums);
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+		}
+		else
+		{
+			Assert(ub_cmpval > 0);
+
+			/* Upper bound of outer range higher than that of the inner. */
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				if (outer_has_default || jointype == JOIN_FULL)
+				{
+					if (!process_inner_partition(&outer_map,
+												 &inner_map,
+												 outer_has_default,
+												 inner_has_default,
+												 inner_part,
+												 outer_default,
+												 jointype,
+												 outer_bi->strategy,
+												 &next_index,
+												 &default_index,
+												 &merged_index))
+						return NULL;
+				}
+
+				merged_lb = &inner_lb;
+				merged_ub = &inner_ub;
+			}
+
+			/* Move to the next partition on the inner side. */
+			Assert(inner_lb_index < inner_bi->ndatums);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+
+		if (merged_index < 0)
+		{
+			/* We didn't find a new merged partition. */
+			continue;
+		}
+
+		/*
+		 * We have a valid partition index for the next partition of join. The
+		 * partition should have valid range.
+		 */
+		Assert(merged_lb && merged_ub);
+
+		/* Try merging new lower bound with the last upper bound. */
+		if (!partition_range_merge_next_lb(partnatts, partsupfuncs,
+										   partcollations,
+										   merged_lb->datums,
+										   merged_lb->kind, &merged_datums,
+										   &merged_kinds, &merged_indexes))
+			return NULL;
+
+		/* Add upper bound with the merged partition index. */
+		merged_datums = lappend(merged_datums, merged_ub->datums);
+		merged_kinds = lappend(merged_kinds, merged_ub->kind);
+		merged_indexes = lappend_int(merged_indexes, merged_index);
+	}
+
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+	{
+		if (!merge_default_partitions(&outer_map,
+									  &inner_map,
+									  outer_has_default,
+									  outer_default,
+									  inner_has_default,
+									  inner_default,
+									  jointype,
+									  &next_index,
+									  &default_index))
+			return NULL;
+	}
+	else
+		Assert(default_index == -1);
+
+	Assert(!outer_map.did_remapping && !inner_map.did_remapping);
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 &outer_map, &inner_map,
+								 next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes,
+													  merged_kinds,
+													  -1, default_index);
+	}
+
+	/* Free any memory we used in this function. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							JoinType jointype,
+							List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			next_index = 0;
+	int			null_index = -1;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			cnto;
+	int			cnti;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *merged_datum = NULL;
+		int			merged_index = -1;
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			cmpval;
+
+		/* Get the list datums of the next pair of partitions. */
+		odatums = cnto < outer_bi->ndatums ? outer_bi->datums[cnto] : NULL;
+		idatums = cnti < inner_bi->ndatums ? inner_bi->datums[cnti] : NULL;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+			cmpval = 1;
+		else if (cnti >= inner_bi->ndatums)
+			cmpval = -1;
+		else
+		{
+			Assert(odatums != NULL && idatums != NULL);
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0],
+													 idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			int			o_index = outer_bi->indexes[cnto];
+			int			i_index = inner_bi->indexes[cnti];
+
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			Assert(o_index >= 0 && i_index >= 0);
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
+													o_index, i_index,
+													&next_index);
+
+			if (merged_index < 0)
+				return NULL;
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			Assert(cnto < outer_bi->ndatums);
+
+			/* A datum missing from the inner side. */
+			merged_datum = odatums;
+
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				int			o_index = outer_bi->indexes[cnto];
+
+				Assert(o_index >= 0);
+				if (!process_outer_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 o_index,
+											 inner_default,
+											 jointype,
+											 outer_bi->strategy,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next datum on the outer side. */
+			cnto++;
+		}
+		else
+		{
+			Assert(cmpval > 0);
+			Assert(cnti < inner_bi->ndatums);
+
+			/* A datum missing from the outer side. */
+			merged_datum = idatums;
+
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				int			i_index = inner_bi->indexes[cnti];
+
+				Assert(i_index >= 0);
+				if (!process_inner_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 i_index,
+											 outer_default,
+											 jointype,
+											 outer_bi->strategy,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next datum on the inner side. */
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	/* Merge null partitions if any. */
+	if (outer_has_null || inner_has_null)
+	{
+		if (!merge_null_partitions(outer_bi, inner_bi,
+								   &outer_map, &inner_map,
+								   outer_has_default, inner_has_default,
+								   outer_has_null, inner_has_null,
+								   jointype, &next_index, &default_index,
+								   &null_index))
+			return NULL;
+	}
+	else
+		Assert(null_index == -1);
+
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+	{
+		if (!merge_default_partitions(&outer_map, &inner_map,
+									  outer_has_default, outer_default,
+									  inner_has_default, inner_default,
+									  jointype, &next_index, &default_index))
+			return NULL;
+	}
+	else
+		Assert(default_index == -1);
+
+	/* Fix the merged_indexes list if necessary. */
+	if (outer_map.did_remapping || inner_map.did_remapping)
+	{
+		Assert(jointype == JOIN_FULL);
+		fix_merged_indexes(&outer_map, &inner_map,
+						   next_index, merged_indexes);
+	}
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 &outer_map, &inner_map,
+								 next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) <= next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes, NIL,
+													  null_index, default_index);
+	}
+
+	/* Free up all extra memory before returning from this function. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+	list_free(merged_datums);
+	list_free(merged_indexes);
+
+	return merged_bounds;
+}
+
+/*
+ * init_partition_map
+ *
+ * Initialize a PartitionMap struct for given relation.
+ */
+static void
+init_partition_map(RelOptInfo *rel, PartitionMap *map)
+{
+	int			nparts = rel->nparts;
+	int			i;
+
+	map->nparts = nparts;
+	map->merged_indexes = (int *) palloc(sizeof(int) * nparts);
+	map->merged = (bool *) palloc(sizeof(bool) * nparts);
+	map->did_remapping = false;
+	map->old_indexes = (int *) palloc(sizeof(int) * nparts);
+	for (i = 0; i < nparts; i++)
+	{
+		map->merged_indexes[i] = map->old_indexes[i] = -1;
+		map->merged[i] = false;
+	}
+}
+
+/*
+ * free_partition_map
+ */
+static void
+free_partition_map(PartitionMap *map)
+{
+	pfree(map->merged_indexes);
+	pfree(map->merged);
+	pfree(map->old_indexes);
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given partitions.
+ */
+static int
+map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_index, int inner_index, int *next_index)
+{
+	int 		outer_merged_index;
+	bool 		outer_merged;
+	int 		inner_merged_index;
+	bool 		inner_merged;
+
+	Assert(outer_index >= 0 && outer_index < outer_map->nparts);
+	outer_merged_index = outer_map->merged_indexes[outer_index];
+	outer_merged = outer_map->merged[outer_index];
+	Assert(inner_index >= 0 && inner_index < inner_map->nparts);
+	inner_merged_index = inner_map->merged_indexes[inner_index];
+	inner_merged = inner_map->merged[inner_index];
+
+	/*
+	 * Handle cases where both partitions are mapped to merged partitions.
+	 */
+	if (outer_merged_index >= 0 && inner_merged_index >= 0)
+	{
+		/*
+		 * If the mereged partitions are the same, no need to do anything;
+		 * return the index of the merged partition.  Otherwise, if both
+		 * partitions are merged with dummy partitions, re-merge them; map
+		 * them to the merged partition with the smaller of the two merged
+		 * indexes and return the smaller index.  Otherwise they can't be
+		 * merged, so return -1.
+		 */
+		if (outer_merged_index == inner_merged_index)
+		{
+			Assert(outer_merged);
+			Assert(inner_merged);
+			return outer_merged_index;
+		}
+		if (!outer_merged && !inner_merged)
+		{
+			/*
+			 * Note that we will fix the larger index that have been added to
+			 * the merged_indexes list so far in fix_merged_indexes().
+			 */
+			if (outer_merged_index < inner_merged_index)
+			{
+				outer_map->merged[outer_index] = true;
+				inner_map->merged_indexes[inner_index] = outer_merged_index;
+				inner_map->merged[inner_index] = true;
+				inner_map->did_remapping = true;
+				inner_map->old_indexes[inner_index] = inner_merged_index;
+				return outer_merged_index;
+			}
+			else
+			{
+				inner_map->merged[inner_index] = true;
+				outer_map->merged_indexes[outer_index] = inner_merged_index;
+				outer_map->merged[outer_index] = true;
+				outer_map->did_remapping = true;
+				outer_map->old_indexes[outer_index] = outer_merged_index;
+				return inner_merged_index;
+			}
+		}
+		return -1;
+	}
+
+	/* At least one partition isn't mapped to a merged partition. */
+	Assert(outer_merged_index == -1 || inner_merged_index == -1);
+
+	/*
+	 * If neither of partitions isn't mapped, assign them a new merged
+	 * partition and return the index of the merged partition.  Otherwise, if
+	 * one of partitions is merged with a dummy relation (and the other isn't
+	 * merged), re-merge it with the other, with the same index, and return
+	 * the index.  Otherwise they can't be merged, so return -1.
+	 */
+	if (outer_merged_index == -1 && inner_merged_index == -1)
+	{
+		int 		merged_index = *next_index;
+
+		Assert(!outer_merged);
+		Assert(!inner_merged);
+		outer_map->merged_indexes[outer_index] = merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged_indexes[inner_index] = merged_index;
+		inner_map->merged[inner_index] = true;
+		*next_index = *next_index + 1;
+		return merged_index;
+	}
+	if (outer_merged_index >= 0 && !outer_map->merged[outer_index])
+	{
+		Assert(inner_merged_index == -1);
+		Assert(!inner_merged);
+		inner_map->merged_indexes[inner_index] = outer_merged_index;
+		inner_map->merged[inner_index] = true;
+		outer_map->merged[outer_index] = true;
+		return outer_merged_index;
+	}
+	if (inner_merged_index >= 0 && !inner_map->merged[inner_index])
+	{
+		Assert(outer_merged_index == -1);
+		Assert(!outer_merged);
+		outer_map->merged_indexes[outer_index] = inner_merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged[inner_index] = true;
+		return inner_merged_index;
+	}
+	return -1;
+}
+
+/*
+ * merge_partition_with_dummy
+ *
+ * *next_index is incremented.
+ */
+static int
+merge_partition_with_dummy(PartitionMap *map, int index, int *next_index)
+{
+	int 		merged_index = *next_index;
+
+	Assert(index >= 0 && index < map->nparts);
+	Assert(map->merged_indexes[index] == -1);
+	Assert(!map->merged[index]);
+	map->merged_indexes[index] = merged_index;
+	/* Leave the merged flag alone! */
+	*next_index = *next_index + 1;
+	return merged_index;
+}
+
+/*
+ * process_outer_partition
+ *
+ * Determine the merged partition associated with the given outer partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given outer partition.
+ */
+static bool
+process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index)
+{
+	Assert(outer_index >= 0);
+
+	/*
+	 * If the inner side has the default partition, the outer partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an outer join, in which case the outer partition has to be
+	 * scanned all the way anyway; if the outer partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the outer partition with a dummy partition.
+	 */
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0);
+
+		/*
+		 * If the outer side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (outer_has_default)
+			return false;
+
+		*merged_index = map_and_merge_partitions(outer_map, inner_map,
+												 outer_index, inner_default,
+												 next_index);
+		if (*merged_index == -1)
+			return false;
+
+		/*
+		 * If this is a FULL join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (*default_index == -1)
+				*default_index = *merged_index;
+			else
+				Assert(*merged_index == *default_index);
+			/* Don't add this index to the list of merged indexes. */
+			*merged_index = -1;
+		}
+	}
+	else
+	{
+		Assert(IS_OUTER_JOIN(jointype));
+		Assert(jointype != JOIN_RIGHT);
+
+		/*
+		 * In range partitioning, if the given outer partition is already
+		 * merged (eg, because we found an overlapping range earlier), we know
+		 * where it fits in the join result; nothing to do in that case.  Else
+		 * create a new merged partition.
+		 */
+		if (outer_map->merged_indexes[outer_index] >= 0)
+		{
+			if (strategy == PARTITION_STRATEGY_LIST)
+				*merged_index = outer_map->merged_indexes[outer_index];
+			else
+			{
+				Assert(strategy == PARTITION_STRATEGY_RANGE);
+				*merged_index = -1;
+			}
+		}
+		else
+			*merged_index = merge_partition_with_dummy(outer_map, outer_index,
+													   next_index);
+	}
+	return true;
+}
+
+/*
+ * process_inner_partition
+ *
+ * Determine the merged partition associated with the given inner partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given inner partition.
+ */
+static bool
+process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index)
+{
+	Assert(inner_index >= 0);
+
+	/*
+	 * If the outer side has the default partition, the inner partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an FULL join, in which case the inner partition has to be
+	 * scanned all the way anyway; if the inner partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the inner partition with a dummy partition.
+	 */
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0);
+
+		/*
+		 * If the inner side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (inner_has_default)
+			return false;
+
+		*merged_index = map_and_merge_partitions(outer_map, inner_map,
+												 outer_default, inner_index,
+												 next_index);
+		if (*merged_index == -1)
+			return false;
+
+		/*
+		 * If this is an outer join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (jointype == JOIN_LEFT || jointype == JOIN_ANTI ||
+			jointype == JOIN_FULL)
+		{
+			if (*default_index == -1)
+				*default_index = *merged_index;
+			else
+				Assert(*merged_index == *default_index);
+			/* Don't add this index to the list of merged indexes. */
+			*merged_index = -1;
+		}
+	}
+	else
+	{
+		Assert(jointype == JOIN_FULL);
+
+		/*
+		 * In range partitioning, if the given inner partition is already
+		 * merged (eg, because we found an overlapping range earlier), we know
+		 * where it fits in the join result; nothing to do in that case.  Else
+		 * create a new merged partition.
+		 */
+		if (inner_map->merged_indexes[inner_index] >= 0)
+		{
+			if (strategy == PARTITION_STRATEGY_LIST)
+				*merged_index = inner_map->merged_indexes[inner_index];
+			else
+			{
+				Assert(strategy == PARTITION_STRATEGY_RANGE);
+				*merged_index = -1;
+			}
+		}
+		else
+			*merged_index = merge_partition_with_dummy(inner_map, inner_index,
+													   next_index);
+	}
+	return true;
+}
+
+/*
+ * fix_merged_indexes
+ */
+static void
+fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes)
+{
+	int		   *new_indexes;
+	int			merged_index;
+	int			i;
+	ListCell   *lc;
+
+	new_indexes = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		new_indexes[i] = -1;
+
+	/* Build the mapping of old merged indexes to new merged indexes. */
+	if (outer_map->did_remapping)
+	{
+		for (i = 0; i < outer_map->nparts; i++)
+		{
+			merged_index = outer_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = outer_map->merged_indexes[i];
+		}
+	}
+	if (inner_map->did_remapping)
+	{
+		for (i = 0; i < inner_map->nparts; i++)
+		{
+			merged_index = inner_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = inner_map->merged_indexes[i];
+		}
+	}
+
+	/* Fix the merged_indexes list using the mapping. */
+	foreach(lc, merged_indexes)
+	{
+		merged_index = lfirst_int(lc);
+		Assert(merged_index >= 0);
+		if (new_indexes[merged_index] >= 0)
+			lfirst_int(lc) = new_indexes[merged_index];
+	}
+
+	pfree(new_indexes);
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * This function produces the list pairs of partitions which produce merged
+ * partitions in the order of merged partition indexes.
+ *
+ * nmerged is the number of merged partitions.
+ *
+ * If successful, the list pairs are returned as two separate lists,
+ * outer_part_list and inner_part_list, one for each side.  Otherwise, those
+ * lists will be set to NIL.
+ */
+static void
+generate_matching_part_pairs(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 PartitionMap *outer_map, PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_part_list, List **inner_part_list)
+{
+	int			outer_nparts = outer_map->nparts;
+	int			inner_nparts = inner_map->nparts;
+	int		   *outer_part_array;
+	int		   *inner_part_array;
+	int			max_nparts;
+	int 		i;
+
+	Assert(outer_nparts == outer_rel->nparts);
+	Assert(inner_nparts == inner_rel->nparts);
+
+	outer_part_array = (int *) palloc(sizeof(int) * nmerged);
+	inner_part_array = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		outer_part_array[i] = inner_part_array[i] = -1;
+
+	/* Set pairs of matching partitions. */
+	max_nparts = Max(outer_nparts, inner_nparts);
+	for (i = 0; i < max_nparts; i++)
+	{
+		if (i < outer_nparts)
+		{
+			int 		merged_index = outer_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				outer_part_array[merged_index] = i;
+			}
+		}
+		if (i < inner_nparts)
+		{
+			int 		merged_index = inner_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				inner_part_array[merged_index] = i;
+			}
+		}
+	}
+
+	*outer_part_list = NIL;
+	*inner_part_list = NIL;
+
+	/* Build the list pairs. */
+	for (i = 0; i < nmerged; i++)
+	{
+		int			outer_part = outer_part_array[i];
+		int			inner_part = inner_part_array[i];
+
+		/*
+		 * If both partitions are dummy, it means the merged partition was
+		 * removed by partition re-merging in map_and_merge_partitions();
+		 * ignore the merged partition
+		 */
+		if (outer_part == -1 && inner_part == -1)
+			continue;
+
+		*outer_part_list = lappend(*outer_part_list, outer_part >= 0 ?
+								   outer_rel->part_rels[outer_part] : NULL);
+		*inner_part_list = lappend(*inner_part_list, inner_part >= 0 ?
+								   inner_rel->part_rels[inner_part] : NULL);
+	}
+
+	pfree(outer_part_array);
+	pfree(inner_part_array);
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * Merge default partitions from both sides, if any, and assign the default
+ * partition for the join result, if necessary.
+ *
+ * If both the relations have default partitions, try mapping those to each
+ * other. If the mapping succeeds corresponding merged partition will act as
+ * the default partition of the join result.
+ *
+ * If inner side of the join has default but not the outer side, rows in it
+ * won't appear in the join result. So don't create a default partition. If
+ * outer side of the join has default but not the inner side, rows in it will
+ * appear in the join result, so create a default merged partition.
+ */
+static bool
+merge_default_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 bool outer_has_default, int outer_default,
+						 bool inner_has_default, int inner_default,
+						 JoinType jointype, int *next_index,
+						 int *default_index)
+{
+	Assert(outer_has_default || inner_has_default);
+
+	if (outer_has_default && !inner_has_default)
+	{
+		if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
+			jointype == JOIN_ANTI)
+		{
+			int			merged_index;
+
+			Assert(outer_default >= 0 && outer_default < outer_map->nparts);
+			merged_index = outer_map->merged_indexes[outer_default];
+			if (merged_index == -1)
+			{
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(outer_map,
+															outer_default,
+															next_index);
+			}
+			else
+				Assert(*default_index == merged_index);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else if (!outer_has_default && inner_has_default)
+	{
+		if (jointype == JOIN_FULL)
+		{
+			int			merged_index;
+
+			Assert(inner_default >= 0 && inner_default < inner_map->nparts);
+			merged_index = inner_map->merged_indexes[inner_default];
+			if (merged_index == -1)
+			{
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(inner_map,
+															inner_default,
+															next_index);
+			}
+			else
+				Assert(*default_index == merged_index);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		*default_index = map_and_merge_partitions(outer_map,
+												  inner_map,
+												  outer_default,
+												  inner_default,
+												  next_index);
+		if (*default_index == -1)
+			return false;
+	}
+
+	return true;
+}
+
+/*
+ * merge_null_partitions
+ *
+ * Merge NULL partitions, i.e. a partition that can hold NULL values for a list
+ * partitioned table, if any. Find the index of merged partition to which the
+ * NULL values would belong in the join result. If one joining relation has a
+ * NULL partition but not the other, try matching it with the default partition
+ * from the other relation since the default partition may have rows with NULL
+ * partition key. We can eliminate a NULL partition when it appears only on the
+ * inner side of the join and the outer side doesn't have a default partition.
+ *
+ * When the equality operator used for join is strict, two NULL values will not
+ * be considered as equal, and thus a NULL partition can be eliminated for an
+ * inner join. But we don't check the strictness operator here.
+ */
+static bool
+merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_map, PartitionMap *inner_map,
+					  bool outer_has_default, bool inner_has_default,
+					  bool outer_has_null, bool inner_has_null,
+					  JoinType jointype, int *next_index,
+					  int *default_index, int *null_index)
+{
+	Assert(outer_has_null || inner_has_null);
+	Assert(*null_index == -1);
+
+	if (outer_has_null && !inner_has_null)
+	{
+		int			merged_index = -1;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join to which the NULL partition matches will
+		 * contain the NULL values and thus become the NULL partition of the
+		 * the join.
+		 */
+		if (inner_has_default || IS_OUTER_JOIN(jointype))
+		{
+			if (!process_outer_partition(outer_map,
+										 inner_map,
+										 outer_has_default,
+										 inner_has_default,
+										 outer_bi->null_index,
+										 inner_bi->default_index,
+										 jointype,
+										 outer_bi->strategy,
+										 next_index,
+										 default_index,
+										 &merged_index))
+				return false;
+		}
+		*null_index = merged_index;
+	}
+	else if (!outer_has_null && inner_has_null)
+	{
+		int			merged_index = -1;
+
+		/*
+		 * If the NULL partition was missing from the outer side of the join,
+		 * the partition of the join to which the NULL partition matches will
+		 * contain the NULL values and thus become the NULL partition of the
+		 * the join.
+		 */
+		if (outer_has_default || jointype == JOIN_FULL)
+		{
+			if (!process_inner_partition(outer_map,
+										 inner_map,
+										 outer_has_default,
+										 inner_has_default,
+										 inner_bi->null_index,
+										 outer_bi->default_index,
+										 jointype,
+										 outer_bi->strategy,
+										 next_index,
+										 default_index,
+										 &merged_index))
+				return false;
+		}
+		*null_index = merged_index;
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		*null_index = map_and_merge_partitions(outer_map,
+											   inner_map,
+											   outer_bi->null_index,
+											   inner_bi->null_index,
+											   next_index);
+		if (*null_index == -1)
+			return false;
+	}
+
+	return true;
+}
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index 23a06d718e3..e06eb9aaae5 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -577,8 +577,10 @@ typedef struct PartitionSchemeData *PartitionScheme;
  *		part_scheme - Partitioning scheme of the relation
  *		nparts - Number of partitions
  *		boundinfo - Partition bounds
+ *		merged - true if partition bounds are merged ones
  *		partition_qual - Partition constraint if not the root
  *		part_rels - RelOptInfos for each partition
+ *		all_partrels - Relids set of all partition relids
  *		partexprs, nullable_partexprs - Partition key expressions
  *		partitioned_child_rels - RT indexes of unpruned partitions of
  *								 this relation that are partitioned tables
@@ -718,9 +720,12 @@ typedef struct RelOptInfo
 	PartitionScheme part_scheme;	/* Partitioning scheme. */
 	int			nparts;			/* number of partitions */
 	struct PartitionBoundInfoData *boundinfo;	/* Partition bounds */
+	bool		merged;			/* true if partition bounds were created by
+								 * partition_bounds_merge() */
 	List	   *partition_qual; /* partition constraint */
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of partitions,
 									 * stored in the same order of bounds */
+	Relids		all_partrels;	/* Relids set of all partition relids */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 	List	   *partitioned_child_rels; /* List of RT indexes. */
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index 0d0fd42b181..9292aa11e5e 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -16,6 +16,7 @@
 #include "nodes/pg_list.h"
 #include "partitioning/partdefs.h"
 #include "utils/relcache.h"
+struct RelOptInfo;				/* avoid including pathnodes.h here */
 
 
 /*
@@ -108,5 +109,11 @@ extern int	partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 										  int nvalues, Datum *values, bool *is_equal);
 extern int	partition_hash_bsearch(PartitionBoundInfo boundinfo,
 								   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   struct RelOptInfo *outer_rel, struct RelOptInfo *inner_rel,
+					   JoinType jointype,
+					   List **outer_parts, List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 975bf6765ca..618862ce87f 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -8,59 +8,86 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-(4 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+(7 rows)
 
 -- left outer join, with whole-row reference; partitionwise join does not apply
 EXPLAIN (COSTS OFF)
@@ -72,35 +99,50 @@ SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER
    ->  Hash Right Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(22 rows)
 
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-      t1      |      t2      
---------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
- (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
- (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
- (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
- (550,0,0550) | 
-(12 rows)
+       t1       |       t2       
+----------------+----------------
+ (-250,0,-0250) | 
+ (-200,0,-0200) | 
+ (-150,0,-0150) | (0,-150,-0150)
+ (-100,0,-0100) | 
+ (-50,0,-0050)  | 
+ (0,0,0000)     | (0,0,0000)
+ (50,0,0050)    | 
+ (100,0,0100)   | 
+ (150,0,0150)   | (0,150,0150)
+ (200,0,0200)   | 
+ (250,0,0250)   | 
+ (300,0,0300)   | (0,300,0300)
+ (350,0,0350)   | 
+ (400,0,0400)   | 
+ (450,0,0450)   | (0,450,0450)
+ (500,0,0500)   | 
+ (550,0,0550)   | 
+ (600,0,0600)   | (0,600,0600)
+ (650,0,0650)   | 
+ (700,0,0700)   | 
+ (750,0,0750)   | (0,750,0750)
+(21 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
@@ -112,35 +154,53 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
    ->  Append
          ->  Hash Right Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Right Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t2_2.b)
-(20 rows)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+      |       | -225 | -0225
+      |       |  -75 | -0075
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(14 rows)
 
 -- full outer join, with placeholder vars
 EXPLAIN (COSTS OFF)
@@ -148,8 +208,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                             QUERY PLAN                            
 ------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               Filter: (((50) = prt1_p0.a) OR ((75) = prt2_p0.b))
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
@@ -174,7 +242,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               Filter: (((50) = prt1_p4.a) OR ((75) = prt2_p4.b))
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
@@ -211,35 +287,44 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Hash Right Join
-         Hash Cond: (prt2_p2.b = prt1_p1.a)
+         Hash Cond: (prt2_p2.b = prt1_p0.a)
          ->  Append
                ->  Seq Scan on prt2_p2
                      Filter: (b > 250)
                ->  Seq Scan on prt2_p3
                      Filter: (b > 250)
+               ->  Seq Scan on prt2_p4
+                     Filter: (b > 250)
          ->  Hash
                ->  Append
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p1
                            Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p2
                            Filter: ((a < 450) AND (b = 0))
-(15 rows)
+(19 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-(9 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+(14 rows)
 
 -- Currently we can't do partitioned join if nullable-side partitions are pruned
 EXPLAIN (COSTS OFF)
@@ -247,11 +332,13 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                       QUERY PLAN                      
 ------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Hash Full Join
-         Hash Cond: (prt1_p1.a = prt2_p2.b)
-         Filter: ((prt1_p1.b = 0) OR (prt2_p2.a = 0))
+         Hash Cond: (prt1_p0.a = prt2_p2.b)
+         Filter: ((prt1_p0.b = 0) OR (prt2_p2.a = 0))
          ->  Append
+               ->  Seq Scan on prt1_p0
+                     Filter: (a < 450)
                ->  Seq Scan on prt1_p1
                      Filter: (a < 450)
                ->  Seq Scan on prt1_p2
@@ -262,64 +349,147 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                            Filter: (b > 250)
                      ->  Seq Scan on prt2_p3
                            Filter: (b > 250)
-(16 rows)
+                     ->  Seq Scan on prt2_p4
+                           Filter: (b > 250)
+(20 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
-(12 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+      |       | 375 | 0375
+      |       | 450 | 0450
+      |       | 525 | 0525
+      |       | 600 | 0600
+      |       | 675 | 0675
+      |       | 750 | 0750
+(20 rows)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Semi Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Semi Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                            Filter: (a = 0)
-         ->  Nested Loop Semi Join
-               Join Filter: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
-               ->  Materialize
-                     ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
-(24 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt1_p3 t2_3
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+                           Filter: (b = 0)
+(37 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
 
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
@@ -330,27 +500,82 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
    ->  Append
          ->  Hash Anti Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Hash Anti Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Hash Anti Join
                Hash Cond: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
-(17 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(27 rows)
 
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
-  sum  |         avg          | sum  |         avg         
--------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+  sum  |         avg          | sum  |        avg         
+-------+----------------------+------+--------------------
+ 95550 | 273.0000000000000000 | 2200 | 6.2857142857142857
 (1 row)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Append
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p0 t1
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+               Index Cond: (a = t1.b)
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.b = t2_1.a)
+         ->  Seq Scan on prt2_p1 t1_1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p1 t2_1
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p2 t1_2
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+               Index Cond: (a = t1_2.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p3 t1_3
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+               Index Cond: (a = t1_3.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p4 t1_4
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+               Index Cond: (a = t1_4.b)
+(27 rows)
+
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+  b   |   c   
+------+-------
+ -225 | -0225
+  -75 | -0075
+   75 | 0075
+  225 | 0225
+  375 | 0375
+  525 | 0525
+  675 | 0675
+(7 rows)
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -362,49 +587,74 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
+                     ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
                            Index Cond: (a = t1.a)
-                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t3
                            Index Cond: (b = t2.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_1
+                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
                            Index Cond: (a = t1_1.a)
-                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3_1
                            Index Cond: (b = t2_1.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_2
+                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
                            Index Cond: (a = t1_2.a)
-                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_2
                            Index Cond: (b = t2_2.a)
-(27 rows)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                           Index Cond: (a = t1_3.a)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_3
+                           Index Cond: (b = t2_3.a)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                           Index Cond: (a = t1_4.a)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t3_4
+                           Index Cond: (b = t2_4.a)
+(43 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   | t2a | t3a | least 
------+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
- 100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
- 250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
- 400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
- 550 | 0 | 0550 |     |     |      
-(12 rows)
+  a   | b |   c   | t2a  | t3a | least 
+------+---+-------+------+-----+-------
+ -250 | 0 | -0250 |      |     |      
+ -200 | 0 | -0200 |      |     |      
+ -150 | 0 | -0150 | -150 |   0 |  -150
+ -100 | 0 | -0100 |      |     |      
+  -50 | 0 | -0050 |      |     |      
+    0 | 0 | 0000  |    0 |   0 |     0
+   50 | 0 | 0050  |      |     |      
+  100 | 0 | 0100  |      |     |      
+  150 | 0 | 0150  |  150 |   0 |   150
+  200 | 0 | 0200  |      |     |      
+  250 | 0 | 0250  |      |     |      
+  300 | 0 | 0300  |  300 |   0 |   300
+  350 | 0 | 0350  |      |     |      
+  400 | 0 | 0400  |      |     |      
+  450 | 0 | 0450  |  450 |   0 |   450
+  500 | 0 | 0500  |      |     |      
+  550 | 0 | 0550  |      |     |      
+  600 | 0 | 0600  |  600 |   0 |   600
+  650 | 0 | 0650  |      |     |      
+  700 | 0 | 0700  |      |     |      
+  750 | 0 | 0750  |  750 |   0 |   750
+(21 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
@@ -418,46 +668,67 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
          Hash Cond: ((t1.c)::text = (t2.c)::text)
          Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Seq Scan on prt1_p1 t1
-               ->  Seq Scan on prt1_p2 t1_1
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
          ->  Hash
                ->  Append
                      ->  Hash Join
                            Hash Cond: (t2.a = t3.b)
-                           ->  Seq Scan on prt1_p1 t2
+                           ->  Seq Scan on prt1_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t3
+                                 ->  Seq Scan on prt2_p0 t3
                      ->  Hash Join
                            Hash Cond: (t2_1.a = t3_1.b)
-                           ->  Seq Scan on prt1_p2 t2_1
+                           ->  Seq Scan on prt1_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t3_1
+                                 ->  Seq Scan on prt2_p1 t3_1
                      ->  Hash Join
                            Hash Cond: (t2_2.a = t3_2.b)
-                           ->  Seq Scan on prt1_p3 t2_2
+                           ->  Seq Scan on prt1_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p3 t3_2
-(26 rows)
+                                 ->  Seq Scan on prt2_p2 t3_2
+                     ->  Hash Join
+                           Hash Cond: (t2_3.a = t3_3.b)
+                           ->  Seq Scan on prt1_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t3_3
+                     ->  Hash Join
+                           Hash Cond: (t2_4.a = t3_4.b)
+                           ->  Seq Scan on prt1_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t3_4
+(38 rows)
 
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, t2.b t2b, t2.c t2c, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
-  a  | t2a | t2c  
------+-----+------
-   0 |   0 | 0000
-  50 |     | 
- 100 |     | 
- 150 | 150 | 0150
- 200 |     | 
- 250 |     | 
- 300 | 300 | 0300
- 350 |     | 
- 400 |     | 
- 450 | 450 | 0450
- 500 |     | 
- 550 |     | 
-(12 rows)
+  a   | t2a  |  t2c  
+------+------+-------
+ -250 |      | 
+ -200 |      | 
+ -150 | -150 | -0150
+ -100 |      | 
+  -50 |      | 
+    0 |    0 | 0000
+   50 |      | 
+  100 |      | 
+  150 |  150 | 0150
+  200 |      | 
+  250 |      | 
+  300 |  300 | 0300
+  350 |      | 
+  400 |      | 
+  450 |  450 | 0450
+  500 |      | 
+  550 |      | 
+  600 |  600 | 0600
+  650 |      | 
+  700 |      | 
+  750 |  750 | 0750
+(21 rows)
 
 -- bug with inadequate sort key representation
 SET enable_partitionwise_aggregate TO true;
@@ -469,49 +740,75 @@ SELECT a, b FROM prt1 FULL JOIN prt2 p2(b,a,c) USING(a,b)
                                                     QUERY PLAN                                                     
 -------------------------------------------------------------------------------------------------------------------
  Group
-   Group Key: (COALESCE(prt1_p1.a, p2.a)), (COALESCE(prt1_p1.b, p2.b))
+   Group Key: (COALESCE(prt1_p0.a, p2.a)), (COALESCE(prt1_p0.b, p2.b))
    ->  Merge Append
-         Sort Key: (COALESCE(prt1_p1.a, p2.a)), (COALESCE(prt1_p1.b, p2.b))
+         Sort Key: (COALESCE(prt1_p0.a, p2.a)), (COALESCE(prt1_p0.b, p2.b))
          ->  Group
-               Group Key: (COALESCE(prt1_p1.a, p2.a)), (COALESCE(prt1_p1.b, p2.b))
+               Group Key: (COALESCE(prt1_p0.a, p2.a)), (COALESCE(prt1_p0.b, p2.b))
                ->  Sort
-                     Sort Key: (COALESCE(prt1_p1.a, p2.a)), (COALESCE(prt1_p1.b, p2.b))
+                     Sort Key: (COALESCE(prt1_p0.a, p2.a)), (COALESCE(prt1_p0.b, p2.b))
                      ->  Merge Full Join
-                           Merge Cond: ((prt1_p1.a = p2.a) AND (prt1_p1.b = p2.b))
-                           Filter: ((COALESCE(prt1_p1.a, p2.a) >= 490) AND (COALESCE(prt1_p1.a, p2.a) <= 510))
+                           Merge Cond: ((prt1_p0.a = p2.a) AND (prt1_p0.b = p2.b))
+                           Filter: ((COALESCE(prt1_p0.a, p2.a) >= 490) AND (COALESCE(prt1_p0.a, p2.a) <= 510))
+                           ->  Sort
+                                 Sort Key: prt1_p0.a, prt1_p0.b
+                                 ->  Seq Scan on prt1_p0
+                           ->  Sort
+                                 Sort Key: p2.a, p2.b
+                                 ->  Seq Scan on prt2_p0 p2
+         ->  Group
+               Group Key: (COALESCE(prt1_p1.a, p2_1.a)), (COALESCE(prt1_p1.b, p2_1.b))
+               ->  Sort
+                     Sort Key: (COALESCE(prt1_p1.a, p2_1.a)), (COALESCE(prt1_p1.b, p2_1.b))
+                     ->  Merge Full Join
+                           Merge Cond: ((prt1_p1.a = p2_1.a) AND (prt1_p1.b = p2_1.b))
+                           Filter: ((COALESCE(prt1_p1.a, p2_1.a) >= 490) AND (COALESCE(prt1_p1.a, p2_1.a) <= 510))
                            ->  Sort
                                  Sort Key: prt1_p1.a, prt1_p1.b
                                  ->  Seq Scan on prt1_p1
                            ->  Sort
-                                 Sort Key: p2.a, p2.b
-                                 ->  Seq Scan on prt2_p1 p2
+                                 Sort Key: p2_1.a, p2_1.b
+                                 ->  Seq Scan on prt2_p1 p2_1
          ->  Group
-               Group Key: (COALESCE(prt1_p2.a, p2_1.a)), (COALESCE(prt1_p2.b, p2_1.b))
+               Group Key: (COALESCE(prt1_p2.a, p2_2.a)), (COALESCE(prt1_p2.b, p2_2.b))
                ->  Sort
-                     Sort Key: (COALESCE(prt1_p2.a, p2_1.a)), (COALESCE(prt1_p2.b, p2_1.b))
+                     Sort Key: (COALESCE(prt1_p2.a, p2_2.a)), (COALESCE(prt1_p2.b, p2_2.b))
                      ->  Merge Full Join
-                           Merge Cond: ((prt1_p2.a = p2_1.a) AND (prt1_p2.b = p2_1.b))
-                           Filter: ((COALESCE(prt1_p2.a, p2_1.a) >= 490) AND (COALESCE(prt1_p2.a, p2_1.a) <= 510))
+                           Merge Cond: ((prt1_p2.a = p2_2.a) AND (prt1_p2.b = p2_2.b))
+                           Filter: ((COALESCE(prt1_p2.a, p2_2.a) >= 490) AND (COALESCE(prt1_p2.a, p2_2.a) <= 510))
                            ->  Sort
                                  Sort Key: prt1_p2.a, prt1_p2.b
                                  ->  Seq Scan on prt1_p2
                            ->  Sort
-                                 Sort Key: p2_1.a, p2_1.b
-                                 ->  Seq Scan on prt2_p2 p2_1
+                                 Sort Key: p2_2.a, p2_2.b
+                                 ->  Seq Scan on prt2_p2 p2_2
          ->  Group
-               Group Key: (COALESCE(prt1_p3.a, p2_2.a)), (COALESCE(prt1_p3.b, p2_2.b))
+               Group Key: (COALESCE(prt1_p3.a, p2_3.a)), (COALESCE(prt1_p3.b, p2_3.b))
                ->  Sort
-                     Sort Key: (COALESCE(prt1_p3.a, p2_2.a)), (COALESCE(prt1_p3.b, p2_2.b))
+                     Sort Key: (COALESCE(prt1_p3.a, p2_3.a)), (COALESCE(prt1_p3.b, p2_3.b))
                      ->  Merge Full Join
-                           Merge Cond: ((prt1_p3.a = p2_2.a) AND (prt1_p3.b = p2_2.b))
-                           Filter: ((COALESCE(prt1_p3.a, p2_2.a) >= 490) AND (COALESCE(prt1_p3.a, p2_2.a) <= 510))
+                           Merge Cond: ((prt1_p3.a = p2_3.a) AND (prt1_p3.b = p2_3.b))
+                           Filter: ((COALESCE(prt1_p3.a, p2_3.a) >= 490) AND (COALESCE(prt1_p3.a, p2_3.a) <= 510))
                            ->  Sort
                                  Sort Key: prt1_p3.a, prt1_p3.b
                                  ->  Seq Scan on prt1_p3
                            ->  Sort
-                                 Sort Key: p2_2.a, p2_2.b
-                                 ->  Seq Scan on prt2_p3 p2_2
-(43 rows)
+                                 Sort Key: p2_3.a, p2_3.b
+                                 ->  Seq Scan on prt2_p3 p2_3
+         ->  Group
+               Group Key: (COALESCE(prt1_p4.a, p2_4.a)), (COALESCE(prt1_p4.b, p2_4.b))
+               ->  Sort
+                     Sort Key: (COALESCE(prt1_p4.a, p2_4.a)), (COALESCE(prt1_p4.b, p2_4.b))
+                     ->  Merge Full Join
+                           Merge Cond: ((prt1_p4.a = p2_4.a) AND (prt1_p4.b = p2_4.b))
+                           Filter: ((COALESCE(prt1_p4.a, p2_4.a) >= 490) AND (COALESCE(prt1_p4.a, p2_4.a) <= 510))
+                           ->  Sort
+                                 Sort Key: prt1_p4.a, prt1_p4.b
+                                 ->  Seq Scan on prt1_p4
+                           ->  Sort
+                                 Sort Key: p2_4.a, p2_4.b
+                                 ->  Seq Scan on prt2_p4 p2_4
+(69 rows)
 
 SELECT a, b FROM prt1 FULL JOIN prt2 p2(b,a,c) USING(a,b)
   WHERE a BETWEEN 490 AND 510
@@ -540,19 +837,29 @@ RESET enable_hashjoin;
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
@@ -563,32 +870,49 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
    ->  Append
          ->  Hash Join
                Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
-               ->  Seq Scan on prt2_e_p1 t2
+               ->  Seq Scan on prt2_e_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 t1
+                     ->  Seq Scan on prt1_e_p0 t1
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
-               ->  Seq Scan on prt2_e_p2 t2_1
+               Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+               ->  Seq Scan on prt1_e_p1 t1_1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_e_p1 t2_1
+         ->  Hash Join
+               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
+               ->  Seq Scan on prt2_e_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 t1_1
+                     ->  Seq Scan on prt1_e_p2 t1_2
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
-               ->  Seq Scan on prt2_e_p3 t2_2
+               Hash Cond: (((t2_3.b + t2_3.a) / 2) = ((t1_3.a + t1_3.b) / 2))
+               ->  Seq Scan on prt2_e_p3 t2_3
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 t1_2
+                     ->  Seq Scan on prt1_e_p3 t1_3
                            Filter: (c = 0)
-(21 rows)
+         ->  Hash Join
+               Hash Cond: (((t2_4.b + t2_4.a) / 2) = ((t1_4.a + t1_4.b) / 2))
+               ->  Seq Scan on prt2_e_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4 t1_4
+                           Filter: (c = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
- 150 | 0 | 150 | 0
- 300 | 0 | 300 | 0
- 450 | 0 | 450 | 0
-(4 rows)
+  a   | c |  b   | c 
+------+---+------+---
+ -250 | 0 | -250 | 0
+ -100 | 0 | -100 | 0
+    0 | 0 |    0 | 0
+    0 | 0 |    0 | 0
+  150 | 0 |  150 | 0
+  300 | 0 |  300 | 0
+  450 | 0 |  450 | 0
+  600 | 0 |  600 | 0
+  750 | 0 |  750 | 0
+(9 rows)
 
 --
 -- N-way join
@@ -601,154 +925,232 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = ((t3.a + t3.b) / 2))
+               Join Filter: (t1.a = t2.b)
                ->  Hash Join
-                     Hash Cond: (t2.b = t1.a)
-                     ->  Seq Scan on prt2_p1 t2
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on prt1_e_p0 t3
                      ->  Hash
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                     Index Cond: (((a + b) / 2) = t2.b)
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
+                     Index Cond: (b = ((t3.a + t3.b) / 2))
          ->  Nested Loop
                Join Filter: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_1.b = t1_1.a)
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
+               ->  Index Scan using iprt1_e_p4_ab2 on prt1_e_p1 t3_1
                      Index Cond: (((a + b) / 2) = t2_1.b)
          ->  Nested Loop
                Join Filter: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_2.b = t1_2.a)
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
+               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_2
                      Index Cond: (((a + b) / 2) = t2_2.b)
-(33 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = ((t3_3.a + t3_3.b) / 2))
+               ->  Nested Loop
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t2_3.b)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t2_4.b)
+               ->  Hash Join
+                     Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                     ->  Seq Scan on prt1_e_p4 t3_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_p4 t1_4
+                                 Filter: (b = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = ((t3_4.a + t3_4.b) / 2))
+(52 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
-(4 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(8 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                          QUERY PLAN                          
---------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Hash Right Join
                Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
-               ->  Seq Scan on prt1_e_p1 t3
+               ->  Seq Scan on prt1_e_p0 t3
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2.b = t1.a)
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_1.a)
-               ->  Seq Scan on prt1_e_p2 t3_1
+               ->  Seq Scan on prt1_e_p1 t3_1
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_1.b = t1_1.a)
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_2.a)
-               ->  Seq Scan on prt1_e_p3 t3_2
+               ->  Seq Scan on prt1_e_p2 t3_2
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_2.b = t1_2.a)
-                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                                        Filter: (b = 0)
-(33 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+               ->  Seq Scan on prt1_e_p4 t3_4
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_4.b = t1_4.a)
+                           ->  Seq Scan on prt2_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_p4 t1_4
+                                       Filter: (b = 0)
+(51 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                            QUERY PLAN                             
--------------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p1 t3
+                           ->  Seq Scan on prt1_e_p0 t3
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p1_b on prt2_p1 t2
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
                      Index Cond: (b = t1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p2 t3_1
+                           ->  Seq Scan on prt1_e_p1 t3_1
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
+               ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
                      Index Cond: (b = t1_1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p3 t3_2
+                           ->  Seq Scan on prt1_e_p2 t3_2
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
                      Index Cond: (b = t1_2.a)
-(30 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_e_p3 t3_3
+                           Filter: (c = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                           Index Cond: (a = ((t3_3.a + t3_3.b) / 2))
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Nested Loop Left Join
+               ->  Hash Right Join
+                     Hash Cond: (t1_4.a = ((t3_4.a + t3_4.b) / 2))
+                     ->  Seq Scan on prt1_p4 t1_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p4 t3_4
+                                 Filter: (c = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = t1_4.a)
+(47 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -757,21 +1159,34 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                                                    QUERY PLAN                                                   
 ----------------------------------------------------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   Sort Key: prt1_p0.a, prt2_p0.b, ((prt1_e_p0.a + prt1_e_p0.b))
    ->  Append
          ->  Hash Full Join
-               Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
-               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               Hash Cond: (prt1_p0.a = ((prt1_e_p0.a + prt1_e_p0.b) / 2))
+               Filter: ((prt1_p0.a = (50)) OR (prt2_p0.b = (75)) OR (((prt1_e_p0.a + prt1_e_p0.b) / 2) = (50)))
                ->  Hash Full Join
-                     Hash Cond: (prt1_p1.a = prt2_p1.b)
-                     ->  Seq Scan on prt1_p1
+                     Hash Cond: (prt1_p0.a = prt2_p0.b)
+                     ->  Seq Scan on prt1_p0
                            Filter: (b = 0)
                      ->  Hash
-                           ->  Seq Scan on prt2_p1
+                           ->  Seq Scan on prt2_p0
                                  Filter: (a = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1
+                     ->  Seq Scan on prt1_e_p0
                            Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = prt1_p1.a)
+               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               ->  Seq Scan on prt1_e_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on prt1_p1
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1
+                                       Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
                Filter: ((prt1_p2.a = (50)) OR (prt2_p2.b = (75)) OR (((prt1_e_p2.a + prt1_e_p2.b) / 2) = (50)))
@@ -798,7 +1213,20 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                ->  Hash
                      ->  Seq Scan on prt1_e_p3
                            Filter: (c = 0)
-(42 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = ((prt1_e_p4.a + prt1_e_p4.b) / 2))
+               Filter: ((prt1_p4.a = (50)) OR (prt2_p4.b = (75)) OR (((prt1_e_p4.a + prt1_e_p4.b) / 2) = (50)))
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = prt2_p4.b)
+                     ->  Seq Scan on prt1_p4
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4
+                                 Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4
+                           Filter: (c = 0)
+(68 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
@@ -816,172 +1244,260 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = t1_3.b)
+               Join Filter: (t1.a = t1_5.b)
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Join
-                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
-                           ->  Seq Scan on prt1_e_p1 t2
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t1_3
+                                 ->  Seq Scan on prt2_p0 t1_5
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
                      Index Cond: (a = ((t2.a + t2.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_1.a = t1_4.b)
+               Join Filter: (t1_1.a = t1_6.b)
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Join
-                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
-                           ->  Seq Scan on prt1_e_p2 t2_1
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_6.b)
+                           ->  Seq Scan on prt1_e_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t1_4
+                                 ->  Seq Scan on prt2_p1 t1_6
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
                      Index Cond: (a = ((t2_1.a + t2_1.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_2.a = t1_5.b)
+               Join Filter: (t1_2.a = t1_7.b)
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
+                           ->  Seq Scan on prt2_p2 t1_7
                                  Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
+                           ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t2_2
+                                 Index Cond: (((a + b) / 2) = t1_7.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t1_8.b)
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Nested Loop
+                           ->  Seq Scan on prt2_p3 t1_8
+                                 Filter: (a = 0)
+                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_3
+                                 Index Cond: (((a + b) / 2) = t1_8.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = ((t2_3.a + t2_3.b) / 2))
+                     Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t1_9.b)
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Join
+                           Hash Cond: (((t2_4.a + t2_4.b) / 2) = t1_9.b)
+                           ->  Seq Scan on prt1_e_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t1_9
+                                       Filter: (a = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = ((t2_4.a + t2_4.b) / 2))
+                     Filter: (b = 0)
+(66 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                               QUERY PLAN                                
--------------------------------------------------------------------------
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
-                           ->  Seq Scan on prt2_p1 t1_3
+                           Hash Cond: (t1_5.b = ((t1_10.a + t1_10.b) / 2))
+                           ->  Seq Scan on prt2_p0 t1_5
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t1_6
+                                 ->  Seq Scan on prt1_e_p0 t1_10
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
-                     Index Cond: (a = t1_3.b)
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t1_5.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
-                           ->  Seq Scan on prt2_p2 t1_4
+                           Hash Cond: (t1_6.b = ((t1_11.a + t1_11.b) / 2))
+                           ->  Seq Scan on prt2_p1 t1_6
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t1_7
+                                 ->  Seq Scan on prt1_e_p1 t1_11
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
-                     Index Cond: (a = t1_4.b)
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
+                     Index Cond: (a = t1_6.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
-                           ->  Seq Scan on prt2_p3 t1_5
+                           Hash Cond: (t1_7.b = ((t1_12.a + t1_12.b) / 2))
+                           ->  Seq Scan on prt2_p2 t1_7
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p3 t1_8
+                                 ->  Seq Scan on prt1_e_p2 t1_12
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t1_5.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t1_7.b)
                      Filter: (b = 0)
-(39 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_8.b = ((t1_13.a + t1_13.b) / 2))
+                           ->  Seq Scan on prt2_p3 t1_8
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p3 t1_13
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t1_8.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_9.b = ((t1_14.a + t1_14.b) / 2))
+                           ->  Seq Scan on prt2_p4 t1_9
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t1_14
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = t1_9.b)
+                     Filter: (b = 0)
+(63 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 -- test merge joins
 SET enable_hashjoin TO off;
 SET enable_nestloop TO off;
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                           QUERY PLAN                           
-----------------------------------------------------------------
+                            QUERY PLAN                            
+------------------------------------------------------------------
  Merge Append
    Sort Key: t1.a
    ->  Merge Semi Join
-         Merge Cond: (t1.a = t1_3.b)
+         Merge Cond: (t1.a = t1_5.b)
          ->  Sort
                Sort Key: t1.a
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+               Merge Cond: (t1_5.b = (((t1_10.a + t1_10.b) / 2)))
                ->  Sort
-                     Sort Key: t1_3.b
-                     ->  Seq Scan on prt2_p1 t1_3
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on prt2_p0 t1_5
                ->  Sort
-                     Sort Key: (((t1_6.a + t1_6.b) / 2))
-                     ->  Seq Scan on prt1_e_p1 t1_6
+                     Sort Key: (((t1_10.a + t1_10.b) / 2))
+                     ->  Seq Scan on prt1_e_p0 t1_10
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_1.a = t1_4.b)
+         Merge Cond: (t1_1.a = t1_6.b)
          ->  Sort
                Sort Key: t1_1.a
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+               Merge Cond: (t1_6.b = (((t1_11.a + t1_11.b) / 2)))
                ->  Sort
-                     Sort Key: t1_4.b
-                     ->  Seq Scan on prt2_p2 t1_4
+                     Sort Key: t1_6.b
+                     ->  Seq Scan on prt2_p1 t1_6
                ->  Sort
-                     Sort Key: (((t1_7.a + t1_7.b) / 2))
-                     ->  Seq Scan on prt1_e_p2 t1_7
+                     Sort Key: (((t1_11.a + t1_11.b) / 2))
+                     ->  Seq Scan on prt1_e_p1 t1_11
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_2.a = t1_5.b)
+         Merge Cond: (t1_2.a = t1_7.b)
          ->  Sort
                Sort Key: t1_2.a
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+               Merge Cond: (t1_7.b = (((t1_12.a + t1_12.b) / 2)))
                ->  Sort
-                     Sort Key: t1_5.b
-                     ->  Seq Scan on prt2_p3 t1_5
+                     Sort Key: t1_7.b
+                     ->  Seq Scan on prt2_p2 t1_7
                ->  Sort
-                     Sort Key: (((t1_8.a + t1_8.b) / 2))
-                     ->  Seq Scan on prt1_e_p3 t1_8
+                     Sort Key: (((t1_12.a + t1_12.b) / 2))
+                     ->  Seq Scan on prt1_e_p2 t1_12
                            Filter: (c = 0)
-(47 rows)
+   ->  Merge Semi Join
+         Merge Cond: (t1_3.a = t1_8.b)
+         ->  Sort
+               Sort Key: t1_3.a
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_8.b = (((t1_13.a + t1_13.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_8.b
+                     ->  Seq Scan on prt2_p3 t1_8
+               ->  Sort
+                     Sort Key: (((t1_13.a + t1_13.b) / 2))
+                     ->  Seq Scan on prt1_e_p3 t1_13
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_4.a = t1_9.b)
+         ->  Sort
+               Sort Key: t1_4.a
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_9.b = (((t1_14.a + t1_14.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_9.b
+                     ->  Seq Scan on prt2_p4 t1_9
+               ->  Sort
+                     Sort Key: (((t1_14.a + t1_14.b) / 2))
+                     ->  Seq Scan on prt1_e_p4 t1_14
+                           Filter: (c = 0)
+(77 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
@@ -998,14 +1514,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
                            ->  Sort
                                  Sort Key: (((t3.a + t3.b) / 2))
-                                 ->  Seq Scan on prt1_e_p1 t3
+                                 ->  Seq Scan on prt1_e_p0 t3
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1.a
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                ->  Sort
                      Sort Key: t2.b
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Merge Left Join
                Merge Cond: (t1_1.a = t2_1.b)
                ->  Sort
@@ -1014,14 +1530,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
                            ->  Sort
                                  Sort Key: (((t3_1.a + t3_1.b) / 2))
-                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                 ->  Seq Scan on prt1_e_p1 t3_1
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_1.a
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                ->  Sort
                      Sort Key: t2_1.b
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Merge Left Join
                Merge Cond: (t1_2.a = t2_2.b)
                ->  Sort
@@ -1030,32 +1546,74 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
                            ->  Sort
                                  Sort Key: (((t3_2.a + t3_2.b) / 2))
-                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                 ->  Seq Scan on prt1_e_p2 t3_2
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_2.a
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                ->  Sort
                      Sort Key: t2_2.b
-                     ->  Seq Scan on prt2_p3 t2_2
-(51 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Merge Left Join
+               Merge Cond: (t1_3.a = t2_3.b)
+               ->  Sort
+                     Sort Key: t1_3.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_3.a + t3_3.b) / 2)) = t1_3.a)
+                           ->  Sort
+                                 Sort Key: (((t3_3.a + t3_3.b) / 2))
+                                 ->  Seq Scan on prt1_e_p3 t3_3
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_3.a
+                                 ->  Seq Scan on prt1_p3 t1_3
+               ->  Sort
+                     Sort Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Merge Left Join
+               Merge Cond: (t1_4.a = t2_4.b)
+               ->  Sort
+                     Sort Key: t1_4.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_4.a + t3_4.b) / 2)) = t1_4.a)
+                           ->  Sort
+                                 Sort Key: (((t3_4.a + t3_4.b) / 2))
+                                 ->  Seq Scan on prt1_e_p4 t3_4
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_4.a
+                                 ->  Seq Scan on prt1_p4 t1_4
+               ->  Sort
+                     Sort Key: t2_4.b
+                     ->  Seq Scan on prt2_p4 t2_4
+(83 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- MergeAppend on nullable column
 -- This should generate a partitionwise join, but currently fails to
@@ -1064,12 +1622,14 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Merge Left Join
-         Merge Cond: (prt1_p1.a = prt2_p2.b)
+         Merge Cond: (prt1_p0.a = prt2_p2.b)
          ->  Sort
-               Sort Key: prt1_p1.a
+               Sort Key: prt1_p0.a
                ->  Append
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p1
                            Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p2
@@ -1081,21 +1641,28 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                            Filter: (b > 250)
                      ->  Seq Scan on prt2_p3
                            Filter: (b > 250)
-(18 rows)
+                     ->  Seq Scan on prt2_p4
+                           Filter: (b > 250)
+(22 rows)
 
 SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  b  
------+-----
-   0 |    
-  50 |    
- 100 |    
- 150 |    
- 200 |    
- 250 |    
- 300 | 300
- 350 |    
- 400 |    
-(9 rows)
+  a   |  b  
+------+-----
+ -250 |    
+ -200 |    
+ -150 |    
+ -100 |    
+  -50 |    
+    0 |    
+   50 |    
+  100 |    
+  150 |    
+  200 |    
+  250 |    
+  300 | 300
+  350 |    
+  400 |    
+(14 rows)
 
 -- merge join when expression with whole-row reference needs to be sorted;
 -- partitionwise join does not apply
@@ -1109,30 +1676,797 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
          Sort Key: t1.a, ((((t1.*)::prt1))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
    ->  Sort
          Sort Key: t2.b, ((((t2.*)::prt2))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt2_p1 t2
-                     ->  Seq Scan on prt2_p2 t2_1
-                     ->  Seq Scan on prt2_p3 t2_2
-(16 rows)
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+(20 rows)
 
 SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.b ORDER BY t1.a;
- a  | b  
-----+----
-  0 |  0
-  6 |  6
- 12 | 12
- 18 | 18
- 24 | 24
-(5 rows)
+  a  |  b  
+-----+-----
+ -24 | -24
+ -18 | -18
+ -12 | -12
+  -6 |  -6
+   0 |   0
+   6 |   6
+  12 |  12
+  18 |  18
+  24 |  24
+(9 rows)
 
 RESET enable_hashjoin;
 RESET enable_nestloop;
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p4 t2_3
+               ->  Seq Scan on prt2_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_4
+                           Filter: (b = 0)
+(22 rows)
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p3 t1_2
+                           Filter: (b = 0)
+(21 rows)
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -150 | -0150 | 0 | -0150
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+(7 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 |   | 
+ -200 | -0200 |   | 
+ -150 | -0150 | 0 | -0150
+ -100 | -0100 |   | 
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+(21 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -200 | 0 | -0200
+ -100 | 0 | -0100
+  -50 | 0 | -0050
+   50 | 0 | 0050
+  100 | 0 | 0100
+  200 | 0 | 0200
+  250 | 0 | 0250
+  350 | 0 | 0350
+  400 | 0 | 0400
+  500 | 0 | 0500
+  550 | 0 | 0550
+  650 | 0 | 0650
+  700 | 0 | 0700
+(14 rows)
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+                             QUERY PLAN                              
+---------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t3.c
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p0_a on prt1_p0 t3
+                           Index Cond: (a = t2.b)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t2.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_1.a = t2_1.b)
+                           ->  Seq Scan on prt1_p1 t3_1
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1 t2_1
+                                       Filter: (a = 0)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p2_a on prt1_p2 t3_2
+                           Index Cond: (a = t2_2.b)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t3_3
+                           Index Cond: (a = t2_3.b)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_4.a = t2_4.b)
+                           ->  Seq Scan on prt1_p4 t3_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t2_4
+                                       Filter: (a = 0)
+(47 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+  a   | a |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.b
+   ->  Hash Right Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p5 t2_5
+                           Filter: (a = 0)
+(24 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: (t1.a = t2.b)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.a, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
+                     Filter: (a = 0)
+(22 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
 --
 -- partitioned by multiple columns
 --
@@ -1202,82 +2536,1552 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 -- test partition matching with N-way join
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                   QUERY PLAN                                   
---------------------------------------------------------------------------------
+                                           QUERY PLAN                                           
+------------------------------------------------------------------------------------------------
  GroupAggregate
    Group Key: t1.c, t2.c, t3.c
    ->  Sort
          Sort Key: t1.c, t3.c
          ->  Append
                ->  Hash Join
-                     Hash Cond: (t1.c = ltrim(t3.c, 'A'::text))
+                     Hash Cond: ((t1.c)::text = ltrim(t3.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1.b = t2.b) AND (t1.c = t2.c))
-                           ->  Seq Scan on plt1_p1 t1
+                           Hash Cond: ((t2.b = t1.b) AND ((t2.c)::text = (t1.c)::text))
+                           ->  Seq Scan on plt2_p4 t2
                            ->  Hash
-                                 ->  Seq Scan on plt2_p1 t2
+                                 ->  Seq Scan on plt1_p4 t1
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p1 t3
+                           ->  Seq Scan on plt1_e_p4 t3
                ->  Hash Join
-                     Hash Cond: (t1_1.c = ltrim(t3_1.c, 'A'::text))
+                     Hash Cond: ((t1_1.c)::text = ltrim(t3_1.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
-                           ->  Seq Scan on plt1_p2 t1_1
+                           Hash Cond: ((t1_1.b = t2_1.b) AND ((t1_1.c)::text = (t2_1.c)::text))
+                           ->  Seq Scan on plt1_p1 t1_1
                            ->  Hash
-                                 ->  Seq Scan on plt2_p2 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p2 t3_1
+                           ->  Seq Scan on plt1_e_p1 t3_1
                ->  Hash Join
-                     Hash Cond: (t1_2.c = ltrim(t3_2.c, 'A'::text))
+                     Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
-                           ->  Seq Scan on plt1_p3 t1_2
+                           Hash Cond: ((t1_2.b = t2_2.b) AND ((t1_2.c)::text = (t2_2.c)::text))
+                           ->  Seq Scan on plt1_p2 t1_2
                            ->  Hash
-                                 ->  Seq Scan on plt2_p3 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p3 t3_2
-(32 rows)
+                           ->  Seq Scan on plt1_e_p2 t3_2
+               ->  Hash Join
+                     Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2_3.b = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text))
+                           ->  Seq Scan on plt2_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p3 t3_3
+(41 rows)
 
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  75.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 123.0000000000000000 | 123.0000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 225.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 273.0000000000000000 | 273.0000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 375.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 423.0000000000000000 | 423.0000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 525.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 573.0000000000000000 | 573.0000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p5 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_3
+                                 ->  Seq Scan on plt2_p3 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Seq Scan on plt2_p2 t2_2
+                                 ->  Seq Scan on plt2_p3 t2_3
+(22 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1302,22 +4106,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
 --------------------------------------------------
  Hash Left Join
    Hash Cond: (t2.b = a)
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t3.a = t2.b)
-               ->  Seq Scan on prt1_p1 t3
-               ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
-         ->  Hash Join
-               Hash Cond: (t3_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t3_1
-               ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
-         ->  Hash Join
-               Hash Cond: (t3_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t3_2
-               ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
+   ->  Hash Join
+         Hash Cond: (t3.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t3
+               ->  Seq Scan on prt1_p1 t3_1
+               ->  Seq Scan on prt1_p2 t3_2
+               ->  Seq Scan on prt1_p3 t3_3
+               ->  Seq Scan on prt1_p4 t3_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
    ->  Hash
          ->  Result
                One-Time Filter: false
@@ -1332,16 +4136,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 --
 -- tests for hash partitioned tables.
@@ -1417,41 +4227,9 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  273.0000000000000000 | 273.0000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
 (6 rows)
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
- Sort
-   Sort Key: t1.a
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
-               ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
-               ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
-                           Filter: (b = 0)
-(21 rows)
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
@@ -1466,26 +4244,24 @@ SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c
    Sort Key: t1.c
    ->  HashAggregate
          Group Key: t1.c, t2.c
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t2.c = t1.c)
-                     ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p1 t1
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on plt1_p4 t1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_1.c = t1_1.c)
-                     ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_p2 t1_1
+                           ->  Seq Scan on plt1_p1 t1_1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_2.c = t1_2.c)
-                     ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Seq Scan on plt1_p2 t1_2
                                  Filter: ((a % 25) = 0)
-(23 rows)
+                           ->  Seq Scan on plt1_p3 t1_3
+                                 Filter: ((a % 25) = 0)
+(21 rows)
 
 --
 -- multiple levels of partitioning
@@ -1881,64 +4657,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
  Hash Join
    Hash Cond: (t1.a = t2.a)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt4_n_p1 t2
                ->  Seq Scan on prt4_n_p2 t2_1
                ->  Seq Scan on prt4_n_p3 t2_2
-(11 rows)
+(13 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a and t1.a = t3.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                        QUERY PLAN                        
+----------------------------------------------------------
  Hash Join
-   Hash Cond: (t2.a = t1.a)
+   Hash Cond: (t3.b = t1.a)
    ->  Append
-         ->  Seq Scan on prt4_n_p1 t2
-         ->  Seq Scan on prt4_n_p2 t2_1
-         ->  Seq Scan on prt4_n_p3 t2_2
+         ->  Seq Scan on prt2_p0 t3
+         ->  Seq Scan on prt2_p1 t3_1
+         ->  Seq Scan on prt2_p2 t3_2
+         ->  Seq Scan on prt2_p3 t3_3
+         ->  Seq Scan on prt2_p4 t3_4
+         ->  Seq Scan on prt2_p5 t3_5
    ->  Hash
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.a = t3_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_2.b)
-                     ->  Seq Scan on prt1_p3 t1_2
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3 t3_2
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt4_n_p1 t2
+                           ->  Seq Scan on prt4_n_p2 t2_1
+                           ->  Seq Scan on prt4_n_p3 t2_2
 (23 rows)
 
 -- partitionwise join can not be applied if there are no equi-join conditions
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (b > t1.a)
-(12 rows)
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+(16 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partitionwise join
@@ -2024,16 +4806,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
-                     ->  Seq Scan on plt1_p1 t3
-                     ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3
+                     ->  Seq Scan on plt1_p1 t3_1
+                     ->  Seq Scan on plt1_p2 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partitionwise join can not be applied for a join between list and range
 -- partitioned tables
@@ -2044,14 +4827,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
  Hash Full Join
    Hash Cond: ((t2.c)::text = (t1.c)::text)
    ->  Append
-         ->  Seq Scan on prt1_p1 t2
-         ->  Seq Scan on prt1_p2 t2_1
-         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p0 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p2 t2_2
+         ->  Seq Scan on prt1_p3 t2_3
+         ->  Seq Scan on prt1_p4 t2_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt1_n_p1 t1
                ->  Seq Scan on prt1_n_p2 t1_1
-(10 rows)
+(12 rows)
 
 -- partitionwise join can not be applied if only one of joining tables has
 -- default partition
@@ -2067,16 +4852,202 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
    ->  Hash Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
+ANALYZE plt1;
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+ANALYZE plt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.c = t2.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt1_p1 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t2_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_1
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+  0 | 0000 |    | 
+  5 | 0000 |    | 
+ 10 | 0000 |    | 
+ 15 | 0000 |    | 
+ 20 | 0000 |    | 
+  2 | 0002 |  2 | 0002
+  2 | 0002 |  7 | 0002
+  2 | 0002 | 12 | 0002
+  2 | 0002 | 17 | 0002
+  2 | 0002 | 22 | 0002
+  7 | 0002 |  2 | 0002
+  7 | 0002 |  7 | 0002
+  7 | 0002 | 12 | 0002
+  7 | 0002 | 17 | 0002
+  7 | 0002 | 22 | 0002
+ 12 | 0002 |  2 | 0002
+ 12 | 0002 |  7 | 0002
+ 12 | 0002 | 12 | 0002
+ 12 | 0002 | 17 | 0002
+ 12 | 0002 | 22 | 0002
+ 17 | 0002 |  2 | 0002
+ 17 | 0002 |  7 | 0002
+ 17 | 0002 | 12 | 0002
+ 17 | 0002 | 17 | 0002
+ 17 | 0002 | 22 | 0002
+ 22 | 0002 |  2 | 0002
+ 22 | 0002 |  7 | 0002
+ 22 | 0002 | 12 | 0002
+ 22 | 0002 | 17 | 0002
+ 22 | 0002 | 22 | 0002
+    |      |  1 | 0001
+    |      |  6 | 0001
+    |      | 11 | 0001
+    |      | 16 | 0001
+    |      | 21 | 0001
+(35 rows)
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1;
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2;
+CREATE TABLE plt3 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt3_p1 PARTITION OF plt3 FOR VALUES IN ('0001');
+CREATE TABLE plt3_p2 PARTITION OF plt3 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2.a, t3.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.c = t3.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1.c = t2.c)
+                     ->  Seq Scan on plt1_p1 t1
+                     ->  Hash
+                           ->  Seq Scan on plt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on plt3_p1 t3
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t3_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_1.c = t2_1.c)
+                     ->  Seq Scan on plt1_p2 t1_1
+                     ->  Hash
+                           ->  Seq Scan on plt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt3_p2 t3_1
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+ a  |  c   | a  |  c   | a  |  c   
+----+------+----+------+----+------
+  0 | 0000 |    |      |    | 
+  5 | 0000 |    |      |    | 
+ 10 | 0000 |    |      |    | 
+ 15 | 0000 |    |      |    | 
+ 20 | 0000 |    |      |    | 
+  1 | 0001 |    |      |  1 | 0001
+  1 | 0001 |    |      |  6 | 0001
+  1 | 0001 |    |      | 11 | 0001
+  1 | 0001 |    |      | 16 | 0001
+  1 | 0001 |    |      | 21 | 0001
+  6 | 0001 |    |      |  1 | 0001
+  6 | 0001 |    |      |  6 | 0001
+  6 | 0001 |    |      | 11 | 0001
+  6 | 0001 |    |      | 16 | 0001
+  6 | 0001 |    |      | 21 | 0001
+ 11 | 0001 |    |      |  1 | 0001
+ 11 | 0001 |    |      |  6 | 0001
+ 11 | 0001 |    |      | 11 | 0001
+ 11 | 0001 |    |      | 16 | 0001
+ 11 | 0001 |    |      | 21 | 0001
+ 16 | 0001 |    |      |  1 | 0001
+ 16 | 0001 |    |      |  6 | 0001
+ 16 | 0001 |    |      | 11 | 0001
+ 16 | 0001 |    |      | 16 | 0001
+ 16 | 0001 |    |      | 21 | 0001
+ 21 | 0001 |    |      |  1 | 0001
+ 21 | 0001 |    |      |  6 | 0001
+ 21 | 0001 |    |      | 11 | 0001
+ 21 | 0001 |    |      | 16 | 0001
+ 21 | 0001 |    |      | 21 | 0001
+  2 | 0002 |  2 | 0002 |    | 
+  2 | 0002 |  7 | 0002 |    | 
+  2 | 0002 | 12 | 0002 |    | 
+  2 | 0002 | 17 | 0002 |    | 
+  2 | 0002 | 22 | 0002 |    | 
+  7 | 0002 |  2 | 0002 |    | 
+  7 | 0002 |  7 | 0002 |    | 
+  7 | 0002 | 12 | 0002 |    | 
+  7 | 0002 | 17 | 0002 |    | 
+  7 | 0002 | 22 | 0002 |    | 
+ 12 | 0002 |  2 | 0002 |    | 
+ 12 | 0002 |  7 | 0002 |    | 
+ 12 | 0002 | 12 | 0002 |    | 
+ 12 | 0002 | 17 | 0002 |    | 
+ 12 | 0002 | 22 | 0002 |    | 
+ 17 | 0002 |  2 | 0002 |    | 
+ 17 | 0002 |  7 | 0002 |    | 
+ 17 | 0002 | 12 | 0002 |    | 
+ 17 | 0002 | 17 | 0002 |    | 
+ 17 | 0002 | 22 | 0002 |    | 
+ 22 | 0002 |  2 | 0002 |    | 
+ 22 | 0002 |  7 | 0002 |    | 
+ 22 | 0002 | 12 | 0002 |    | 
+ 22 | 0002 | 17 | 0002 |    | 
+ 22 | 0002 | 22 | 0002 |    | 
+(55 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 92994b479bb..b7f53388d22 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -10,25 +10,39 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -69,11 +83,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -110,20 +132,30 @@ RESET enable_hashjoin;
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 
 EXPLAIN (COSTS OFF)
@@ -187,6 +219,128 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches default partition and
+-- prt2_p4 from prt2 and default partition from prt2 matches default partition
+-- and prt1_p0 from prt1
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- partition-wise join should be possible when we drop the first and last
+-- partitions from both sides
+ALTER TABLE prt1 DETACH PARTITION prt1_p0;
+ALTER TABLE prt1 DETACH PARTITION prt1_p4;
+ANALYZE prt1;
+ALTER TABLE prt2 DETACH PARTITION prt2_p0;
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 ATTACH PARTITION prt1_p0 FOR VALUES FROM (MINVALUE) TO (0);
+ALTER TABLE prt1 ATTACH PARTITION prt1_p4 FOR VALUES FROM (600) TO (800);
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p0 FOR VALUES FROM (-250) TO (0);
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt2;
+
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -211,28 +365,79 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
 -- test partition matching with N-way join
@@ -240,6 +445,175 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
@@ -285,27 +659,18 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
 ALTER TABLE plt2 DETACH PARTITION plt2_p3;
 ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
 ANALYZE plt2;
-
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.a % 25 = 0 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
 --
 -- multiple levels of partitioning
 --
@@ -450,3 +815,47 @@ ANALYZE prt2;
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
+ANALYZE plt1;
+
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+ANALYZE plt2;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a;
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1;
+
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2;
+
+CREATE TABLE plt3 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt3_p1 PARTITION OF plt3 FOR VALUES IN ('0001');
+CREATE TABLE plt3_p2 PARTITION OF plt3 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
-- 
2.18.0

#104Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: amul sul (#103)
2 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Hi Amul,

On Wed, Nov 6, 2019 at 3:12 PM amul sul <sulamul@gmail.com> wrote:

Attached is the rebase atop the latest master head(a9056cc637f).

Thanks for that!

On Tue, Nov 5, 2019 at 6:44 PM amul sul <sulamul@gmail.com> wrote:

On Fri, Nov 1, 2019 at 3:58 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

Done. Attached is a new version of the patch.

A query and comments for v25:

583 + * The function returns NULL if we can not find the matching pair of
584 + * partitions. This happens if 1. multiple partitions on one side match with
585 + * one partition on the other side. 2. a given partition on the outer side
586 + * doesn't have a matching partition on the inner side. We can not support the
587 + * first case since we don't have a way to represent multiple partitions as a
588 + * single relation (RelOptInfo) and then perform join using the ganged
589 + * relation. We can not support the second case since the missing inner
590 + * partition needs to be represented as an empty relation and we don't have a
591 + * way to introduce empty relation during join planning after creating paths
592 + * for all the base relations.
593 + */
594 +PartitionBoundInfo
595 +partition_bounds_merge(int partnatts,

I think the second condition mention for partition_bounds_merge() looks
outdated, do you think we should remove that or am I missing something here?

Yeah, I think so. In addition to that, the output arguments for that
function *outer_pars and *inner_parts don't store partition indexes
anymore, so I updated comments for that function totally. Does that
make sense?

1768 +
1769 + /*
1770 + * If this is an outer join, the merged partition would act as the
1771 + * default partition of the join; record the index in *default_index
1772 + * if not done yet.
1773 + */
1774 + if (jointype == JOIN_LEFT || jointype == JOIN_ANTI ||
1775 + jointype == JOIN_FULL)
1776 + {

As decided need to replace this list by IS_OUTER_JOIN(jointype).

Done.

2020 + if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
2021 + jointype == JOIN_ANTI)
2022 + {

Same as the previous.

Done.

I tried a coverage testing and tried to adjust & add a few tests to improved the
code coverage for the v25 patch. Please have a look at the attached 0002 & also
attach the coverage output with & without this patch, TWIMW.

Good idea! I merged that to the main patch after modifying that a bit
so to reduce the outputs of SELECTs.

Attached is an updated version of the main patch. Thanks for reviewing!

I looked at partition_range_bounds_merge() more closely. Here are my comments:

* When merging partition bounds from the outer/inner sides in the
while loop of that function, we only moves to the next partition on
one side when both partitions overlap (ie, overlap=true) and the upper
bounds of partitions are not the same (ie, ub_cmpval<0 or
ub_cmpval>0); but I think that's inefficient. Let me explain using an
example:

create table prt1 (a int, b int) partition by range (a);
create table prt1_p1 partition of prt1 for values from (0) to (50);
create table prt1_p2 partition of prt1 for values from (100) to (150);
create table prt2 (a int, b int) partition by range (a);
create table prt2_p1 partition of prt2 for values from (0) to (100);
create table prt2_p2 partition of prt2 for values from (100) to (200);
insert into prt1 select i, i from generate_series(0, 49) i;
insert into prt1 select i, i from generate_series(100, 149) i;
insert into prt2 select i, i from generate_series(0, 49) i;
insert into prt2 select i, i from generate_series(100, 149) i;
analyze prt1;
analyze prt2;
set enable_partitionwise_join to on;
explain verbose select * from prt1 t1 full join prt2 t2 on (t1.a = t2.a);
QUERY PLAN
--------------------------------------------------------------------------------------
Append (cost=2.12..9.12 rows=100 width=16)
-> Hash Full Join (cost=2.12..4.31 rows=50 width=16)
Output: t1.a, t1.b, t2.a, t2.b
Hash Cond: (t1.a = t2.a)
-> Seq Scan on public.prt1_p1 t1 (cost=0.00..1.50 rows=50 width=8)
Output: t1.a, t1.b
-> Hash (cost=1.50..1.50 rows=50 width=8)
Output: t2.a, t2.b
-> Seq Scan on public.prt2_p1 t2 (cost=0.00..1.50
rows=50 width=8)
Output: t2.a, t2.b
-> Hash Full Join (cost=2.12..4.31 rows=50 width=16)
Output: t1_1.a, t1_1.b, t2_1.a, t2_1.b
Hash Cond: (t1_1.a = t2_1.a)
-> Seq Scan on public.prt1_p2 t1_1 (cost=0.00..1.50 rows=50 width=8)
Output: t1_1.a, t1_1.b
-> Hash (cost=1.50..1.50 rows=50 width=8)
Output: t2_1.a, t2_1.b
-> Seq Scan on public.prt2_p2 t2_1 (cost=0.00..1.50
rows=50 width=8)
Output: t2_1.a, t2_1.b
(19 rows)

For this query, the merging would proceed as follow:

1) In the first iteration of the while loop, we have overlap=true, so
we merge prt1_p1 and prt2_p1 and move to the next partition on *only
the outer side* (prt1_p2) as we have ub_cmpval<0. 2) In the second
iteration, we have overlap=false and ub_cmpval>0, so we perform
process_inner_partition() to prt2_p1 and move to the next partition on
the inner side (prt2_p2). 3) In the third iteration, we have
overlap=true, so we merge prt1_p2 and prt2_p2 and move to the next
partition on *only the outer side* as we have ub_cmpval<0 (but can't
as the outer side is exhausted). 4) In the forth iteration, we have
overlap=false and ub_cmpval>0, so we perform process_inner_partition()
to prt2_p2 and move to the next partition on the inner side (but can't
as the inner side is exhausted). And we are done.

We do this in the process_inner_partition() call in the second and
forth iterations:

/*
* In range partitioning, if the given inner partition is already
* merged (eg, because we found an overlapping range earlier), we know
* where it fits in the join result; nothing to do in that case. Else
* create a new merged partition.
*/
if (inner_map->merged_indexes[inner_index] >= 0)
{
if (strategy == PARTITION_STRATEGY_LIST)
*merged_index = inner_map->merged_indexes[inner_index];
else
{
Assert(strategy == PARTITION_STRATEGY_RANGE);
*merged_index = -1;
}
}

What we do in that function is actually a no-op, so the second and
forth iterations are done only to move to the next partition on the
inner side, which I think is inefficient. To avoid that, why not move
to *the next pair of partitions* in the first and third iterations
like the attached? This needs an extra check to see if we can safely
move to the next pair of partitions in the first and third iterations,
but doesn't need meaningless iterations like the second and fourth
iterations in the example. This change also makes the code in
process_inner_partition() shown above unnecessary, so I removed it
from that function (and process_outer_partition()).

* I don't like this:

+           if ((lb_cmpval < 0 && inner_has_default) ||
+               /* Non-overlapping range on the lower side of outer range. */
+               (lb_cmpval > 0 && outer_has_default) ||
+               /* Non-overlapping range on the lower side of inner range. */
+               (ub_cmpval < 0 && outer_has_default) ||
+               /* Non-overlapping range on the upper side of inner range. */
+               (ub_cmpval > 0 && inner_has_default))
+               /* Non-overlapping range on the upper side of outer range. */
+               return NULL;

because it looks like that eg, the first comment "Non-overlapping
range on the lower side of outer range." explains the second condition
"(lb_cmpval > 0 && outer_has_default)", which isn't intended, I think.
Also, it seems a bit verbose to me to add a comment to each of the
four cases. How about simplifying the comments and moving them to
above the if block like the attached?

* In partition_range_merge_next_lb(), do we really need this bit?

+   /*
+    * The lower bound is lower than the last upper bound, thus does not fit
+    * the bounds created so far and hence can not be merged with the existing
+    * bounds.
+    */
+   if (cmpval < 0)
+       return false;

I think that if we have such a lower bound, we would error out before
calling that function. Also, I think it would be easier to understand
to add the upper bound as well within that function. So I modified
that function that way (and renamed it). I also added a bit of
comments to that function.

* Since this is called many times, I changed it to a macro:

+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+                         Oid *partcollations, PartitionRangeBound *bound1,
+                         PartitionRangeBound *bound2)
+{
+   return partition_rbound_cmp(partnatts, partsupfunc, partcollations,
+                               bound1->datums, bound1->kind, bound1->lower,
+                               bound2);
+}

* This is just my taste, but how about renaming
partition_range_bound_cmp() and partition_range_cmp() to more common?
or readable? names eg, compare_range_bounds() and
compare_range_partitions(), respectively? Also, how about renaming
partition_range_merge() to eg, get_merged_range_bounds()?

Please find attached the delta patch for that. Any feedback welcome!

Best regards,
Etsuro Fujita

Attachments:

v27-0001-Improve-partition-matching-for-partitionwise-join.patchapplication/octet-stream; name=v27-0001-Improve-partition-matching-for-partitionwise-join.patchDownload
From d509e9f72362cc7dc22f962296f02b1c3ef70b08 Mon Sep 17 00:00:00 2001
From: Etsuro Fujita <efujita@postgresql.org>
Date: Tue, 12 Nov 2019 21:56:44 +0900
Subject: [PATCH 1/2] Improve partition matching for partitionwise join

---
 src/backend/nodes/outfuncs.c                 |    2 +
 src/backend/optimizer/path/joinrels.c        |  246 +-
 src/backend/optimizer/util/inherit.c         |    2 +
 src/backend/optimizer/util/relnode.c         |   45 +-
 src/backend/partitioning/partbounds.c        | 1682 +++++++
 src/include/nodes/pathnodes.h                |    5 +
 src/include/partitioning/partbounds.h        |    7 +
 src/test/regress/expected/partition_join.out | 4429 +++++++++++++++---
 src/test/regress/sql/partition_join.sql      |  482 +-
 9 files changed, 6122 insertions(+), 778 deletions(-)

diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index b0dcd02ff6..b815157402 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -2281,6 +2281,8 @@ _outRelOptInfo(StringInfo str, const RelOptInfo *node)
 	WRITE_BOOL_FIELD(has_eclass_joins);
 	WRITE_BOOL_FIELD(consider_partitionwise_join);
 	WRITE_BITMAPSET_FIELD(top_parent_relids);
+	WRITE_BOOL_FIELD(merged);
+	WRITE_BITMAPSET_FIELD(all_partrels);
 	WRITE_NODE_FIELD(partitioned_child_rels);
 }
 
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 6a480ab764..ed7bc23c7b 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -48,6 +48,9 @@ static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 												Relids left_relids, Relids right_relids);
 static int	match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel,
 										 bool strict_op);
+static void get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2);
 
 
 /*
@@ -1357,25 +1360,30 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	bool		merged = false;
+	List	   *parts1 = NIL;
+	List	   *parts2 = NIL;
+	ListCell   *lcr1 = NULL;
+	ListCell   *lcr2 = NULL;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1390,34 +1398,107 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		   joinrel->part_scheme == rel2->part_scheme);
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * If we don't have the partition bounds for the join rel yet, try to
+	 * create it along with pairs of partitions to be joined; else generate
+	 * those using the partitioning info for the join rel we already have.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	if (joinrel->nparts == -1)
+	{
+		PartitionScheme part_scheme = joinrel->part_scheme;
+		PartitionBoundInfo boundinfo = NULL;
+		int			nparts = 0;
+
+		Assert(joinrel->boundinfo == NULL);
+		Assert(joinrel->part_rels == NULL);
+
+		/*
+		 * See if the partition bounds for inputs are exactly the same, in
+		 * which case we don't need to work hard: partitions with the same
+		 * partition indexes will form join pairs, and the join rel will have
+		 * the same partition bounds as inputs; otherwise try to merge the
+		 * partition bounds along with generating join pairs.
+		 *
+		 * Even if one or both inputs have merged partition bounds, it'd be
+		 * possible for the partition bounds to be exactly the same, but it
+		 * seems unlikely to be worth the cycles to check; do this check only
+		 * if both inputs have non-merged partition bounds.
+		 */
+		if (!rel1->merged &&
+			!rel2->merged &&
+			rel1->nparts == rel2->nparts &&
+			partition_bounds_equal(part_scheme->partnatts,
+								   part_scheme->parttyplen,
+								   part_scheme->parttypbyval,
+								   rel1->boundinfo, rel2->boundinfo))
+		{
+			boundinfo = rel1->boundinfo;
+			nparts = rel1->nparts;
+		}
+		else
+		{
+			boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											   part_scheme->parttyplen,
+											   part_scheme->parttypbyval,
+											   part_scheme->partsupfunc,
+											   part_scheme->partcollation,
+											   rel1, rel2,
+											   parent_sjinfo->jointype,
+											   &parts1, &parts2);
+			if (boundinfo == NULL)
+			{
+				joinrel->nparts = 0;
+				return;
+			}
+			nparts = list_length(parts1);
+			merged = true;
+		}
+
+		Assert(nparts > 0);
+		joinrel->boundinfo = boundinfo;
+		joinrel->merged = merged;
+		joinrel->nparts = nparts;
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+	}
+	else
+	{
+		Assert(joinrel->nparts > 0);
+		Assert(joinrel->boundinfo);
+		Assert(joinrel->part_rels);
+
+		/*
+		 * If the partition bounds for the join rel are not merged ones,
+		 * inputs are guaranteed to have the same partition bounds, so
+		 * partitions with the same partition indexes will form join pairs;
+		 * else let get_matching_part_pairs() do the work.
+		 */
+		if (joinrel->merged)
+		{
+			get_matching_part_pairs(root, joinrel, rel1, rel2,
+									&parts1, &parts2);
+			Assert(list_length(parts1) == joinrel->nparts);
+			Assert(list_length(parts2) == joinrel->nparts);
+			merged = true;
+		}
+	}
 
-	nparts = joinrel->nparts;
+	if (merged)
+	{
+		lcr1 = list_head(parts1);
+		lcr2 = list_head(parts2);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
-		bool		rel1_empty = (child_rel1 == NULL ||
-								  IS_DUMMY_REL(child_rel1));
-		bool		rel2_empty = (child_rel2 == NULL ||
-								  IS_DUMMY_REL(child_rel2));
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		bool		rel1_empty;
+		bool		rel2_empty;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1425,6 +1506,22 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		if (merged)
+		{
+			child_rel1 = lfirst_node(RelOptInfo, lcr1);
+			child_rel2 = lfirst_node(RelOptInfo, lcr2);
+			lcr1 = lnext(parts1, lcr1);
+			lcr2 = lnext(parts2, lcr2);
+		}
+		else
+		{
+			child_rel1 = rel1->part_rels[cnt_parts];
+			child_rel2 = rel2->part_rels[cnt_parts];
+		}
+
+		rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1));
+		rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2));
+
 		/*
 		 * Check for cases where we can prove that this segment of the join
 		 * returns no rows, due to one or both inputs being empty (including
@@ -1522,6 +1619,8 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 												 child_sjinfo,
 												 child_sjinfo->jointype);
 			joinrel->part_rels[cnt_parts] = child_joinrel;
+			joinrel->all_partrels = bms_add_members(joinrel->all_partrels,
+													child_joinrel->relids);
 		}
 
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
@@ -1529,6 +1628,7 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
+
 	}
 }
 
@@ -1738,3 +1838,99 @@ match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op)
 
 	return -1;
 }
+
+/*
+ * get_matching_part_pairs
+ *		Generate join pairs of partitions for the two inputs
+ */
+static void
+get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2)
+{
+	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
+	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
+	int 		cnt_parts;
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
+	{
+		RelOptInfo *child_joinrel = joinrel->part_rels[cnt_parts];
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		Relids		child_relids1;
+		Relids		child_relids2;
+
+		/*
+		 * If this segment of the join is empty, it means that this segment
+		 * was ignored when previously creating child-join paths for it in
+		 * try_partitionwise_join() as it would not contribute to the join
+		 * result, due to one or both inputs being empty; add NULL to each of
+		 * the given lists so that this segment will be ignored again in that
+		 * function.
+		 */
+		if (!child_joinrel)
+		{
+			*parts1 = lappend(*parts1, NULL);
+			*parts2 = lappend(*parts2, NULL);
+			continue;
+		}
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel1 side.
+		 */
+		child_relids1 = bms_intersect(child_joinrel->relids,
+									  rel1->all_partrels);
+		Assert(bms_num_members(child_relids1) == bms_num_members(rel1->relids));
+
+		/*
+		 * Get a child rel for rel1 with the relids.  Note that we should have
+		 * the child rel even if rel1 is a join rel, because in that case the
+		 * partitions specified in the relids would have matching/overlapping
+		 * boundaries, so those partitions should be considered as ones to be
+		 * joined even when planning partitionwise joins of rel1, meaning that
+		 * the child rel would have been built by the time we get here.
+		 */
+		if (rel1_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids1);
+
+			child_rel1 = find_base_rel(root, varno);
+		}
+		else
+			child_rel1 = find_join_rel(root, child_relids1);
+		Assert(child_rel1);
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel2 side.
+		 */
+		child_relids2 = bms_intersect(child_joinrel->relids,
+									  rel2->all_partrels);
+		Assert(bms_num_members(child_relids2) == bms_num_members(rel2->relids));
+
+		/*
+		 * Get a child rel for rel2 with the relids.  See above comments.
+		 */
+		if (rel2_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids2);
+
+			child_rel2 = find_base_rel(root, varno);
+		}
+		else
+			child_rel2 = find_join_rel(root, child_relids2);
+		Assert(child_rel2);
+
+		/*
+		 * The join of rel1 and rel2 is legal, so is the join of the child
+		 * rels obtained above; add them to the given lists as a join pair
+		 * producing this join segment.
+		 */
+		*parts1 = lappend(*parts1, child_rel1);
+		*parts2 = lappend(*parts2, child_rel2);
+	}
+}
diff --git a/src/backend/optimizer/util/inherit.c b/src/backend/optimizer/util/inherit.c
index 38bc61e687..caf6039c10 100644
--- a/src/backend/optimizer/util/inherit.c
+++ b/src/backend/optimizer/util/inherit.c
@@ -376,6 +376,8 @@ expand_partitioned_rtentry(PlannerInfo *root, RelOptInfo *relinfo,
 		/* Create the otherrel RelOptInfo too. */
 		childrelinfo = build_simple_rel(root, childRTindex, relinfo);
 		relinfo->part_rels[i] = childrelinfo;
+		relinfo->all_partrels = bms_add_members(relinfo->all_partrels,
+												childrelinfo->relids);
 
 		/* If this child is itself partitioned, recurse */
 		if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 03e02423b2..a1cf632932 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -240,10 +240,12 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->has_eclass_joins = false;
 	rel->consider_partitionwise_join = false;	/* might get changed later */
 	rel->part_scheme = NULL;
-	rel->nparts = 0;
+	rel->nparts = -1;
 	rel->boundinfo = NULL;
+	rel->merged = false;
 	rel->partition_qual = NIL;
 	rel->part_rels = NULL;
+	rel->all_partrels = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->partitioned_child_rels = NIL;
@@ -653,10 +655,12 @@ build_join_rel(PlannerInfo *root,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -829,10 +833,12 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -1639,7 +1645,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1652,24 +1658,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1681,17 +1669,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index cfb44e23e2..f145851871 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -25,6 +25,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/pathnodes.h"
 #include "parser/parse_coerce.h"
 #include "partitioning/partbounds.h"
 #include "partitioning/partdesc.h"
@@ -69,6 +70,17 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+typedef struct PartitionMap
+{
+	int			nparts;			/* number of partitions */
+	int		   *merged_indexes;	/* indexes of merged partitions */
+	bool	   *merged;			/* flags to indicate whether partitions are
+								 * merged with non-dummy partitions */
+	bool		did_remapping;	/* did we remap partitions? */
+	int		   *old_indexes;	/* old indexes of merged partitions if
+								 * did_remapping */
+} PartitionMap;
+
 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);
@@ -108,6 +120,85 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 									 Expr **keyCol,
 									 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static PartitionBoundInfo partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts);
+static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							JoinType jointype,
+							List **outer_parts, List **inner_parts);
+static void init_partition_map(RelOptInfo *rel, PartitionMap *map);
+static void free_partition_map(PartitionMap *map);
+static int map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_part, int inner_part, int *next_index);
+static int merge_partition_with_dummy(PartitionMap *map, int index,
+						   int *next_index);
+static bool process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index);
+static bool process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index);
+static void fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes);
+static void generate_matching_part_pairs(RelOptInfo *outer_rel,
+							 RelOptInfo *inner_rel,
+							 PartitionMap *outer_map,
+							 PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_part_list,
+							 List **inner_part_list);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *collations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2);
+static bool partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
+						   Oid *collations, PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, int *ub_cmpval,
+						   int *lb_cmpval);
+static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes);
+static bool merge_default_partitions(PartitionMap *outer_map,
+						 PartitionMap *inner_map,
+						 bool outer_has_default,
+						 int outer_default,
+						 bool inner_has_default,
+						 int inner_default,
+						 JoinType jointype,
+						 int *next_index,
+						 int *default_index);
+static bool merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_map, PartitionMap *inner_map,
+					  bool outer_has_default, bool inner_has_default,
+					  bool outer_has_null, bool inner_has_null,
+					  JoinType jointype, int *next_index,
+					  int *default_index, int *null_index);
 
 /*
  * get_qual_from_partbound
@@ -2995,3 +3086,1594 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * This function builds and returns the partition bounds for a join relation
+ * between input relations, creating two lists of partitions, which are
+ * returned to *outer_parts and *inner_parts respectively.  The lists contain
+ * the same number of partitions, and the partitions at the same positions in
+ * the lists indicate join pairs used for partitioned join.
+ *
+ * This function returns NULL, setting *outer_parts and *inner_parts to NIL,
+ * if a partition on one side matches multiple partitions on the other side,
+ * in which case we currently don't support partitioned join.
+ */
+PartitionBoundInfo
+partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
+{
+	PartitionBoundInfo 	merged_bounds;
+	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
+						inner_binfo = inner_rel->boundinfo;
+	char				strategy = outer_binfo->strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = NULL;
+
+			break;
+
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partsupfunc,
+														partcollation,
+														outer_rel,
+														inner_rel,
+														jointype,
+														outer_parts,
+														inner_parts);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(partnatts,
+														 partsupfunc,
+														 partcollation,
+														 outer_rel,
+														 inner_rel,
+														 jointype,
+														 outer_parts,
+														 inner_parts);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	return merged_bounds;
+}
+
+/*
+ * partition_get_range_bounds
+ *
+ * Given the index of lower bound in datums array, return lower and upper
+ * bounds and the index of the partition with that lower bound.
+ */
+static int
+partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
+						   PartitionRangeBound *lower,
+						   PartitionRangeBound *upper)
+{
+	int			part_index;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The lower bound should correspond to a valid partition. */
+	part_index = bi->indexes[lb_index + 1];
+	Assert(part_index >= 0);
+
+	lower->kind = bi->kind[lb_index];
+	lower->datums = bi->datums[lb_index];
+	lower->lower = true;
+	upper->kind = bi->kind[lb_index + 1];
+	upper->datums = bi->datums[lb_index + 1];
+	upper->lower = false;
+
+	return part_index;
+}
+
+/*
+ * partition_range_get_next_lb_index
+ *
+ * Given the index of lower bound in datums array return the
+ * index of lower bound of the next partition. When the given index corresponds
+ * to the last partition, return number of datums (ndatums).
+ */
+static int
+partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
+{
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The partition index corresponding to the upper bound should be valid. */
+	Assert(bi->indexes[lb_index + 1] >= 0);
+
+	/*
+	 * If there are no bounds left beyond the upper bound, we have reached the
+	 * last partition.
+	 */
+	if (lb_index + 2 < bi->ndatums)
+	{
+		/*
+		 * If the bound next to the upper bound corresponds to no partition,
+		 * that's the next lower bound of the next partition. Otherwise, the
+		 * current upper bound is the lower bound of the next partition.
+		 */
+		if (bi->indexes[lb_index + 2] < 0)
+			return lb_index + 2;
+		else
+			return lb_index + 1;
+	}
+	else
+		return bi->ndatums;
+}
+
+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *partcollations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2)
+{
+	return partition_rbound_cmp(partnatts, partsupfunc, partcollations,
+								bound1->datums, bound1->kind, bound1->lower,
+								bound2);
+}
+
+/*
+ * partition_range_cmp
+ *
+ * Compare the bounds of two range partitions. Set ub_cmpval <, = or > 0, if the
+ * first partition's upper bound is lower than, equal to or higher than the
+ * second partition's upper bound resp. Similarly set lb_cmpval <, =  or > 0,
+ * if the first partition's lower bound is lower than, equal to or higher than
+ * the second partition's lower bound resp.
+ *
+ * Return true, if the ranges overlap, otherwise return false.
+ */
+static bool
+partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
+					PartitionRangeBound *lower_bound1,
+					PartitionRangeBound *upper_bound1,
+					PartitionRangeBound *lower_bound2,
+					PartitionRangeBound *upper_bound2, int *ub_cmpval,
+					int *lb_cmpval)
+{
+	bool		overlap;
+
+	/*
+	 * Compare upper bound of the first partition with the lower bound of the
+	 * second and vice-versa. If lower bound is higher than the upper bound,
+	 * the partitions are not overlapping. All other cases indicate overlapping
+	 * partitions.
+	 */
+	if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+								  lower_bound1, upper_bound2) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = 1;
+		*lb_cmpval = 1;
+	}
+	else if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+									   lower_bound2, upper_bound1) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = -1;
+		*lb_cmpval = -1;
+	}
+	else
+	{
+		overlap = true;
+		*ub_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, upper_bound1,
+											   upper_bound2);
+		*lb_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, lower_bound1,
+											   lower_bound2);
+	}
+
+	return overlap;
+}
+
+/*
+ * partition_range_merge
+ *
+ * Merge the partition bounds of given two partitions such that the join
+ * between the given two partitions fits merged bounds.
+ *
+ * "merged_upper" will be set to one of the given upper bounds and
+ * "merged_lower" will be set to one of the given lower bounds.
+ */
+static void
+partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
+					  Oid *partcollations, JoinType jointype,
+					  PartitionRangeBound *left_lb,
+					  PartitionRangeBound *left_ub,
+					  PartitionRangeBound *right_lb,
+					  PartitionRangeBound *right_ub,
+					  PartitionRangeBound **merged_lb,
+					  PartitionRangeBound **merged_ub)
+{
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = left_ub;
+			*merged_lb = left_lb;
+			break;
+
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) < 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) > 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		case JOIN_FULL:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) > 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) < 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+}
+
+/*
+ * Add the lower bound of the next range to the list of bounds, if the lower
+ * bound is higher or equal to the previous upper bound. If successful return
+ * true, otherwise false.
+ */
+static bool
+partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
+							  Oid *partcollations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		prev_ub.datums = llast(*merged_datums);
+		prev_ub.kind = llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  next_lb_datums, next_lb_kind, false,
+									  &prev_ub);
+	}
+
+	/*
+	 * The lower bound is lower than the last upper bound, thus does not fit
+	 * the bounds created so far and hence can not be merged with the existing
+	 * bounds.
+	 */
+	if (cmpval < 0)
+		return false;
+
+	/*
+	 * Add bounds of the new merged partition. If the next lower bound is
+	 * higher than the last upper bound, add new range with index
+	 * corresponding to the lower bound as -1. If the merged lower bound
+	 * is same as the last merged upper bound, the last upper bound will be
+	 * reused as the lower bound of the next range.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, next_lb_datums);
+		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	return true;
+}
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			next_index = 0;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_kinds = NIL;
+	List	   *merged_indexes = NIL;
+	int			outer_lb_index;
+	int			inner_lb_index;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = inner_lb_index = 0;
+	while (outer_lb_index < outer_bi->ndatums ||
+		   inner_lb_index < inner_bi->ndatums)
+	{
+		PartitionRangeBound *merged_lb = NULL;
+		PartitionRangeBound *merged_ub = NULL;
+		int			merged_index = -1;
+		PartitionRangeBound outer_lb;
+		PartitionRangeBound outer_ub;
+		PartitionRangeBound inner_lb;
+		PartitionRangeBound inner_ub;
+		int			outer_part = -1;
+		int			inner_part = -1;
+		bool		overlap;
+		int			ub_cmpval;
+		int			lb_cmpval;
+
+		/* Get the range bounds of the next pair of partitions. */
+		if (outer_lb_index < outer_bi->ndatums)
+			outer_part = partition_get_range_bounds(outer_bi, outer_lb_index,
+													&outer_lb, &outer_ub);
+		if (inner_lb_index < inner_bi->ndatums)
+			inner_part = partition_get_range_bounds(inner_bi, inner_lb_index,
+													&inner_lb, &inner_ub);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (outer_lb_index >= outer_bi->ndatums)
+		{
+			overlap = false;
+			ub_cmpval = 1;
+			lb_cmpval = 1;
+		}
+		else if (inner_lb_index >= inner_bi->ndatums)
+		{
+			overlap = false;
+			ub_cmpval = -1;
+			lb_cmpval = -1;
+		}
+		else
+			overlap = partition_range_cmp(partnatts, partsupfuncs, partcollations,
+										  &outer_lb, &outer_ub, &inner_lb,
+										  &inner_ub, &ub_cmpval, &lb_cmpval);
+
+		if (overlap)
+		{
+			/*
+			 * The rows from overlapping portion of ranges on both sides may
+			 * join, hence the corresponding pair of partitions form a joining
+			 * pair. Match them and produce the bounds of the joint partition
+			 * and its index by merging the bounds according to the type of
+			 * join.
+			 */
+			partition_range_merge(partnatts, partsupfuncs, partcollations,
+								  jointype, &outer_lb, &outer_ub, &inner_lb,
+								  &inner_ub, &merged_lb, &merged_ub);
+
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
+													outer_part, inner_part,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				/* Failed to match the partitions. */
+				return NULL;
+			}
+
+			/*
+			 * If the ranges overlap but don't exactly match, a row from
+			 * non-overlapping portion of the range from one side of join may
+			 * find its join partner in the previous or next overlapping
+			 * partition or default partition on the other side , if such a
+			 * partition exists. All those cases, if true, will cause one
+			 * partition from that side to match at least two partitions on the
+			 * other side; a case that we do not support now. Previous
+			 * partition has been delt with in the previous iteration of this
+			 * loop, next partition will be delt in the next iteration. We will
+			 * deal with the default partition here.
+			 */
+			if ((lb_cmpval < 0 && inner_has_default) ||
+				/* Non-overlapping range on the lower side of outer range. */
+				(lb_cmpval > 0 && outer_has_default) ||
+				/* Non-overlapping range on the lower side of inner range. */
+				(ub_cmpval < 0 && outer_has_default) ||
+				/* Non-overlapping range on the upper side of inner range. */
+				(ub_cmpval > 0 && inner_has_default))
+				/* Non-overlapping range on the upper side of outer range. */
+				return NULL;
+		}
+
+		if (ub_cmpval == 0)
+		{
+			/* Upper bounds of both the ranges match. */
+			Assert(overlap);
+
+			/* Move to the next pair of partitions. */
+			Assert(outer_lb_index < outer_bi->ndatums);
+			Assert(inner_lb_index < inner_bi->ndatums);
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				if (inner_has_default || IS_OUTER_JOIN(jointype))
+				{
+					if (!process_outer_partition(&outer_map,
+												 &inner_map,
+												 outer_has_default,
+												 inner_has_default,
+												 outer_part,
+												 inner_default,
+												 jointype,
+												 outer_bi->strategy,
+												 &next_index,
+												 &default_index,
+												 &merged_index))
+						return NULL;
+				}
+
+				merged_lb = &outer_lb;
+				merged_ub = &outer_ub;
+			}
+
+			/* Move to the next partition on the outer side. */
+			Assert(outer_lb_index < outer_bi->ndatums);
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+		}
+		else
+		{
+			Assert(ub_cmpval > 0);
+
+			/* Upper bound of outer range higher than that of the inner. */
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				if (outer_has_default || jointype == JOIN_FULL)
+				{
+					if (!process_inner_partition(&outer_map,
+												 &inner_map,
+												 outer_has_default,
+												 inner_has_default,
+												 inner_part,
+												 outer_default,
+												 jointype,
+												 outer_bi->strategy,
+												 &next_index,
+												 &default_index,
+												 &merged_index))
+						return NULL;
+				}
+
+				merged_lb = &inner_lb;
+				merged_ub = &inner_ub;
+			}
+
+			/* Move to the next partition on the inner side. */
+			Assert(inner_lb_index < inner_bi->ndatums);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+
+		if (merged_index < 0)
+		{
+			/* We didn't find a new merged partition. */
+			continue;
+		}
+
+		/*
+		 * We have a valid partition index for the next partition of join. The
+		 * partition should have valid range.
+		 */
+		Assert(merged_lb && merged_ub);
+
+		/* Try merging new lower bound with the last upper bound. */
+		if (!partition_range_merge_next_lb(partnatts, partsupfuncs,
+										   partcollations,
+										   merged_lb->datums,
+										   merged_lb->kind, &merged_datums,
+										   &merged_kinds, &merged_indexes))
+			return NULL;
+
+		/* Add upper bound with the merged partition index. */
+		merged_datums = lappend(merged_datums, merged_ub->datums);
+		merged_kinds = lappend(merged_kinds, merged_ub->kind);
+		merged_indexes = lappend_int(merged_indexes, merged_index);
+	}
+
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+	{
+		if (!merge_default_partitions(&outer_map,
+									  &inner_map,
+									  outer_has_default,
+									  outer_default,
+									  inner_has_default,
+									  inner_default,
+									  jointype,
+									  &next_index,
+									  &default_index))
+			return NULL;
+	}
+	else
+		Assert(default_index == -1);
+
+	Assert(!outer_map.did_remapping && !inner_map.did_remapping);
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 &outer_map, &inner_map,
+								 next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes,
+													  merged_kinds,
+													  -1, default_index);
+	}
+
+	/* Free any memory we used in this function. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							JoinType jointype,
+							List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			next_index = 0;
+	int			null_index = -1;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			cnto;
+	int			cnti;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *merged_datum = NULL;
+		int			merged_index = -1;
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			cmpval;
+
+		/* Get the list datums of the next pair of partitions. */
+		odatums = cnto < outer_bi->ndatums ? outer_bi->datums[cnto] : NULL;
+		idatums = cnti < inner_bi->ndatums ? inner_bi->datums[cnti] : NULL;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+			cmpval = 1;
+		else if (cnti >= inner_bi->ndatums)
+			cmpval = -1;
+		else
+		{
+			Assert(odatums != NULL && idatums != NULL);
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0],
+													 idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			int			o_index = outer_bi->indexes[cnto];
+			int			i_index = inner_bi->indexes[cnti];
+
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			Assert(o_index >= 0 && i_index >= 0);
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
+													o_index, i_index,
+													&next_index);
+
+			if (merged_index < 0)
+				return NULL;
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			Assert(cnto < outer_bi->ndatums);
+
+			/* A datum missing from the inner side. */
+			merged_datum = odatums;
+
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				int			o_index = outer_bi->indexes[cnto];
+
+				Assert(o_index >= 0);
+				if (!process_outer_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 o_index,
+											 inner_default,
+											 jointype,
+											 outer_bi->strategy,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next datum on the outer side. */
+			cnto++;
+		}
+		else
+		{
+			Assert(cmpval > 0);
+			Assert(cnti < inner_bi->ndatums);
+
+			/* A datum missing from the outer side. */
+			merged_datum = idatums;
+
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				int			i_index = inner_bi->indexes[cnti];
+
+				Assert(i_index >= 0);
+				if (!process_inner_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 i_index,
+											 outer_default,
+											 jointype,
+											 outer_bi->strategy,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next datum on the inner side. */
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	/* Merge null partitions if any. */
+	if (outer_has_null || inner_has_null)
+	{
+		if (!merge_null_partitions(outer_bi, inner_bi,
+								   &outer_map, &inner_map,
+								   outer_has_default, inner_has_default,
+								   outer_has_null, inner_has_null,
+								   jointype, &next_index, &default_index,
+								   &null_index))
+			return NULL;
+	}
+	else
+		Assert(null_index == -1);
+
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+	{
+		if (!merge_default_partitions(&outer_map, &inner_map,
+									  outer_has_default, outer_default,
+									  inner_has_default, inner_default,
+									  jointype, &next_index, &default_index))
+			return NULL;
+	}
+	else
+		Assert(default_index == -1);
+
+	/* Fix the merged_indexes list if necessary. */
+	if (outer_map.did_remapping || inner_map.did_remapping)
+	{
+		Assert(jointype == JOIN_FULL);
+		fix_merged_indexes(&outer_map, &inner_map,
+						   next_index, merged_indexes);
+	}
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 &outer_map, &inner_map,
+								 next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) <= next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes, NIL,
+													  null_index, default_index);
+	}
+
+	/* Free up all extra memory before returning from this function. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+	list_free(merged_datums);
+	list_free(merged_indexes);
+
+	return merged_bounds;
+}
+
+/*
+ * init_partition_map
+ *
+ * Initialize a PartitionMap struct for given relation.
+ */
+static void
+init_partition_map(RelOptInfo *rel, PartitionMap *map)
+{
+	int			nparts = rel->nparts;
+	int			i;
+
+	map->nparts = nparts;
+	map->merged_indexes = (int *) palloc(sizeof(int) * nparts);
+	map->merged = (bool *) palloc(sizeof(bool) * nparts);
+	map->did_remapping = false;
+	map->old_indexes = (int *) palloc(sizeof(int) * nparts);
+	for (i = 0; i < nparts; i++)
+	{
+		map->merged_indexes[i] = map->old_indexes[i] = -1;
+		map->merged[i] = false;
+	}
+}
+
+/*
+ * free_partition_map
+ */
+static void
+free_partition_map(PartitionMap *map)
+{
+	pfree(map->merged_indexes);
+	pfree(map->merged);
+	pfree(map->old_indexes);
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given partitions.
+ */
+static int
+map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_index, int inner_index, int *next_index)
+{
+	int 		outer_merged_index;
+	bool 		outer_merged;
+	int 		inner_merged_index;
+	bool 		inner_merged;
+
+	Assert(outer_index >= 0 && outer_index < outer_map->nparts);
+	outer_merged_index = outer_map->merged_indexes[outer_index];
+	outer_merged = outer_map->merged[outer_index];
+	Assert(inner_index >= 0 && inner_index < inner_map->nparts);
+	inner_merged_index = inner_map->merged_indexes[inner_index];
+	inner_merged = inner_map->merged[inner_index];
+
+	/*
+	 * Handle cases where both partitions are mapped to merged partitions.
+	 */
+	if (outer_merged_index >= 0 && inner_merged_index >= 0)
+	{
+		/*
+		 * If the mereged partitions are the same, no need to do anything;
+		 * return the index of the merged partition.  Otherwise, if both
+		 * partitions are merged with dummy partitions, re-merge them; map
+		 * them to the merged partition with the smaller of the two merged
+		 * indexes and return the smaller index.  Otherwise they can't be
+		 * merged, so return -1.
+		 */
+		if (outer_merged_index == inner_merged_index)
+		{
+			Assert(outer_merged);
+			Assert(inner_merged);
+			return outer_merged_index;
+		}
+		if (!outer_merged && !inner_merged)
+		{
+			/*
+			 * Note that we will fix the larger index that have been added to
+			 * the merged_indexes list so far in fix_merged_indexes().
+			 */
+			if (outer_merged_index < inner_merged_index)
+			{
+				outer_map->merged[outer_index] = true;
+				inner_map->merged_indexes[inner_index] = outer_merged_index;
+				inner_map->merged[inner_index] = true;
+				inner_map->did_remapping = true;
+				inner_map->old_indexes[inner_index] = inner_merged_index;
+				return outer_merged_index;
+			}
+			else
+			{
+				inner_map->merged[inner_index] = true;
+				outer_map->merged_indexes[outer_index] = inner_merged_index;
+				outer_map->merged[outer_index] = true;
+				outer_map->did_remapping = true;
+				outer_map->old_indexes[outer_index] = outer_merged_index;
+				return inner_merged_index;
+			}
+		}
+		return -1;
+	}
+
+	/* At least one partition isn't mapped to a merged partition. */
+	Assert(outer_merged_index == -1 || inner_merged_index == -1);
+
+	/*
+	 * If neither of partitions isn't mapped, assign them a new merged
+	 * partition and return the index of the merged partition.  Otherwise, if
+	 * one of partitions is merged with a dummy relation (and the other isn't
+	 * merged), re-merge it with the other, with the same index, and return
+	 * the index.  Otherwise they can't be merged, so return -1.
+	 */
+	if (outer_merged_index == -1 && inner_merged_index == -1)
+	{
+		int 		merged_index = *next_index;
+
+		Assert(!outer_merged);
+		Assert(!inner_merged);
+		outer_map->merged_indexes[outer_index] = merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged_indexes[inner_index] = merged_index;
+		inner_map->merged[inner_index] = true;
+		*next_index = *next_index + 1;
+		return merged_index;
+	}
+	if (outer_merged_index >= 0 && !outer_map->merged[outer_index])
+	{
+		Assert(inner_merged_index == -1);
+		Assert(!inner_merged);
+		inner_map->merged_indexes[inner_index] = outer_merged_index;
+		inner_map->merged[inner_index] = true;
+		outer_map->merged[outer_index] = true;
+		return outer_merged_index;
+	}
+	if (inner_merged_index >= 0 && !inner_map->merged[inner_index])
+	{
+		Assert(outer_merged_index == -1);
+		Assert(!outer_merged);
+		outer_map->merged_indexes[outer_index] = inner_merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged[inner_index] = true;
+		return inner_merged_index;
+	}
+	return -1;
+}
+
+/*
+ * merge_partition_with_dummy
+ *
+ * *next_index is incremented.
+ */
+static int
+merge_partition_with_dummy(PartitionMap *map, int index, int *next_index)
+{
+	int 		merged_index = *next_index;
+
+	Assert(index >= 0 && index < map->nparts);
+	Assert(map->merged_indexes[index] == -1);
+	Assert(!map->merged[index]);
+	map->merged_indexes[index] = merged_index;
+	/* Leave the merged flag alone! */
+	*next_index = *next_index + 1;
+	return merged_index;
+}
+
+/*
+ * process_outer_partition
+ *
+ * Determine the merged partition associated with the given outer partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given outer partition.
+ */
+static bool
+process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index)
+{
+	Assert(outer_index >= 0);
+
+	/*
+	 * If the inner side has the default partition, the outer partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an outer join, in which case the outer partition has to be
+	 * scanned all the way anyway; if the outer partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the outer partition with a dummy partition.
+	 */
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0);
+
+		/*
+		 * If the outer side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (outer_has_default)
+			return false;
+
+		*merged_index = map_and_merge_partitions(outer_map, inner_map,
+												 outer_index, inner_default,
+												 next_index);
+		if (*merged_index == -1)
+			return false;
+
+		/*
+		 * If this is a FULL join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (*default_index == -1)
+				*default_index = *merged_index;
+			else
+				Assert(*merged_index == *default_index);
+			/* Don't add this index to the list of merged indexes. */
+			*merged_index = -1;
+		}
+	}
+	else
+	{
+		Assert(IS_OUTER_JOIN(jointype));
+		Assert(jointype != JOIN_RIGHT);
+
+		/*
+		 * In range partitioning, if the given outer partition is already
+		 * merged (eg, because we found an overlapping range earlier), we know
+		 * where it fits in the join result; nothing to do in that case.  Else
+		 * create a new merged partition.
+		 */
+		if (outer_map->merged_indexes[outer_index] >= 0)
+		{
+			if (strategy == PARTITION_STRATEGY_LIST)
+				*merged_index = outer_map->merged_indexes[outer_index];
+			else
+			{
+				Assert(strategy == PARTITION_STRATEGY_RANGE);
+				*merged_index = -1;
+			}
+		}
+		else
+			*merged_index = merge_partition_with_dummy(outer_map, outer_index,
+													   next_index);
+	}
+	return true;
+}
+
+/*
+ * process_inner_partition
+ *
+ * Determine the merged partition associated with the given inner partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given inner partition.
+ */
+static bool
+process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index)
+{
+	Assert(inner_index >= 0);
+
+	/*
+	 * If the outer side has the default partition, the inner partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an FULL join, in which case the inner partition has to be
+	 * scanned all the way anyway; if the inner partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the inner partition with a dummy partition.
+	 */
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0);
+
+		/*
+		 * If the inner side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (inner_has_default)
+			return false;
+
+		*merged_index = map_and_merge_partitions(outer_map, inner_map,
+												 outer_default, inner_index,
+												 next_index);
+		if (*merged_index == -1)
+			return false;
+
+		/*
+		 * If this is an outer join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			if (*default_index == -1)
+				*default_index = *merged_index;
+			else
+				Assert(*merged_index == *default_index);
+			/* Don't add this index to the list of merged indexes. */
+			*merged_index = -1;
+		}
+	}
+	else
+	{
+		Assert(jointype == JOIN_FULL);
+
+		/*
+		 * In range partitioning, if the given inner partition is already
+		 * merged (eg, because we found an overlapping range earlier), we know
+		 * where it fits in the join result; nothing to do in that case.  Else
+		 * create a new merged partition.
+		 */
+		if (inner_map->merged_indexes[inner_index] >= 0)
+		{
+			if (strategy == PARTITION_STRATEGY_LIST)
+				*merged_index = inner_map->merged_indexes[inner_index];
+			else
+			{
+				Assert(strategy == PARTITION_STRATEGY_RANGE);
+				*merged_index = -1;
+			}
+		}
+		else
+			*merged_index = merge_partition_with_dummy(inner_map, inner_index,
+													   next_index);
+	}
+	return true;
+}
+
+/*
+ * fix_merged_indexes
+ */
+static void
+fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes)
+{
+	int		   *new_indexes;
+	int			merged_index;
+	int			i;
+	ListCell   *lc;
+
+	new_indexes = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		new_indexes[i] = -1;
+
+	/* Build the mapping of old merged indexes to new merged indexes. */
+	if (outer_map->did_remapping)
+	{
+		for (i = 0; i < outer_map->nparts; i++)
+		{
+			merged_index = outer_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = outer_map->merged_indexes[i];
+		}
+	}
+	if (inner_map->did_remapping)
+	{
+		for (i = 0; i < inner_map->nparts; i++)
+		{
+			merged_index = inner_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = inner_map->merged_indexes[i];
+		}
+	}
+
+	/* Fix the merged_indexes list using the mapping. */
+	foreach(lc, merged_indexes)
+	{
+		merged_index = lfirst_int(lc);
+		Assert(merged_index >= 0);
+		if (new_indexes[merged_index] >= 0)
+			lfirst_int(lc) = new_indexes[merged_index];
+	}
+
+	pfree(new_indexes);
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * This function produces the list pairs of partitions which produce merged
+ * partitions in the order of merged partition indexes.
+ *
+ * nmerged is the number of merged partitions.
+ *
+ * If successful, the list pairs are returned as two separate lists,
+ * outer_part_list and inner_part_list, one for each side.  Otherwise, those
+ * lists will be set to NIL.
+ */
+static void
+generate_matching_part_pairs(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 PartitionMap *outer_map, PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_part_list, List **inner_part_list)
+{
+	int			outer_nparts = outer_map->nparts;
+	int			inner_nparts = inner_map->nparts;
+	int		   *outer_part_array;
+	int		   *inner_part_array;
+	int			max_nparts;
+	int 		i;
+
+	Assert(outer_nparts == outer_rel->nparts);
+	Assert(inner_nparts == inner_rel->nparts);
+
+	outer_part_array = (int *) palloc(sizeof(int) * nmerged);
+	inner_part_array = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		outer_part_array[i] = inner_part_array[i] = -1;
+
+	/* Set pairs of matching partitions. */
+	max_nparts = Max(outer_nparts, inner_nparts);
+	for (i = 0; i < max_nparts; i++)
+	{
+		if (i < outer_nparts)
+		{
+			int 		merged_index = outer_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				outer_part_array[merged_index] = i;
+			}
+		}
+		if (i < inner_nparts)
+		{
+			int 		merged_index = inner_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				inner_part_array[merged_index] = i;
+			}
+		}
+	}
+
+	*outer_part_list = NIL;
+	*inner_part_list = NIL;
+
+	/* Build the list pairs. */
+	for (i = 0; i < nmerged; i++)
+	{
+		int			outer_part = outer_part_array[i];
+		int			inner_part = inner_part_array[i];
+
+		/*
+		 * If both partitions are dummy, it means the merged partition was
+		 * removed by partition re-merging in map_and_merge_partitions();
+		 * ignore the merged partition
+		 */
+		if (outer_part == -1 && inner_part == -1)
+			continue;
+
+		*outer_part_list = lappend(*outer_part_list, outer_part >= 0 ?
+								   outer_rel->part_rels[outer_part] : NULL);
+		*inner_part_list = lappend(*inner_part_list, inner_part >= 0 ?
+								   inner_rel->part_rels[inner_part] : NULL);
+	}
+
+	pfree(outer_part_array);
+	pfree(inner_part_array);
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * Merge default partitions from both sides, if any, and assign the default
+ * partition for the join result, if necessary.
+ *
+ * If both the relations have default partitions, try mapping those to each
+ * other. If the mapping succeeds corresponding merged partition will act as
+ * the default partition of the join result.
+ *
+ * If inner side of the join has default but not the outer side, rows in it
+ * won't appear in the join result. So don't create a default partition. If
+ * outer side of the join has default but not the inner side, rows in it will
+ * appear in the join result, so create a default merged partition.
+ */
+static bool
+merge_default_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 bool outer_has_default, int outer_default,
+						 bool inner_has_default, int inner_default,
+						 JoinType jointype, int *next_index,
+						 int *default_index)
+{
+	Assert(outer_has_default || inner_has_default);
+
+	if (outer_has_default && !inner_has_default)
+	{
+		if (IS_OUTER_JOIN(jointype))
+		{
+			int			merged_index;
+
+			Assert(jointype != JOIN_RIGHT);
+			Assert(outer_default >= 0 && outer_default < outer_map->nparts);
+			merged_index = outer_map->merged_indexes[outer_default];
+			if (merged_index == -1)
+			{
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(outer_map,
+															outer_default,
+															next_index);
+			}
+			else
+				Assert(*default_index == merged_index);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else if (!outer_has_default && inner_has_default)
+	{
+		if (jointype == JOIN_FULL)
+		{
+			int			merged_index;
+
+			Assert(inner_default >= 0 && inner_default < inner_map->nparts);
+			merged_index = inner_map->merged_indexes[inner_default];
+			if (merged_index == -1)
+			{
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(inner_map,
+															inner_default,
+															next_index);
+			}
+			else
+				Assert(*default_index == merged_index);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		*default_index = map_and_merge_partitions(outer_map,
+												  inner_map,
+												  outer_default,
+												  inner_default,
+												  next_index);
+		if (*default_index == -1)
+			return false;
+	}
+
+	return true;
+}
+
+/*
+ * merge_null_partitions
+ *
+ * Merge NULL partitions, i.e. a partition that can hold NULL values for a list
+ * partitioned table, if any. Find the index of merged partition to which the
+ * NULL values would belong in the join result. If one joining relation has a
+ * NULL partition but not the other, try matching it with the default partition
+ * from the other relation since the default partition may have rows with NULL
+ * partition key. We can eliminate a NULL partition when it appears only on the
+ * inner side of the join and the outer side doesn't have a default partition.
+ *
+ * When the equality operator used for join is strict, two NULL values will not
+ * be considered as equal, and thus a NULL partition can be eliminated for an
+ * inner join. But we don't check the strictness operator here.
+ */
+static bool
+merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_map, PartitionMap *inner_map,
+					  bool outer_has_default, bool inner_has_default,
+					  bool outer_has_null, bool inner_has_null,
+					  JoinType jointype, int *next_index,
+					  int *default_index, int *null_index)
+{
+	Assert(outer_has_null || inner_has_null);
+	Assert(*null_index == -1);
+
+	if (outer_has_null && !inner_has_null)
+	{
+		int			merged_index = -1;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join to which the NULL partition matches will
+		 * contain the NULL values and thus become the NULL partition of the
+		 * the join.
+		 */
+		if (inner_has_default || IS_OUTER_JOIN(jointype))
+		{
+			if (!process_outer_partition(outer_map,
+										 inner_map,
+										 outer_has_default,
+										 inner_has_default,
+										 outer_bi->null_index,
+										 inner_bi->default_index,
+										 jointype,
+										 outer_bi->strategy,
+										 next_index,
+										 default_index,
+										 &merged_index))
+				return false;
+		}
+		*null_index = merged_index;
+	}
+	else if (!outer_has_null && inner_has_null)
+	{
+		int			merged_index = -1;
+
+		/*
+		 * If the NULL partition was missing from the outer side of the join,
+		 * the partition of the join to which the NULL partition matches will
+		 * contain the NULL values and thus become the NULL partition of the
+		 * the join.
+		 */
+		if (outer_has_default || jointype == JOIN_FULL)
+		{
+			if (!process_inner_partition(outer_map,
+										 inner_map,
+										 outer_has_default,
+										 inner_has_default,
+										 inner_bi->null_index,
+										 outer_bi->default_index,
+										 jointype,
+										 outer_bi->strategy,
+										 next_index,
+										 default_index,
+										 &merged_index))
+				return false;
+		}
+		*null_index = merged_index;
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		*null_index = map_and_merge_partitions(outer_map,
+											   inner_map,
+											   outer_bi->null_index,
+											   inner_bi->null_index,
+											   next_index);
+		if (*null_index == -1)
+			return false;
+	}
+
+	return true;
+}
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index 23a06d718e..e06eb9aaae 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -577,8 +577,10 @@ typedef struct PartitionSchemeData *PartitionScheme;
  *		part_scheme - Partitioning scheme of the relation
  *		nparts - Number of partitions
  *		boundinfo - Partition bounds
+ *		merged - true if partition bounds are merged ones
  *		partition_qual - Partition constraint if not the root
  *		part_rels - RelOptInfos for each partition
+ *		all_partrels - Relids set of all partition relids
  *		partexprs, nullable_partexprs - Partition key expressions
  *		partitioned_child_rels - RT indexes of unpruned partitions of
  *								 this relation that are partitioned tables
@@ -718,9 +720,12 @@ typedef struct RelOptInfo
 	PartitionScheme part_scheme;	/* Partitioning scheme. */
 	int			nparts;			/* number of partitions */
 	struct PartitionBoundInfoData *boundinfo;	/* Partition bounds */
+	bool		merged;			/* true if partition bounds were created by
+								 * partition_bounds_merge() */
 	List	   *partition_qual; /* partition constraint */
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of partitions,
 									 * stored in the same order of bounds */
+	Relids		all_partrels;	/* Relids set of all partition relids */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 	List	   *partitioned_child_rels; /* List of RT indexes. */
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index 0d0fd42b18..9292aa11e5 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -16,6 +16,7 @@
 #include "nodes/pg_list.h"
 #include "partitioning/partdefs.h"
 #include "utils/relcache.h"
+struct RelOptInfo;				/* avoid including pathnodes.h here */
 
 
 /*
@@ -108,5 +109,11 @@ extern int	partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 										  int nvalues, Datum *values, bool *is_equal);
 extern int	partition_hash_bsearch(PartitionBoundInfo boundinfo,
 								   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   struct RelOptInfo *outer_rel, struct RelOptInfo *inner_rel,
+					   JoinType jointype,
+					   List **outer_parts, List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 975bf6765c..1675abb1e7 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -8,59 +8,86 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-(4 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+(7 rows)
 
 -- left outer join, with whole-row reference; partitionwise join does not apply
 EXPLAIN (COSTS OFF)
@@ -72,35 +99,50 @@ SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER
    ->  Hash Right Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(22 rows)
 
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-      t1      |      t2      
---------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
- (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
- (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
- (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
- (550,0,0550) | 
-(12 rows)
+       t1       |       t2       
+----------------+----------------
+ (-250,0,-0250) | 
+ (-200,0,-0200) | 
+ (-150,0,-0150) | (0,-150,-0150)
+ (-100,0,-0100) | 
+ (-50,0,-0050)  | 
+ (0,0,0000)     | (0,0,0000)
+ (50,0,0050)    | 
+ (100,0,0100)   | 
+ (150,0,0150)   | (0,150,0150)
+ (200,0,0200)   | 
+ (250,0,0250)   | 
+ (300,0,0300)   | (0,300,0300)
+ (350,0,0350)   | 
+ (400,0,0400)   | 
+ (450,0,0450)   | (0,450,0450)
+ (500,0,0500)   | 
+ (550,0,0550)   | 
+ (600,0,0600)   | (0,600,0600)
+ (650,0,0650)   | 
+ (700,0,0700)   | 
+ (750,0,0750)   | (0,750,0750)
+(21 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
@@ -112,35 +154,53 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
    ->  Append
          ->  Hash Right Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Right Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t2_2.b)
-(20 rows)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+      |       | -225 | -0225
+      |       |  -75 | -0075
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(14 rows)
 
 -- full outer join, with placeholder vars
 EXPLAIN (COSTS OFF)
@@ -148,8 +208,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                             QUERY PLAN                            
 ------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               Filter: (((50) = prt1_p0.a) OR ((75) = prt2_p0.b))
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
@@ -174,7 +242,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               Filter: (((50) = prt1_p4.a) OR ((75) = prt2_p4.b))
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
@@ -211,35 +287,44 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Hash Right Join
-         Hash Cond: (prt2_p2.b = prt1_p1.a)
+         Hash Cond: (prt2_p2.b = prt1_p0.a)
          ->  Append
                ->  Seq Scan on prt2_p2
                      Filter: (b > 250)
                ->  Seq Scan on prt2_p3
                      Filter: (b > 250)
+               ->  Seq Scan on prt2_p4
+                     Filter: (b > 250)
          ->  Hash
                ->  Append
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p1
                            Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p2
                            Filter: ((a < 450) AND (b = 0))
-(15 rows)
+(19 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-(9 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+(14 rows)
 
 -- Currently we can't do partitioned join if nullable-side partitions are pruned
 EXPLAIN (COSTS OFF)
@@ -247,11 +332,13 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                       QUERY PLAN                      
 ------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Hash Full Join
-         Hash Cond: (prt1_p1.a = prt2_p2.b)
-         Filter: ((prt1_p1.b = 0) OR (prt2_p2.a = 0))
+         Hash Cond: (prt1_p0.a = prt2_p2.b)
+         Filter: ((prt1_p0.b = 0) OR (prt2_p2.a = 0))
          ->  Append
+               ->  Seq Scan on prt1_p0
+                     Filter: (a < 450)
                ->  Seq Scan on prt1_p1
                      Filter: (a < 450)
                ->  Seq Scan on prt1_p2
@@ -262,64 +349,147 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                            Filter: (b > 250)
                      ->  Seq Scan on prt2_p3
                            Filter: (b > 250)
-(16 rows)
+                     ->  Seq Scan on prt2_p4
+                           Filter: (b > 250)
+(20 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
-(12 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+      |       | 375 | 0375
+      |       | 450 | 0450
+      |       | 525 | 0525
+      |       | 600 | 0600
+      |       | 675 | 0675
+      |       | 750 | 0750
+(20 rows)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Semi Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Semi Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                            Filter: (a = 0)
-         ->  Nested Loop Semi Join
-               Join Filter: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
-               ->  Materialize
-                     ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
-(24 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt1_p3 t2_3
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+                           Filter: (b = 0)
+(37 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
 
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
@@ -330,27 +500,82 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
    ->  Append
          ->  Hash Anti Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Hash Anti Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Hash Anti Join
                Hash Cond: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
-(17 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(27 rows)
 
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
-  sum  |         avg          | sum  |         avg         
--------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+  sum  |         avg          | sum  |        avg         
+-------+----------------------+------+--------------------
+ 95550 | 273.0000000000000000 | 2200 | 6.2857142857142857
 (1 row)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Append
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p0 t1
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+               Index Cond: (a = t1.b)
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.b = t2_1.a)
+         ->  Seq Scan on prt2_p1 t1_1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p1 t2_1
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p2 t1_2
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+               Index Cond: (a = t1_2.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p3 t1_3
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+               Index Cond: (a = t1_3.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p4 t1_4
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+               Index Cond: (a = t1_4.b)
+(27 rows)
+
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+  b   |   c   
+------+-------
+ -225 | -0225
+  -75 | -0075
+   75 | 0075
+  225 | 0225
+  375 | 0375
+  525 | 0525
+  675 | 0675
+(7 rows)
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -362,49 +587,74 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
+                     ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
                            Index Cond: (a = t1.a)
-                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t3
                            Index Cond: (b = t2.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_1
+                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
                            Index Cond: (a = t1_1.a)
-                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3_1
                            Index Cond: (b = t2_1.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_2
+                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
                            Index Cond: (a = t1_2.a)
-                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_2
                            Index Cond: (b = t2_2.a)
-(27 rows)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                           Index Cond: (a = t1_3.a)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_3
+                           Index Cond: (b = t2_3.a)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                           Index Cond: (a = t1_4.a)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t3_4
+                           Index Cond: (b = t2_4.a)
+(43 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   | t2a | t3a | least 
------+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
- 100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
- 250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
- 400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
- 550 | 0 | 0550 |     |     |      
-(12 rows)
+  a   | b |   c   | t2a  | t3a | least 
+------+---+-------+------+-----+-------
+ -250 | 0 | -0250 |      |     |      
+ -200 | 0 | -0200 |      |     |      
+ -150 | 0 | -0150 | -150 |   0 |  -150
+ -100 | 0 | -0100 |      |     |      
+  -50 | 0 | -0050 |      |     |      
+    0 | 0 | 0000  |    0 |   0 |     0
+   50 | 0 | 0050  |      |     |      
+  100 | 0 | 0100  |      |     |      
+  150 | 0 | 0150  |  150 |   0 |   150
+  200 | 0 | 0200  |      |     |      
+  250 | 0 | 0250  |      |     |      
+  300 | 0 | 0300  |  300 |   0 |   300
+  350 | 0 | 0350  |      |     |      
+  400 | 0 | 0400  |      |     |      
+  450 | 0 | 0450  |  450 |   0 |   450
+  500 | 0 | 0500  |      |     |      
+  550 | 0 | 0550  |      |     |      
+  600 | 0 | 0600  |  600 |   0 |   600
+  650 | 0 | 0650  |      |     |      
+  700 | 0 | 0700  |      |     |      
+  750 | 0 | 0750  |  750 |   0 |   750
+(21 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
@@ -418,46 +668,67 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
          Hash Cond: ((t1.c)::text = (t2.c)::text)
          Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Seq Scan on prt1_p1 t1
-               ->  Seq Scan on prt1_p2 t1_1
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
          ->  Hash
                ->  Append
                      ->  Hash Join
                            Hash Cond: (t2.a = t3.b)
-                           ->  Seq Scan on prt1_p1 t2
+                           ->  Seq Scan on prt1_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t3
+                                 ->  Seq Scan on prt2_p0 t3
                      ->  Hash Join
                            Hash Cond: (t2_1.a = t3_1.b)
-                           ->  Seq Scan on prt1_p2 t2_1
+                           ->  Seq Scan on prt1_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t3_1
+                                 ->  Seq Scan on prt2_p1 t3_1
                      ->  Hash Join
                            Hash Cond: (t2_2.a = t3_2.b)
-                           ->  Seq Scan on prt1_p3 t2_2
+                           ->  Seq Scan on prt1_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p3 t3_2
-(26 rows)
+                                 ->  Seq Scan on prt2_p2 t3_2
+                     ->  Hash Join
+                           Hash Cond: (t2_3.a = t3_3.b)
+                           ->  Seq Scan on prt1_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t3_3
+                     ->  Hash Join
+                           Hash Cond: (t2_4.a = t3_4.b)
+                           ->  Seq Scan on prt1_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t3_4
+(38 rows)
 
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, t2.b t2b, t2.c t2c, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
-  a  | t2a | t2c  
------+-----+------
-   0 |   0 | 0000
-  50 |     | 
- 100 |     | 
- 150 | 150 | 0150
- 200 |     | 
- 250 |     | 
- 300 | 300 | 0300
- 350 |     | 
- 400 |     | 
- 450 | 450 | 0450
- 500 |     | 
- 550 |     | 
-(12 rows)
+  a   | t2a  |  t2c  
+------+------+-------
+ -250 |      | 
+ -200 |      | 
+ -150 | -150 | -0150
+ -100 |      | 
+  -50 |      | 
+    0 |    0 | 0000
+   50 |      | 
+  100 |      | 
+  150 |  150 | 0150
+  200 |      | 
+  250 |      | 
+  300 |  300 | 0300
+  350 |      | 
+  400 |      | 
+  450 |  450 | 0450
+  500 |      | 
+  550 |      | 
+  600 |  600 | 0600
+  650 |      | 
+  700 |      | 
+  750 |  750 | 0750
+(21 rows)
 
 -- bug with inadequate sort key representation
 SET enable_partitionwise_aggregate TO true;
@@ -469,49 +740,75 @@ SELECT a, b FROM prt1 FULL JOIN prt2 p2(b,a,c) USING(a,b)
                                                     QUERY PLAN                                                     
 -------------------------------------------------------------------------------------------------------------------
  Group
-   Group Key: (COALESCE(prt1_p1.a, p2.a)), (COALESCE(prt1_p1.b, p2.b))
+   Group Key: (COALESCE(prt1_p0.a, p2.a)), (COALESCE(prt1_p0.b, p2.b))
    ->  Merge Append
-         Sort Key: (COALESCE(prt1_p1.a, p2.a)), (COALESCE(prt1_p1.b, p2.b))
+         Sort Key: (COALESCE(prt1_p0.a, p2.a)), (COALESCE(prt1_p0.b, p2.b))
+         ->  Group
+               Group Key: (COALESCE(prt1_p0.a, p2.a)), (COALESCE(prt1_p0.b, p2.b))
+               ->  Sort
+                     Sort Key: (COALESCE(prt1_p0.a, p2.a)), (COALESCE(prt1_p0.b, p2.b))
+                     ->  Merge Full Join
+                           Merge Cond: ((prt1_p0.a = p2.a) AND (prt1_p0.b = p2.b))
+                           Filter: ((COALESCE(prt1_p0.a, p2.a) >= 490) AND (COALESCE(prt1_p0.a, p2.a) <= 510))
+                           ->  Sort
+                                 Sort Key: prt1_p0.a, prt1_p0.b
+                                 ->  Seq Scan on prt1_p0
+                           ->  Sort
+                                 Sort Key: p2.a, p2.b
+                                 ->  Seq Scan on prt2_p0 p2
          ->  Group
-               Group Key: (COALESCE(prt1_p1.a, p2.a)), (COALESCE(prt1_p1.b, p2.b))
+               Group Key: (COALESCE(prt1_p1.a, p2_1.a)), (COALESCE(prt1_p1.b, p2_1.b))
                ->  Sort
-                     Sort Key: (COALESCE(prt1_p1.a, p2.a)), (COALESCE(prt1_p1.b, p2.b))
+                     Sort Key: (COALESCE(prt1_p1.a, p2_1.a)), (COALESCE(prt1_p1.b, p2_1.b))
                      ->  Merge Full Join
-                           Merge Cond: ((prt1_p1.a = p2.a) AND (prt1_p1.b = p2.b))
-                           Filter: ((COALESCE(prt1_p1.a, p2.a) >= 490) AND (COALESCE(prt1_p1.a, p2.a) <= 510))
+                           Merge Cond: ((prt1_p1.a = p2_1.a) AND (prt1_p1.b = p2_1.b))
+                           Filter: ((COALESCE(prt1_p1.a, p2_1.a) >= 490) AND (COALESCE(prt1_p1.a, p2_1.a) <= 510))
                            ->  Sort
                                  Sort Key: prt1_p1.a, prt1_p1.b
                                  ->  Seq Scan on prt1_p1
                            ->  Sort
-                                 Sort Key: p2.a, p2.b
-                                 ->  Seq Scan on prt2_p1 p2
+                                 Sort Key: p2_1.a, p2_1.b
+                                 ->  Seq Scan on prt2_p1 p2_1
          ->  Group
-               Group Key: (COALESCE(prt1_p2.a, p2_1.a)), (COALESCE(prt1_p2.b, p2_1.b))
+               Group Key: (COALESCE(prt1_p2.a, p2_2.a)), (COALESCE(prt1_p2.b, p2_2.b))
                ->  Sort
-                     Sort Key: (COALESCE(prt1_p2.a, p2_1.a)), (COALESCE(prt1_p2.b, p2_1.b))
+                     Sort Key: (COALESCE(prt1_p2.a, p2_2.a)), (COALESCE(prt1_p2.b, p2_2.b))
                      ->  Merge Full Join
-                           Merge Cond: ((prt1_p2.a = p2_1.a) AND (prt1_p2.b = p2_1.b))
-                           Filter: ((COALESCE(prt1_p2.a, p2_1.a) >= 490) AND (COALESCE(prt1_p2.a, p2_1.a) <= 510))
+                           Merge Cond: ((prt1_p2.a = p2_2.a) AND (prt1_p2.b = p2_2.b))
+                           Filter: ((COALESCE(prt1_p2.a, p2_2.a) >= 490) AND (COALESCE(prt1_p2.a, p2_2.a) <= 510))
                            ->  Sort
                                  Sort Key: prt1_p2.a, prt1_p2.b
                                  ->  Seq Scan on prt1_p2
                            ->  Sort
-                                 Sort Key: p2_1.a, p2_1.b
-                                 ->  Seq Scan on prt2_p2 p2_1
+                                 Sort Key: p2_2.a, p2_2.b
+                                 ->  Seq Scan on prt2_p2 p2_2
          ->  Group
-               Group Key: (COALESCE(prt1_p3.a, p2_2.a)), (COALESCE(prt1_p3.b, p2_2.b))
+               Group Key: (COALESCE(prt1_p3.a, p2_3.a)), (COALESCE(prt1_p3.b, p2_3.b))
                ->  Sort
-                     Sort Key: (COALESCE(prt1_p3.a, p2_2.a)), (COALESCE(prt1_p3.b, p2_2.b))
+                     Sort Key: (COALESCE(prt1_p3.a, p2_3.a)), (COALESCE(prt1_p3.b, p2_3.b))
                      ->  Merge Full Join
-                           Merge Cond: ((prt1_p3.a = p2_2.a) AND (prt1_p3.b = p2_2.b))
-                           Filter: ((COALESCE(prt1_p3.a, p2_2.a) >= 490) AND (COALESCE(prt1_p3.a, p2_2.a) <= 510))
+                           Merge Cond: ((prt1_p3.a = p2_3.a) AND (prt1_p3.b = p2_3.b))
+                           Filter: ((COALESCE(prt1_p3.a, p2_3.a) >= 490) AND (COALESCE(prt1_p3.a, p2_3.a) <= 510))
                            ->  Sort
                                  Sort Key: prt1_p3.a, prt1_p3.b
                                  ->  Seq Scan on prt1_p3
                            ->  Sort
-                                 Sort Key: p2_2.a, p2_2.b
-                                 ->  Seq Scan on prt2_p3 p2_2
-(43 rows)
+                                 Sort Key: p2_3.a, p2_3.b
+                                 ->  Seq Scan on prt2_p3 p2_3
+         ->  Group
+               Group Key: (COALESCE(prt1_p4.a, p2_4.a)), (COALESCE(prt1_p4.b, p2_4.b))
+               ->  Sort
+                     Sort Key: (COALESCE(prt1_p4.a, p2_4.a)), (COALESCE(prt1_p4.b, p2_4.b))
+                     ->  Merge Full Join
+                           Merge Cond: ((prt1_p4.a = p2_4.a) AND (prt1_p4.b = p2_4.b))
+                           Filter: ((COALESCE(prt1_p4.a, p2_4.a) >= 490) AND (COALESCE(prt1_p4.a, p2_4.a) <= 510))
+                           ->  Sort
+                                 Sort Key: prt1_p4.a, prt1_p4.b
+                                 ->  Seq Scan on prt1_p4
+                           ->  Sort
+                                 Sort Key: p2_4.a, p2_4.b
+                                 ->  Seq Scan on prt2_p4 p2_4
+(69 rows)
 
 SELECT a, b FROM prt1 FULL JOIN prt2 p2(b,a,c) USING(a,b)
   WHERE a BETWEEN 490 AND 510
@@ -540,19 +837,29 @@ RESET enable_hashjoin;
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
@@ -563,32 +870,49 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
    ->  Append
          ->  Hash Join
                Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
-               ->  Seq Scan on prt2_e_p1 t2
+               ->  Seq Scan on prt2_e_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 t1
+                     ->  Seq Scan on prt1_e_p0 t1
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
-               ->  Seq Scan on prt2_e_p2 t2_1
+               Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+               ->  Seq Scan on prt1_e_p1 t1_1
+                     Filter: (c = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 t1_1
-                           Filter: (c = 0)
+                     ->  Seq Scan on prt2_e_p1 t2_1
          ->  Hash Join
                Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
-               ->  Seq Scan on prt2_e_p3 t2_2
+               ->  Seq Scan on prt2_e_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 t1_2
+                     ->  Seq Scan on prt1_e_p2 t1_2
                            Filter: (c = 0)
-(21 rows)
+         ->  Hash Join
+               Hash Cond: (((t2_3.b + t2_3.a) / 2) = ((t1_3.a + t1_3.b) / 2))
+               ->  Seq Scan on prt2_e_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p3 t1_3
+                           Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (((t2_4.b + t2_4.a) / 2) = ((t1_4.a + t1_4.b) / 2))
+               ->  Seq Scan on prt2_e_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4 t1_4
+                           Filter: (c = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
- 150 | 0 | 150 | 0
- 300 | 0 | 300 | 0
- 450 | 0 | 450 | 0
-(4 rows)
+  a   | c |  b   | c 
+------+---+------+---
+ -250 | 0 | -250 | 0
+ -100 | 0 | -100 | 0
+    0 | 0 |    0 | 0
+    0 | 0 |    0 | 0
+  150 | 0 |  150 | 0
+  300 | 0 |  300 | 0
+  450 | 0 |  450 | 0
+  600 | 0 |  600 | 0
+  750 | 0 |  750 | 0
+(9 rows)
 
 --
 -- N-way join
@@ -601,154 +925,232 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = ((t3.a + t3.b) / 2))
+               Join Filter: (t1.a = t2.b)
                ->  Hash Join
-                     Hash Cond: (t2.b = t1.a)
-                     ->  Seq Scan on prt2_p1 t2
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on prt1_e_p0 t3
                      ->  Hash
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                     Index Cond: (((a + b) / 2) = t2.b)
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
+                     Index Cond: (b = ((t3.a + t3.b) / 2))
          ->  Nested Loop
                Join Filter: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_1.b = t1_1.a)
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
+               ->  Index Scan using iprt1_e_p4_ab2 on prt1_e_p1 t3_1
                      Index Cond: (((a + b) / 2) = t2_1.b)
          ->  Nested Loop
                Join Filter: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_2.b = t1_2.a)
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
+               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_2
                      Index Cond: (((a + b) / 2) = t2_2.b)
-(33 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = ((t3_3.a + t3_3.b) / 2))
+               ->  Nested Loop
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t2_3.b)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t2_4.b)
+               ->  Hash Join
+                     Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                     ->  Seq Scan on prt1_e_p4 t3_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_p4 t1_4
+                                 Filter: (b = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = ((t3_4.a + t3_4.b) / 2))
+(52 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
-(4 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(8 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                          QUERY PLAN                          
---------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Hash Right Join
                Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
-               ->  Seq Scan on prt1_e_p1 t3
+               ->  Seq Scan on prt1_e_p0 t3
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2.b = t1.a)
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_1.a)
-               ->  Seq Scan on prt1_e_p2 t3_1
+               ->  Seq Scan on prt1_e_p1 t3_1
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_1.b = t1_1.a)
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_2.a)
-               ->  Seq Scan on prt1_e_p3 t3_2
+               ->  Seq Scan on prt1_e_p2 t3_2
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_2.b = t1_2.a)
-                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                                        Filter: (b = 0)
-(33 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+               ->  Seq Scan on prt1_e_p4 t3_4
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_4.b = t1_4.a)
+                           ->  Seq Scan on prt2_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_p4 t1_4
+                                       Filter: (b = 0)
+(51 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                            QUERY PLAN                             
--------------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p1 t3
+                           ->  Seq Scan on prt1_e_p0 t3
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p1_b on prt2_p1 t2
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
                      Index Cond: (b = t1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p2 t3_1
+                           ->  Seq Scan on prt1_e_p1 t3_1
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
+               ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
                      Index Cond: (b = t1_1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p3 t3_2
+                           ->  Seq Scan on prt1_e_p2 t3_2
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
                      Index Cond: (b = t1_2.a)
-(30 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_e_p3 t3_3
+                           Filter: (c = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                           Index Cond: (a = ((t3_3.a + t3_3.b) / 2))
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Nested Loop Left Join
+               ->  Hash Right Join
+                     Hash Cond: (t1_4.a = ((t3_4.a + t3_4.b) / 2))
+                     ->  Seq Scan on prt1_p4 t1_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p4 t3_4
+                                 Filter: (c = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = t1_4.a)
+(47 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -757,21 +1159,34 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                                                    QUERY PLAN                                                   
 ----------------------------------------------------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   Sort Key: prt1_p0.a, prt2_p0.b, ((prt1_e_p0.a + prt1_e_p0.b))
    ->  Append
          ->  Hash Full Join
-               Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
-               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               Hash Cond: (prt1_p0.a = ((prt1_e_p0.a + prt1_e_p0.b) / 2))
+               Filter: ((prt1_p0.a = (50)) OR (prt2_p0.b = (75)) OR (((prt1_e_p0.a + prt1_e_p0.b) / 2) = (50)))
                ->  Hash Full Join
-                     Hash Cond: (prt1_p1.a = prt2_p1.b)
-                     ->  Seq Scan on prt1_p1
+                     Hash Cond: (prt1_p0.a = prt2_p0.b)
+                     ->  Seq Scan on prt1_p0
                            Filter: (b = 0)
                      ->  Hash
-                           ->  Seq Scan on prt2_p1
+                           ->  Seq Scan on prt2_p0
                                  Filter: (a = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1
+                     ->  Seq Scan on prt1_e_p0
                            Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = prt1_p1.a)
+               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               ->  Seq Scan on prt1_e_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on prt1_p1
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1
+                                       Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
                Filter: ((prt1_p2.a = (50)) OR (prt2_p2.b = (75)) OR (((prt1_e_p2.a + prt1_e_p2.b) / 2) = (50)))
@@ -798,7 +1213,20 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                ->  Hash
                      ->  Seq Scan on prt1_e_p3
                            Filter: (c = 0)
-(42 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = ((prt1_e_p4.a + prt1_e_p4.b) / 2))
+               Filter: ((prt1_p4.a = (50)) OR (prt2_p4.b = (75)) OR (((prt1_e_p4.a + prt1_e_p4.b) / 2) = (50)))
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = prt2_p4.b)
+                     ->  Seq Scan on prt1_p4
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4
+                                 Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4
+                           Filter: (c = 0)
+(68 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
@@ -816,172 +1244,260 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = t1_3.b)
+               Join Filter: (t1.a = t1_5.b)
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Join
-                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
-                           ->  Seq Scan on prt1_e_p1 t2
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t1_3
+                                 ->  Seq Scan on prt2_p0 t1_5
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
                      Index Cond: (a = ((t2.a + t2.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_1.a = t1_4.b)
+               Join Filter: (t1_1.a = t1_6.b)
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Join
-                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
-                           ->  Seq Scan on prt1_e_p2 t2_1
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_6.b)
+                           ->  Seq Scan on prt1_e_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t1_4
+                                 ->  Seq Scan on prt2_p1 t1_6
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
                      Index Cond: (a = ((t2_1.a + t2_1.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_2.a = t1_5.b)
+               Join Filter: (t1_2.a = t1_7.b)
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
+                           ->  Seq Scan on prt2_p2 t1_7
                                  Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
+                           ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t2_2
+                                 Index Cond: (((a + b) / 2) = t1_7.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t1_8.b)
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Nested Loop
+                           ->  Seq Scan on prt2_p3 t1_8
+                                 Filter: (a = 0)
+                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_3
+                                 Index Cond: (((a + b) / 2) = t1_8.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = ((t2_3.a + t2_3.b) / 2))
+                     Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t1_9.b)
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Join
+                           Hash Cond: (((t2_4.a + t2_4.b) / 2) = t1_9.b)
+                           ->  Seq Scan on prt1_e_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t1_9
+                                       Filter: (a = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = ((t2_4.a + t2_4.b) / 2))
+                     Filter: (b = 0)
+(66 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                               QUERY PLAN                                
--------------------------------------------------------------------------
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
-                           ->  Seq Scan on prt2_p1 t1_3
+                           Hash Cond: (t1_5.b = ((t1_10.a + t1_10.b) / 2))
+                           ->  Seq Scan on prt2_p0 t1_5
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t1_6
+                                 ->  Seq Scan on prt1_e_p0 t1_10
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
-                     Index Cond: (a = t1_3.b)
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t1_5.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
-                           ->  Seq Scan on prt2_p2 t1_4
+                           Hash Cond: (t1_6.b = ((t1_11.a + t1_11.b) / 2))
+                           ->  Seq Scan on prt2_p1 t1_6
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t1_7
+                                 ->  Seq Scan on prt1_e_p1 t1_11
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
-                     Index Cond: (a = t1_4.b)
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
+                     Index Cond: (a = t1_6.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
-                           ->  Seq Scan on prt2_p3 t1_5
+                           Hash Cond: (t1_7.b = ((t1_12.a + t1_12.b) / 2))
+                           ->  Seq Scan on prt2_p2 t1_7
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p3 t1_8
+                                 ->  Seq Scan on prt1_e_p2 t1_12
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t1_5.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t1_7.b)
                      Filter: (b = 0)
-(39 rows)
-
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_8.b = ((t1_13.a + t1_13.b) / 2))
+                           ->  Seq Scan on prt2_p3 t1_8
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p3 t1_13
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t1_8.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_9.b = ((t1_14.a + t1_14.b) / 2))
+                           ->  Seq Scan on prt2_p4 t1_9
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t1_14
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = t1_9.b)
+                     Filter: (b = 0)
+(63 rows)
+
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 -- test merge joins
 SET enable_hashjoin TO off;
 SET enable_nestloop TO off;
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                           QUERY PLAN                           
-----------------------------------------------------------------
+                            QUERY PLAN                            
+------------------------------------------------------------------
  Merge Append
    Sort Key: t1.a
    ->  Merge Semi Join
-         Merge Cond: (t1.a = t1_3.b)
+         Merge Cond: (t1.a = t1_5.b)
          ->  Sort
                Sort Key: t1.a
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+               Merge Cond: (t1_5.b = (((t1_10.a + t1_10.b) / 2)))
                ->  Sort
-                     Sort Key: t1_3.b
-                     ->  Seq Scan on prt2_p1 t1_3
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on prt2_p0 t1_5
                ->  Sort
-                     Sort Key: (((t1_6.a + t1_6.b) / 2))
-                     ->  Seq Scan on prt1_e_p1 t1_6
+                     Sort Key: (((t1_10.a + t1_10.b) / 2))
+                     ->  Seq Scan on prt1_e_p0 t1_10
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_1.a = t1_4.b)
+         Merge Cond: (t1_1.a = t1_6.b)
          ->  Sort
                Sort Key: t1_1.a
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+               Merge Cond: (t1_6.b = (((t1_11.a + t1_11.b) / 2)))
                ->  Sort
-                     Sort Key: t1_4.b
-                     ->  Seq Scan on prt2_p2 t1_4
+                     Sort Key: t1_6.b
+                     ->  Seq Scan on prt2_p1 t1_6
                ->  Sort
-                     Sort Key: (((t1_7.a + t1_7.b) / 2))
-                     ->  Seq Scan on prt1_e_p2 t1_7
+                     Sort Key: (((t1_11.a + t1_11.b) / 2))
+                     ->  Seq Scan on prt1_e_p1 t1_11
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_2.a = t1_5.b)
+         Merge Cond: (t1_2.a = t1_7.b)
          ->  Sort
                Sort Key: t1_2.a
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+               Merge Cond: (t1_7.b = (((t1_12.a + t1_12.b) / 2)))
                ->  Sort
-                     Sort Key: t1_5.b
-                     ->  Seq Scan on prt2_p3 t1_5
+                     Sort Key: t1_7.b
+                     ->  Seq Scan on prt2_p2 t1_7
                ->  Sort
-                     Sort Key: (((t1_8.a + t1_8.b) / 2))
-                     ->  Seq Scan on prt1_e_p3 t1_8
+                     Sort Key: (((t1_12.a + t1_12.b) / 2))
+                     ->  Seq Scan on prt1_e_p2 t1_12
                            Filter: (c = 0)
-(47 rows)
+   ->  Merge Semi Join
+         Merge Cond: (t1_3.a = t1_8.b)
+         ->  Sort
+               Sort Key: t1_3.a
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_8.b = (((t1_13.a + t1_13.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_8.b
+                     ->  Seq Scan on prt2_p3 t1_8
+               ->  Sort
+                     Sort Key: (((t1_13.a + t1_13.b) / 2))
+                     ->  Seq Scan on prt1_e_p3 t1_13
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_4.a = t1_9.b)
+         ->  Sort
+               Sort Key: t1_4.a
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_9.b = (((t1_14.a + t1_14.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_9.b
+                     ->  Seq Scan on prt2_p4 t1_9
+               ->  Sort
+                     Sort Key: (((t1_14.a + t1_14.b) / 2))
+                     ->  Seq Scan on prt1_e_p4 t1_14
+                           Filter: (c = 0)
+(77 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
@@ -998,14 +1514,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
                            ->  Sort
                                  Sort Key: (((t3.a + t3.b) / 2))
-                                 ->  Seq Scan on prt1_e_p1 t3
+                                 ->  Seq Scan on prt1_e_p0 t3
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1.a
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                ->  Sort
                      Sort Key: t2.b
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Merge Left Join
                Merge Cond: (t1_1.a = t2_1.b)
                ->  Sort
@@ -1014,14 +1530,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
                            ->  Sort
                                  Sort Key: (((t3_1.a + t3_1.b) / 2))
-                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                 ->  Seq Scan on prt1_e_p1 t3_1
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_1.a
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                ->  Sort
                      Sort Key: t2_1.b
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Merge Left Join
                Merge Cond: (t1_2.a = t2_2.b)
                ->  Sort
@@ -1030,32 +1546,74 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
                            ->  Sort
                                  Sort Key: (((t3_2.a + t3_2.b) / 2))
-                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                 ->  Seq Scan on prt1_e_p2 t3_2
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_2.a
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                ->  Sort
                      Sort Key: t2_2.b
-                     ->  Seq Scan on prt2_p3 t2_2
-(51 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Merge Left Join
+               Merge Cond: (t1_3.a = t2_3.b)
+               ->  Sort
+                     Sort Key: t1_3.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_3.a + t3_3.b) / 2)) = t1_3.a)
+                           ->  Sort
+                                 Sort Key: (((t3_3.a + t3_3.b) / 2))
+                                 ->  Seq Scan on prt1_e_p3 t3_3
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_3.a
+                                 ->  Seq Scan on prt1_p3 t1_3
+               ->  Sort
+                     Sort Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Merge Left Join
+               Merge Cond: (t1_4.a = t2_4.b)
+               ->  Sort
+                     Sort Key: t1_4.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_4.a + t3_4.b) / 2)) = t1_4.a)
+                           ->  Sort
+                                 Sort Key: (((t3_4.a + t3_4.b) / 2))
+                                 ->  Seq Scan on prt1_e_p4 t3_4
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_4.a
+                                 ->  Seq Scan on prt1_p4 t1_4
+               ->  Sort
+                     Sort Key: t2_4.b
+                     ->  Seq Scan on prt2_p4 t2_4
+(83 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- MergeAppend on nullable column
 -- This should generate a partitionwise join, but currently fails to
@@ -1064,12 +1622,14 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Merge Left Join
-         Merge Cond: (prt1_p1.a = prt2_p2.b)
+         Merge Cond: (prt1_p0.a = prt2_p2.b)
          ->  Sort
-               Sort Key: prt1_p1.a
+               Sort Key: prt1_p0.a
                ->  Append
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p1
                            Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p2
@@ -1081,21 +1641,28 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                            Filter: (b > 250)
                      ->  Seq Scan on prt2_p3
                            Filter: (b > 250)
-(18 rows)
+                     ->  Seq Scan on prt2_p4
+                           Filter: (b > 250)
+(22 rows)
 
 SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  b  
------+-----
-   0 |    
-  50 |    
- 100 |    
- 150 |    
- 200 |    
- 250 |    
- 300 | 300
- 350 |    
- 400 |    
-(9 rows)
+  a   |  b  
+------+-----
+ -250 |    
+ -200 |    
+ -150 |    
+ -100 |    
+  -50 |    
+    0 |    
+   50 |    
+  100 |    
+  150 |    
+  200 |    
+  250 |    
+  300 | 300
+  350 |    
+  400 |    
+(14 rows)
 
 -- merge join when expression with whole-row reference needs to be sorted;
 -- partitionwise join does not apply
@@ -1109,175 +1676,2403 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
          Sort Key: t1.a, ((((t1.*)::prt1))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
    ->  Sort
          Sort Key: t2.b, ((((t2.*)::prt2))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt2_p1 t2
-                     ->  Seq Scan on prt2_p2 t2_1
-                     ->  Seq Scan on prt2_p3 t2_2
-(16 rows)
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+(20 rows)
 
 SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.b ORDER BY t1.a;
- a  | b  
-----+----
-  0 |  0
-  6 |  6
- 12 | 12
- 18 | 18
- 24 | 24
-(5 rows)
+  a  |  b  
+-----+-----
+ -24 | -24
+ -18 | -18
+ -12 | -12
+  -6 |  -6
+   0 |   0
+   6 |   6
+  12 |  12
+  18 |  18
+  24 |  24
+(9 rows)
 
 RESET enable_hashjoin;
 RESET enable_nestloop;
---
--- partitioned by multiple columns
---
-CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
-CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
-ANALYZE prt1_m;
-CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
-CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
-ANALYZE prt2_m;
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches prt2_p3 and
+-- prt2_p4 partition from prt2.
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-                                                             QUERY PLAN                                                             
-------------------------------------------------------------------------------------------------------------------------------------
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
  Sort
-   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_4
+                           Filter: (b = 0)
+(22 rows)
+
+-- partitionwise join should be possible when we drop prt2_p4 from prt2.
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
    ->  Append
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
-               ->  Seq Scan on prt1_m_p1
-                     Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt2_m_p1
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
-               ->  Seq Scan on prt1_m_p2
-                     Filter: (c = 0)
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt2_m_p2
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
-               ->  Seq Scan on prt1_m_p3
-                     Filter: (c = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt2_m_p3
-                           Filter: (c = 0)
-(24 rows)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+(26 rows)
 
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
-  50 | 0 |     |  
- 100 | 0 |     |  
- 150 | 0 | 150 | 0
- 200 | 0 |     |  
- 250 | 0 |     |  
- 300 | 0 | 300 | 0
- 350 | 0 |     |  
- 400 | 0 |     |  
- 450 | 0 | 450 | 0
- 500 | 0 |     |  
- 550 | 0 |     |  
-     |   |  75 | 0
-     |   | 225 | 0
-     |   | 375 | 0
-     |   | 525 | 0
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ANALYZE prt2;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -150 | -0150 | 0 | -0150
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+(7 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 |   | 
+ -200 | -0200 |   | 
+ -150 | -0150 | 0 | -0150
+ -100 | -0100 |   | 
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+(21 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -200 | 0 | -0200
+ -100 | 0 | -0100
+  -50 | 0 | -0050
+   50 | 0 | 0050
+  100 | 0 | 0100
+  200 | 0 | 0200
+  250 | 0 | 0250
+  350 | 0 | 0350
+  400 | 0 | 0400
+  500 | 0 | 0500
+  550 | 0 | 0550
+  650 | 0 | 0650
+  700 | 0 | 0700
+(14 rows)
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+                             QUERY PLAN                              
+---------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t3.c
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p0_a on prt1_p0 t3
+                           Index Cond: (a = t2.b)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t2.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_1.a = t2_1.b)
+                           ->  Seq Scan on prt1_p1 t3_1
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1 t2_1
+                                       Filter: (a = 0)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p2_a on prt1_p2 t3_2
+                           Index Cond: (a = t2_2.b)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t3_3
+                           Index Cond: (a = t2_3.b)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_4.a = t2_4.b)
+                           ->  Seq Scan on prt1_p4 t3_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t2_4
+                                       Filter: (a = 0)
+(47 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+  a   | a |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.b
+   ->  Hash Right Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p5 t2_5
+                           Filter: (a = 0)
+(24 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: (t1.a = t2.b)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.a, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
+                     Filter: (a = 0)
+(22 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
+               ->  Seq Scan on prt1_m_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p1
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
+               ->  Seq Scan on prt1_m_p2
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p2
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
+               ->  Seq Scan on prt1_m_p3
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p3
+                           Filter: (c = 0)
+(24 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+  a  | c |  b  | c 
+-----+---+-----+---
+   0 | 0 |   0 | 0
+  50 | 0 |     |  
+ 100 | 0 |     |  
+ 150 | 0 | 150 | 0
+ 200 | 0 |     |  
+ 250 | 0 |     |  
+ 300 | 0 | 300 | 0
+ 350 | 0 |     |  
+ 400 | 0 |     |  
+ 450 | 0 | 450 | 0
+ 500 | 0 |     |  
+ 550 | 0 |     |  
+     |   |  75 | 0
+     |   | 225 | 0
+     |   | 375 | 0
+     |   | 525 | 0
+(16 rows)
+
+--
+-- tests for list partitioned tables.
+--
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
+--
+-- list partitioned by expression
+--
+CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
+ANALYZE plt1_e;
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+                                           QUERY PLAN                                           
+------------------------------------------------------------------------------------------------
+ GroupAggregate
+   Group Key: t1.c, t2.c, t3.c
+   ->  Sort
+         Sort Key: t1.c, t3.c
+         ->  Append
+               ->  Hash Join
+                     Hash Cond: ((t1.c)::text = ltrim(t3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2.b = t1.b) AND ((t2.c)::text = (t1.c)::text))
+                           ->  Seq Scan on plt2_p4 t2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p4 t1
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p4 t3
+               ->  Hash Join
+                     Hash Cond: ((t1_1.c)::text = ltrim(t3_1.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t1_1.b = t2_1.b) AND ((t1_1.c)::text = (t2_1.c)::text))
+                           ->  Seq Scan on plt1_p1 t1_1
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p1 t2_1
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p1 t3_1
+               ->  Hash Join
+                     Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t1_2.b = t2_2.b) AND ((t1_2.c)::text = (t2_2.c)::text))
+                           ->  Seq Scan on plt1_p2 t1_2
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p2 t2_2
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p2 t3_2
+               ->  Hash Join
+                     Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2_3.b = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text))
+                           ->  Seq Scan on plt2_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p3 t3_3
+(41 rows)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p5 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_3
+                                 ->  Seq Scan on plt2_p3 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
 (16 rows)
 
---
--- tests for list partitioned tables.
---
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
---
--- list partitioned by expression
---
-CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1_e;
--- test partition matching with N-way join
+-- semi join
 EXPLAIN (COSTS OFF)
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                   QUERY PLAN                                   
---------------------------------------------------------------------------------
- GroupAggregate
-   Group Key: t1.c, t2.c, t3.c
-   ->  Sort
-         Sort Key: t1.c, t3.c
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
          ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.c = ltrim(t3.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1.b = t2.b) AND (t1.c = t2.c))
-                           ->  Seq Scan on plt1_p1 t1
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.c = ltrim(t3_1.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
-                           ->  Seq Scan on plt1_p2 t1_1
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.c = ltrim(t3_2.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
-                           ->  Seq Scan on plt1_p3 t1_2
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p3 t3_2
-(32 rows)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Seq Scan on plt2_p2 t2_2
+                                 ->  Seq Scan on plt2_p3 t2_3
+(22 rows)
 
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  75.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 123.0000000000000000 | 123.0000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 225.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 273.0000000000000000 | 273.0000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 375.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 423.0000000000000000 | 423.0000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 525.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 573.0000000000000000 | 573.0000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1302,22 +4097,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
 --------------------------------------------------
  Hash Left Join
    Hash Cond: (t2.b = a)
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t3.a = t2.b)
-               ->  Seq Scan on prt1_p1 t3
-               ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
-         ->  Hash Join
-               Hash Cond: (t3_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t3_1
-               ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
-         ->  Hash Join
-               Hash Cond: (t3_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t3_2
-               ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
+   ->  Hash Join
+         Hash Cond: (t3.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t3
+               ->  Seq Scan on prt1_p1 t3_1
+               ->  Seq Scan on prt1_p2 t3_2
+               ->  Seq Scan on prt1_p3 t3_3
+               ->  Seq Scan on prt1_p4 t3_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
    ->  Hash
          ->  Result
                One-Time Filter: false
@@ -1332,16 +4127,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 --
 -- tests for hash partitioned tables.
@@ -1417,41 +4218,9 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  273.0000000000000000 | 273.0000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
 (6 rows)
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
- Sort
-   Sort Key: t1.a
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
-               ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
-               ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
-                           Filter: (b = 0)
-(21 rows)
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
@@ -1466,26 +4235,24 @@ SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c
    Sort Key: t1.c
    ->  HashAggregate
          Group Key: t1.c, t2.c
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t2.c = t1.c)
-                     ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p1 t1
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on plt1_p4 t1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_1.c = t1_1.c)
-                     ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_p2 t1_1
+                           ->  Seq Scan on plt1_p1 t1_1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_2.c = t1_2.c)
-                     ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Seq Scan on plt1_p2 t1_2
                                  Filter: ((a % 25) = 0)
-(23 rows)
+                           ->  Seq Scan on plt1_p3 t1_3
+                                 Filter: ((a % 25) = 0)
+(21 rows)
 
 --
 -- multiple levels of partitioning
@@ -1881,64 +4648,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
  Hash Join
    Hash Cond: (t1.a = t2.a)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt4_n_p1 t2
                ->  Seq Scan on prt4_n_p2 t2_1
                ->  Seq Scan on prt4_n_p3 t2_2
-(11 rows)
+(13 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a and t1.a = t3.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                        QUERY PLAN                        
+----------------------------------------------------------
  Hash Join
-   Hash Cond: (t2.a = t1.a)
+   Hash Cond: (t3.b = t1.a)
    ->  Append
-         ->  Seq Scan on prt4_n_p1 t2
-         ->  Seq Scan on prt4_n_p2 t2_1
-         ->  Seq Scan on prt4_n_p3 t2_2
+         ->  Seq Scan on prt2_p0 t3
+         ->  Seq Scan on prt2_p1 t3_1
+         ->  Seq Scan on prt2_p2 t3_2
+         ->  Seq Scan on prt2_p3 t3_3
+         ->  Seq Scan on prt2_p4 t3_4
+         ->  Seq Scan on prt2_p5 t3_5
    ->  Hash
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.a = t3_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_2.b)
-                     ->  Seq Scan on prt1_p3 t1_2
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3 t3_2
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt4_n_p1 t2
+                           ->  Seq Scan on prt4_n_p2 t2_1
+                           ->  Seq Scan on prt4_n_p3 t2_2
 (23 rows)
 
 -- partitionwise join can not be applied if there are no equi-join conditions
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (b > t1.a)
-(12 rows)
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+(16 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partitionwise join
@@ -2024,16 +4797,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
-                     ->  Seq Scan on plt1_p1 t3
-                     ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3
+                     ->  Seq Scan on plt1_p1 t3_1
+                     ->  Seq Scan on plt1_p2 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partitionwise join can not be applied for a join between list and range
 -- partitioned tables
@@ -2044,14 +4818,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
  Hash Full Join
    Hash Cond: ((t2.c)::text = (t1.c)::text)
    ->  Append
-         ->  Seq Scan on prt1_p1 t2
-         ->  Seq Scan on prt1_p2 t2_1
-         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p0 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p2 t2_2
+         ->  Seq Scan on prt1_p3 t2_3
+         ->  Seq Scan on prt1_p4 t2_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt1_n_p1 t1
                ->  Seq Scan on prt1_n_p2 t1_1
-(10 rows)
+(12 rows)
 
 -- partitionwise join can not be applied if only one of joining tables has
 -- default partition
@@ -2067,16 +4843,279 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
    ->  Hash Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007');
+INSERT INTO plt1 SELECT i, i, to_char(i % 8, 'FM0000') FROM generate_series(0, 39) i WHERE i % 8 IN (0, 2, 3, 4, 6, 7);
+ANALYZE plt1;
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0005', '0007');
+INSERT INTO plt2 SELECT i, i, to_char(i % 8, 'FM0000') FROM generate_series(0, 39) i WHERE i % 8 IN (1, 2, 3, 4, 5, 7);
+ANALYZE plt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 8 != 2 AND COALESCE(t1.a, 0) % 8 != 3 AND COALESCE(t1.a, 0) % 8 != 4 AND COALESCE(t1.a, 0) % 8 != 7 ORDER BY t1.c, t2.c, t1.a, t2.a;
+                                                                               QUERY PLAN                                                                                
+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t2.c, t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.c = t2.c)
+               Filter: (((COALESCE(t1.a, 0) % 8) <> 2) AND ((COALESCE(t1.a, 0) % 8) <> 3) AND ((COALESCE(t1.a, 0) % 8) <> 4) AND ((COALESCE(t1.a, 0) % 8) <> 7))
+               ->  Seq Scan on plt1_p1 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t2_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 8) <> 2) AND ((COALESCE(t1_1.a, 0) % 8) <> 3) AND ((COALESCE(t1_1.a, 0) % 8) <> 4) AND ((COALESCE(t1_1.a, 0) % 8) <> 7))
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_1
+         ->  Hash Full Join
+               Hash Cond: (t1_2.c = t2_2.c)
+               Filter: (((COALESCE(t1_2.a, 0) % 8) <> 2) AND ((COALESCE(t1_2.a, 0) % 8) <> 3) AND ((COALESCE(t1_2.a, 0) % 8) <> 4) AND ((COALESCE(t1_2.a, 0) % 8) <> 7))
+               ->  Seq Scan on plt1_p3 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_2
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 8 != 2 AND COALESCE(t1.a, 0) % 8 != 3 AND COALESCE(t1.a, 0) % 8 != 4 AND COALESCE(t1.a, 0) % 8 != 7 ORDER BY t1.c, t2.c, t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+  0 | 0000 |    | 
+  8 | 0000 |    | 
+ 16 | 0000 |    | 
+ 24 | 0000 |    | 
+ 32 | 0000 |    | 
+  6 | 0006 |    | 
+ 14 | 0006 |    | 
+ 22 | 0006 |    | 
+ 30 | 0006 |    | 
+ 38 | 0006 |    | 
+    |      |  1 | 0001
+    |      |  9 | 0001
+    |      | 17 | 0001
+    |      | 25 | 0001
+    |      | 33 | 0001
+    |      |  5 | 0005
+    |      | 13 | 0005
+    |      | 21 | 0005
+    |      | 29 | 0005
+    |      | 37 | 0005
+(20 rows)
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN (NULL, '0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
+ANALYZE plt1;
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 DEFAULT;
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+ANALYZE plt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t2_1.c, t1.a, t2_1.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.c = t2_1.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 2) AND ((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt1_p1 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t2.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 2) AND ((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+  0 | 0000 |    | 
+  5 | 0000 |    | 
+ 10 | 0000 |    | 
+ 15 | 0000 |    | 
+ 20 | 0000 |    | 
+    |      |  1 | 0001
+    |      |  6 | 0001
+    |      | 11 | 0001
+    |      | 16 | 0001
+    |      | 21 | 0001
+(10 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2 t2 FULL JOIN plt1 t1 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t2_1.c, t1.a, t2_1.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t2_1.c = t1.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 2) AND ((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1
+         ->  Hash Full Join
+               Hash Cond: (t2.c = t1_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 2) AND ((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt2_p2 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_1
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2 t2 FULL JOIN plt1 t1 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+  0 | 0000 |    | 
+  5 | 0000 |    | 
+ 10 | 0000 |    | 
+ 15 | 0000 |    | 
+ 20 | 0000 |    | 
+    |      |  1 | 0001
+    |      |  6 | 0001
+    |      | 11 | 0001
+    |      | 16 | 0001
+    |      | 21 | 0001
+(10 rows)
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1;
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2;
+CREATE TABLE plt3 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt3_p1 PARTITION OF plt3 FOR VALUES IN ('0001');
+CREATE TABLE plt3_p2 PARTITION OF plt3 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2.a, t3.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.c = t3.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1.c = t2.c)
+                     ->  Seq Scan on plt1_p1 t1
+                     ->  Hash
+                           ->  Seq Scan on plt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on plt3_p1 t3
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t3_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_1.c = t2_1.c)
+                     ->  Seq Scan on plt1_p2 t1_1
+                     ->  Hash
+                           ->  Seq Scan on plt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt3_p2 t3_1
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+ a  |  c   | a  |  c   | a  |  c   
+----+------+----+------+----+------
+  0 | 0000 |    |      |    | 
+  5 | 0000 |    |      |    | 
+ 10 | 0000 |    |      |    | 
+ 15 | 0000 |    |      |    | 
+ 20 | 0000 |    |      |    | 
+  1 | 0001 |    |      |  1 | 0001
+  1 | 0001 |    |      |  6 | 0001
+  1 | 0001 |    |      | 11 | 0001
+  1 | 0001 |    |      | 16 | 0001
+  1 | 0001 |    |      | 21 | 0001
+  6 | 0001 |    |      |  1 | 0001
+  6 | 0001 |    |      |  6 | 0001
+  6 | 0001 |    |      | 11 | 0001
+  6 | 0001 |    |      | 16 | 0001
+  6 | 0001 |    |      | 21 | 0001
+ 11 | 0001 |    |      |  1 | 0001
+ 11 | 0001 |    |      |  6 | 0001
+ 11 | 0001 |    |      | 11 | 0001
+ 11 | 0001 |    |      | 16 | 0001
+ 11 | 0001 |    |      | 21 | 0001
+ 16 | 0001 |    |      |  1 | 0001
+ 16 | 0001 |    |      |  6 | 0001
+ 16 | 0001 |    |      | 11 | 0001
+ 16 | 0001 |    |      | 16 | 0001
+ 16 | 0001 |    |      | 21 | 0001
+ 21 | 0001 |    |      |  1 | 0001
+ 21 | 0001 |    |      |  6 | 0001
+ 21 | 0001 |    |      | 11 | 0001
+ 21 | 0001 |    |      | 16 | 0001
+ 21 | 0001 |    |      | 21 | 0001
+  2 | 0002 |  2 | 0002 |    | 
+  2 | 0002 |  7 | 0002 |    | 
+  2 | 0002 | 12 | 0002 |    | 
+  2 | 0002 | 17 | 0002 |    | 
+  2 | 0002 | 22 | 0002 |    | 
+  7 | 0002 |  2 | 0002 |    | 
+  7 | 0002 |  7 | 0002 |    | 
+  7 | 0002 | 12 | 0002 |    | 
+  7 | 0002 | 17 | 0002 |    | 
+  7 | 0002 | 22 | 0002 |    | 
+ 12 | 0002 |  2 | 0002 |    | 
+ 12 | 0002 |  7 | 0002 |    | 
+ 12 | 0002 | 12 | 0002 |    | 
+ 12 | 0002 | 17 | 0002 |    | 
+ 12 | 0002 | 22 | 0002 |    | 
+ 17 | 0002 |  2 | 0002 |    | 
+ 17 | 0002 |  7 | 0002 |    | 
+ 17 | 0002 | 12 | 0002 |    | 
+ 17 | 0002 | 17 | 0002 |    | 
+ 17 | 0002 | 22 | 0002 |    | 
+ 22 | 0002 |  2 | 0002 |    | 
+ 22 | 0002 |  7 | 0002 |    | 
+ 22 | 0002 | 12 | 0002 |    | 
+ 22 | 0002 | 17 | 0002 |    | 
+ 22 | 0002 | 22 | 0002 |    | 
+(55 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 92994b479b..d8434f6b1a 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -10,25 +10,39 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -69,11 +83,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -110,20 +132,30 @@ RESET enable_hashjoin;
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 
 EXPLAIN (COSTS OFF)
@@ -187,6 +219,114 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches prt2_p3 and
+-- prt2_p4 partition from prt2.
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- partitionwise join should be possible when we drop prt2_p4 from prt2.
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ANALYZE prt2;
+
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -211,28 +351,79 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
 -- test partition matching with N-way join
@@ -240,6 +431,175 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
@@ -285,27 +645,18 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
 ALTER TABLE plt2 DETACH PARTITION plt2_p3;
 ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
 ANALYZE plt2;
-
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.a % 25 = 0 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
 --
 -- multiple levels of partitioning
 --
@@ -450,3 +801,72 @@ ANALYZE prt2;
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007');
+INSERT INTO plt1 SELECT i, i, to_char(i % 8, 'FM0000') FROM generate_series(0, 39) i WHERE i % 8 IN (0, 2, 3, 4, 6, 7);
+ANALYZE plt1;
+
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0005', '0007');
+INSERT INTO plt2 SELECT i, i, to_char(i % 8, 'FM0000') FROM generate_series(0, 39) i WHERE i % 8 IN (1, 2, 3, 4, 5, 7);
+ANALYZE plt2;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 8 != 2 AND COALESCE(t1.a, 0) % 8 != 3 AND COALESCE(t1.a, 0) % 8 != 4 AND COALESCE(t1.a, 0) % 8 != 7 ORDER BY t1.c, t2.c, t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 8 != 2 AND COALESCE(t1.a, 0) % 8 != 3 AND COALESCE(t1.a, 0) % 8 != 4 AND COALESCE(t1.a, 0) % 8 != 7 ORDER BY t1.c, t2.c, t1.a, t2.a;
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN (NULL, '0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
+ANALYZE plt1;
+
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 DEFAULT;
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+ANALYZE plt2;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2 t2 FULL JOIN plt1 t1 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2 t2 FULL JOIN plt1 t1 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1;
+
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2;
+
+CREATE TABLE plt3 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt3_p1 PARTITION OF plt3 FOR VALUES IN ('0001');
+CREATE TABLE plt3_p2 PARTITION OF plt3 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
-- 
2.19.2

v27-0002-Modify-partition_range_bounds_merge.patchapplication/octet-stream; name=v27-0002-Modify-partition_range_bounds_merge.patchDownload
From fa1dd52a2caec26d834fd3e80195140b1a125742 Mon Sep 17 00:00:00 2001
From: Etsuro Fujita <efujita@postgresql.org>
Date: Tue, 12 Nov 2019 21:58:34 +0900
Subject: [PATCH 2/2] Modify partition_range_bounds_merge()

---
 src/backend/partitioning/partbounds.c | 594 ++++++++++++--------------
 1 file changed, 266 insertions(+), 328 deletions(-)

diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index f145851871..4aebc71ebb 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -81,6 +81,12 @@ typedef struct PartitionMap
 								 * did_remapping */
 } PartitionMap;
 
+#define compare_range_bounds(partnatts, partsupfunc, partcollations, \
+							 bound1, bound2) \
+	(partition_rbound_cmp(partnatts, partsupfunc, partcollations, \
+						  (bound1)->datums, (bound1)->kind, (bound1)->lower, \
+						  bound2))
+
 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);
@@ -142,7 +148,6 @@ static bool process_outer_partition(PartitionMap *outer_map,
 						int outer_index,
 						int inner_default,
 						JoinType jointype,
-						char strategy,
 						int *next_index,
 						int *default_index,
 						int *merged_index);
@@ -153,7 +158,6 @@ static bool process_inner_partition(PartitionMap *outer_map,
 						int inner_index,
 						int outer_default,
 						JoinType jointype,
-						char strategy,
 						int *next_index,
 						int *default_index,
 						int *merged_index);
@@ -170,20 +174,31 @@ static PartitionBoundInfo build_merged_partition_bounds(char strategy,
 							  List *merged_datums, List *merged_indexes,
 							  List *merged_contents, int null_index,
 							  int default_index);
-static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
-						  Oid *collations, PartitionRangeBound *bound1,
-						  PartitionRangeBound *bound2);
-static bool partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
-						   Oid *collations, PartitionRangeBound *lower_bound1,
-						   PartitionRangeBound *upper_bound1,
-						   PartitionRangeBound *lower_bound2,
-						   PartitionRangeBound *upper_bound2, int *ub_cmpval,
-						   int *lb_cmpval);
-static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
-							  Oid *collations, Datum *next_lb_datums,
-							  PartitionRangeDatumKind *next_lb_kind,
-							  List **merged_datums, List **merged_kinds,
-							  List **merged_indexes);
+static int get_range_partition(PartitionBoundInfo bi, int *lb_pos,
+					PartitionRangeBound *lb, PartitionRangeBound *ub);
+static bool compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+						 Oid *partcollations,
+						 PartitionRangeBound *lower_bound1,
+						 PartitionRangeBound *upper_bound1,
+						 PartitionRangeBound *lower_bound2,
+						 PartitionRangeBound *upper_bound2,
+						 int *ub_cmpval, int *lb_cmpval);
+static void get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations, JoinType jointype,
+						PartitionRangeBound *left_lb,
+						PartitionRangeBound *left_ub,
+						PartitionRangeBound *right_lb,
+						PartitionRangeBound *right_ub,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub);
+static void add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub,
+						int merged_index,
+						List **merged_datums,
+						List **merged_kinds,
+						List **merged_indexes);
 static bool merge_default_partitions(PartitionMap *outer_map,
 						 PartitionMap *inner_map,
 						 bool outer_has_default,
@@ -3164,83 +3179,60 @@ partition_bounds_merge(int partnatts,
 }
 
 /*
- * partition_get_range_bounds
+ * get_range_partition
+ *		Returns the index of the range partition with the given lower bound
  *
- * Given the index of lower bound in datums array, return lower and upper
- * bounds and the index of the partition with that lower bound.
+ * *lb and *ub are set to the lower and upper bounds of the range partition
+ * respectively, and *lb_index is advanced to the next lower bound, if any.
  */
 static int
-partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
-						   PartitionRangeBound *lower,
-						   PartitionRangeBound *upper)
+get_range_partition(PartitionBoundInfo bi, int *lb_index,
+					PartitionRangeBound *lb, PartitionRangeBound *ub)
 {
-	int			part_index;
+	/* Return the index as -1 if we've exhausted all the lower bounds. */
+	if (*lb_index >= bi->ndatums)
+		return -1;
 
 	/* A lower bound should have at least one more bound after it. */
-	Assert(lb_index < bi->ndatums - 1);
-
-	/* The lower bound should correspond to a valid partition. */
-	part_index = bi->indexes[lb_index + 1];
-	Assert(part_index >= 0);
-
-	lower->kind = bi->kind[lb_index];
-	lower->datums = bi->datums[lb_index];
-	lower->lower = true;
-	upper->kind = bi->kind[lb_index + 1];
-	upper->datums = bi->datums[lb_index + 1];
-	upper->lower = false;
+	Assert(*lb_index + 1 < bi->ndatums);
 
-	return part_index;
-}
-
-/*
- * partition_range_get_next_lb_index
- *
- * Given the index of lower bound in datums array return the
- * index of lower bound of the next partition. When the given index corresponds
- * to the last partition, return number of datums (ndatums).
- */
-static int
-partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
-{
-	/* A lower bound should have at least one more bound after it. */
-	Assert(lb_index < bi->ndatums - 1);
+	lb->index = bi->indexes[*lb_index];
+	lb->kind = bi->kind[*lb_index];
+	lb->datums = bi->datums[*lb_index];
+	lb->lower = true;
+	ub->index = bi->indexes[*lb_index + 1];
+	ub->kind = bi->kind[*lb_index + 1];
+	ub->datums = bi->datums[*lb_index + 1];
+	ub->lower = false;
 
-	/* The partition index corresponding to the upper bound should be valid. */
-	Assert(bi->indexes[lb_index + 1] >= 0);
+	/* The partition index of an upper bound should be valid. */
+	Assert(ub->index >= 0);
 
 	/*
-	 * If there are no bounds left beyond the upper bound, we have reached the
-	 * last partition.
+	 * Advance the lower bound; if there are no bounds left beyond the upper
+	 * bound, we have reached the last lower bound.
 	 */
-	if (lb_index + 2 < bi->ndatums)
+	if (*lb_index + 2 >= bi->ndatums)
+		*lb_index = bi->ndatums;
+	else
 	{
 		/*
-		 * If the bound next to the upper bound corresponds to no partition,
-		 * that's the next lower bound of the next partition. Otherwise, the
-		 * current upper bound is the lower bound of the next partition.
+		 * If the index assigned to the bound next to the upper bound isn't
+		 * valid, that is the lower bound of the next range partition; else,
+		 * the upper bound of the current range partition is also the lower
+		 * bound of the next range partition.
 		 */
-		if (bi->indexes[lb_index + 2] < 0)
-			return lb_index + 2;
+		if (bi->indexes[*lb_index + 2] < 0)
+			*lb_index = *lb_index + 2;
 		else
-			return lb_index + 1;
+			*lb_index = *lb_index + 1;
 	}
-	else
-		return bi->ndatums;
-}
 
-static int32
-partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
-						  Oid *partcollations, PartitionRangeBound *bound1,
-						  PartitionRangeBound *bound2)
-{
-	return partition_rbound_cmp(partnatts, partsupfunc, partcollations,
-								bound1->datums, bound1->kind, bound1->lower,
-								bound2);
+	return ub->index;
 }
 
 /*
- * partition_range_cmp
+ * compare_range_partitions
  *
  * Compare the bounds of two range partitions. Set ub_cmpval <, = or > 0, if the
  * first partition's upper bound is lower than, equal to or higher than the
@@ -3251,12 +3243,13 @@ partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
  * Return true, if the ranges overlap, otherwise return false.
  */
 static bool
-partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
-					PartitionRangeBound *lower_bound1,
-					PartitionRangeBound *upper_bound1,
-					PartitionRangeBound *lower_bound2,
-					PartitionRangeBound *upper_bound2, int *ub_cmpval,
-					int *lb_cmpval)
+compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+						 Oid *partcollations,
+						 PartitionRangeBound *lower_bound1,
+						 PartitionRangeBound *upper_bound1,
+						 PartitionRangeBound *lower_bound2,
+						 PartitionRangeBound *upper_bound2,
+						 int *ub_cmpval, int *lb_cmpval)
 {
 	bool		overlap;
 
@@ -3266,15 +3259,15 @@ partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
 	 * the partitions are not overlapping. All other cases indicate overlapping
 	 * partitions.
 	 */
-	if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
-								  lower_bound1, upper_bound2) > 0)
+	if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+							 lower_bound1, upper_bound2) > 0)
 	{
 		overlap = false;
 		*ub_cmpval = 1;
 		*lb_cmpval = 1;
 	}
-	else if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
-									   lower_bound2, upper_bound1) > 0)
+	else if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								  lower_bound2, upper_bound1) > 0)
 	{
 		overlap = false;
 		*ub_cmpval = -1;
@@ -3283,35 +3276,34 @@ partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
 	else
 	{
 		overlap = true;
-		*ub_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
-											   partcollations, upper_bound1,
-											   upper_bound2);
-		*lb_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
-											   partcollations, lower_bound1,
-											   lower_bound2);
+		*ub_cmpval = compare_range_bounds(partnatts, partsupfuncs,
+										  partcollations, upper_bound1,
+										  upper_bound2);
+		*lb_cmpval = compare_range_bounds(partnatts, partsupfuncs,
+										  partcollations, lower_bound1,
+										  lower_bound2);
 	}
 
 	return overlap;
 }
 
 /*
- * partition_range_merge
- *
- * Merge the partition bounds of given two partitions such that the join
- * between the given two partitions fits merged bounds.
+ * get_merged_range_bounds
+ *		Given the bounds of range partitions to be join, determine the range
+ *		bounds of the merged partition produced from the range partitions
  *
- * "merged_upper" will be set to one of the given upper bounds and
- * "merged_lower" will be set to one of the given lower bounds.
+ * *merged_lb and *merged_ub are set to the lower and upper bounds of the
+ * merged partition.
  */
 static void
-partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
-					  Oid *partcollations, JoinType jointype,
-					  PartitionRangeBound *left_lb,
-					  PartitionRangeBound *left_ub,
-					  PartitionRangeBound *right_lb,
-					  PartitionRangeBound *right_ub,
-					  PartitionRangeBound **merged_lb,
-					  PartitionRangeBound **merged_ub)
+get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations, JoinType jointype,
+						PartitionRangeBound *left_lb,
+						PartitionRangeBound *left_ub,
+						PartitionRangeBound *right_lb,
+						PartitionRangeBound *right_ub,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub)
 {
 	/*
 	 * An outer join will have all the rows from the outer side, so merged
@@ -3324,41 +3316,37 @@ partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
 	{
 		case JOIN_LEFT:
 		case JOIN_ANTI:
-			*merged_ub = left_ub;
-			*merged_lb = left_lb;
+			*merged_ub = *left_ub;
+			*merged_lb = *left_lb;
 			break;
 
 		case JOIN_INNER:
 		case JOIN_SEMI:
-			if (partition_range_bound_cmp(partnatts, partsupfuncs,
-										  partcollations, left_ub,
-										  right_ub) < 0)
-				*merged_ub = left_ub;
+			if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 left_ub, right_ub) < 0)
+				*merged_ub = *left_ub;
 			else
-				*merged_ub = right_ub;
+				*merged_ub = *right_ub;
 
-			if (partition_range_bound_cmp(partnatts, partsupfuncs,
-										  partcollations, left_lb,
-										  right_lb) > 0)
-				*merged_lb = left_lb;
+			if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 left_lb, right_lb) > 0)
+				*merged_lb = *left_lb;
 			else
-				*merged_lb = right_lb;
+				*merged_lb = *right_lb;
 			break;
 
 		case JOIN_FULL:
-			if (partition_range_bound_cmp(partnatts, partsupfuncs,
-										  partcollations, left_ub,
-										  right_ub) > 0)
-				*merged_ub = left_ub;
+			if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 left_ub, right_ub) > 0)
+				*merged_ub = *left_ub;
 			else
-				*merged_ub = right_ub;
+				*merged_ub = *right_ub;
 
-			if (partition_range_bound_cmp(partnatts, partsupfuncs,
-										  partcollations, left_lb,
-										  right_lb) < 0)
-				*merged_lb = left_lb;
+			if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 left_lb, right_lb) < 0)
+				*merged_lb = *left_lb;
 			else
-				*merged_lb = right_lb;
+				*merged_lb = *right_lb;
 			break;
 
 		default:
@@ -3367,16 +3355,19 @@ partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
 }
 
 /*
- * Add the lower bound of the next range to the list of bounds, if the lower
- * bound is higher or equal to the previous upper bound. If successful return
- * true, otherwise false.
+ * add_merged_range_bounds
+ *		Add the range bounds of a merged partition to the lists of range
+ *		bounds
  */
-static bool
-partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
-							  Oid *partcollations, Datum *next_lb_datums,
-							  PartitionRangeDatumKind *next_lb_kind,
-							  List **merged_datums, List **merged_kinds,
-							  List **merged_indexes)
+static void
+add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub,
+						int merged_index,
+						List **merged_datums,
+						List **merged_kinds,
+						List **merged_indexes)
 {
 	int			cmpval;
 
@@ -3389,38 +3380,43 @@ partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
 	{
 		PartitionRangeBound	prev_ub;
 
-		prev_ub.datums = llast(*merged_datums);
-		prev_ub.kind = llast(*merged_kinds);
+		Assert(*merged_kinds && *merged_indexes);
+
+		/* Get the last upper bound. */
+		prev_ub.index = llast_int(*merged_indexes);
+		prev_ub.datums = (Datum *) llast(*merged_datums);
+		prev_ub.kind = (PartitionRangeDatumKind *) llast(*merged_kinds);
 		prev_ub.lower = false;
 
+		/*
+		 * We pass to partition_rbound_cmp() lower1 as false to prevent it
+		 * from considering the last upper bound to be smaller than the lower
+		 * bound of the merged partition when the values of the two range
+		 * bounds compare equal.
+		 */
 		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
-									  next_lb_datums, next_lb_kind, false,
-									  &prev_ub);
+									  merged_lb->datums, merged_lb->kind,
+									  false, &prev_ub);
+		Assert(cmpval >= 0);
 	}
 
 	/*
-	 * The lower bound is lower than the last upper bound, thus does not fit
-	 * the bounds created so far and hence can not be merged with the existing
-	 * bounds.
-	 */
-	if (cmpval < 0)
-		return false;
-
-	/*
-	 * Add bounds of the new merged partition. If the next lower bound is
-	 * higher than the last upper bound, add new range with index
-	 * corresponding to the lower bound as -1. If the merged lower bound
-	 * is same as the last merged upper bound, the last upper bound will be
-	 * reused as the lower bound of the next range.
+	 * If the lower bound is higher than the last upper bound, add the lower
+	 * bound with the index as -1 indicating that that is a lower bound; else,
+	 * the last upper bound will be reused as the lower bound of the merged
+	 * partition, so skip this.
 	 */
 	if (cmpval > 0)
 	{
-		*merged_datums = lappend(*merged_datums, next_lb_datums);
-		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_datums = lappend(*merged_datums, merged_lb->datums);
+		*merged_kinds = lappend(*merged_kinds, merged_lb->kind);
 		*merged_indexes = lappend_int(*merged_indexes, -1);
 	}
 
-	return true;
+	/* Add the upper bound and index of the merged partition. */
+	*merged_datums = lappend(*merged_datums, merged_ub->datums);
+	*merged_kinds = lappend(*merged_kinds, merged_ub->kind);
+	*merged_indexes = lappend_int(*merged_indexes, merged_index);
 }
 
 /*
@@ -3444,13 +3440,19 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 	int			inner_default = inner_bi->default_index;
 	PartitionMap outer_map;
 	PartitionMap inner_map;
+	int			outer_part;
+	int			inner_part;
+	int			outer_lb_index;
+	int			inner_lb_index;
+	PartitionRangeBound outer_lb;
+	PartitionRangeBound outer_ub;
+	PartitionRangeBound inner_lb;
+	PartitionRangeBound inner_ub;
 	int			next_index = 0;
 	int			default_index = -1;
 	List	   *merged_datums = NIL;
 	List	   *merged_kinds = NIL;
 	List	   *merged_indexes = NIL;
-	int			outer_lb_index;
-	int			inner_lb_index;
 
 	Assert(outer_bi->strategy == inner_bi->strategy &&
 		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
@@ -3471,29 +3473,25 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 	 * datum in PartitionBoundInfo::datums of that side.
 	 */
 	outer_lb_index = inner_lb_index = 0;
-	while (outer_lb_index < outer_bi->ndatums ||
-		   inner_lb_index < inner_bi->ndatums)
+	outer_part = get_range_partition(outer_bi, &outer_lb_index,
+									 &outer_lb, &outer_ub);
+	inner_part = get_range_partition(inner_bi, &inner_lb_index,
+									 &inner_lb, &inner_ub);
+	while (outer_part >= 0 || inner_part >= 0)
 	{
-		PartitionRangeBound *merged_lb = NULL;
-		PartitionRangeBound *merged_ub = NULL;
+		PartitionRangeBound merged_lb;
+		PartitionRangeBound merged_ub;
 		int			merged_index = -1;
-		PartitionRangeBound outer_lb;
-		PartitionRangeBound outer_ub;
-		PartitionRangeBound inner_lb;
-		PartitionRangeBound inner_ub;
-		int			outer_part = -1;
-		int			inner_part = -1;
 		bool		overlap;
 		int			ub_cmpval;
 		int			lb_cmpval;
 
-		/* Get the range bounds of the next pair of partitions. */
-		if (outer_lb_index < outer_bi->ndatums)
-			outer_part = partition_get_range_bounds(outer_bi, outer_lb_index,
-													&outer_lb, &outer_ub);
-		if (inner_lb_index < inner_bi->ndatums)
-			inner_part = partition_get_range_bounds(inner_bi, inner_lb_index,
-													&inner_lb, &inner_ub);
+		if (outer_part >= 0)
+			Assert(outer_map.merged_indexes[outer_part] == -1 &&
+				   outer_map.merged[outer_part] == false);
+		if (inner_part >= 0)
+			Assert(inner_map.merged_indexes[inner_part] == -1 &&
+				   inner_map.merged[inner_part] == false);
 
 		/*
 		 * We run this loop till both the sides finish. This allows to avoid
@@ -3505,25 +3503,30 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 		 * side. That way we advance the partitions on that side till all of
 		 * them are  exhausted.
 		 */
-		if (outer_lb_index >= outer_bi->ndatums)
+		if (outer_part == -1)
 		{
 			overlap = false;
 			ub_cmpval = 1;
 			lb_cmpval = 1;
 		}
-		else if (inner_lb_index >= inner_bi->ndatums)
+		else if (inner_part == -1)
 		{
 			overlap = false;
 			ub_cmpval = -1;
 			lb_cmpval = -1;
 		}
 		else
-			overlap = partition_range_cmp(partnatts, partsupfuncs, partcollations,
-										  &outer_lb, &outer_ub, &inner_lb,
-										  &inner_ub, &ub_cmpval, &lb_cmpval);
+			overlap = compare_range_partitions(partnatts, partsupfuncs,
+											   partcollations,
+											   &outer_lb, &outer_ub,
+											   &inner_lb, &inner_ub,
+											   &ub_cmpval, &lb_cmpval);
 
 		if (overlap)
 		{
+			PartitionRangeBound save_outer_ub;
+			PartitionRangeBound save_inner_ub;
+
 			/*
 			 * The rows from overlapping portion of ranges on both sides may
 			 * join, hence the corresponding pair of partitions form a joining
@@ -3531,152 +3534,117 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 			 * and its index by merging the bounds according to the type of
 			 * join.
 			 */
-			partition_range_merge(partnatts, partsupfuncs, partcollations,
-								  jointype, &outer_lb, &outer_ub, &inner_lb,
-								  &inner_ub, &merged_lb, &merged_ub);
+			get_merged_range_bounds(partnatts, partsupfuncs, partcollations,
+									jointype, &outer_lb, &outer_ub, &inner_lb,
+									&inner_ub, &merged_lb, &merged_ub);
 
+			/*
+			 * Both partitions are not merged yet, so they should be merged
+			 * successfully.
+			 */
 			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
 													outer_part, inner_part,
 													&next_index);
+			Assert(merged_index >= 0);
 
-			if (merged_index < 0)
-			{
-				/* Failed to match the partitions. */
-				return NULL;
-			}
+			/* Save the upper bounds of both partitions for use below. */
+			save_outer_ub = outer_ub;
+			save_inner_ub = inner_ub;
+
+			/* Move to the next pair of partitions. */
+			outer_part = get_range_partition(outer_bi, &outer_lb_index,
+											 &outer_lb, &outer_ub);
+			inner_part = get_range_partition(inner_bi, &inner_lb_index,
+											 &inner_lb, &inner_ub);
 
 			/*
-			 * If the ranges overlap but don't exactly match, a row from
-			 * non-overlapping portion of the range from one side of join may
-			 * find its join partner in the previous or next overlapping
-			 * partition or default partition on the other side , if such a
-			 * partition exists. All those cases, if true, will cause one
-			 * partition from that side to match at least two partitions on the
-			 * other side; a case that we do not support now. Previous
-			 * partition has been delt with in the previous iteration of this
-			 * loop, next partition will be delt in the next iteration. We will
-			 * deal with the default partition here.
+			 * If the range of a partition on one side overlaps the range of
+			 * the next partition on the other side, that will cause the
+			 * partition on one side to match at least two partitions on the
+			 * other side, which is the case that we currently don't support
+			 * partitioned join for; give up.
 			 */
-			if ((lb_cmpval < 0 && inner_has_default) ||
-				/* Non-overlapping range on the lower side of outer range. */
-				(lb_cmpval > 0 && outer_has_default) ||
-				/* Non-overlapping range on the lower side of inner range. */
-				(ub_cmpval < 0 && outer_has_default) ||
-				/* Non-overlapping range on the upper side of inner range. */
-				(ub_cmpval > 0 && inner_has_default))
-				/* Non-overlapping range on the upper side of outer range. */
+			if (ub_cmpval > 0 && inner_part >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &save_outer_ub, &inner_lb) > 0)
+				return NULL;
+			if (ub_cmpval < 0 && outer_part >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &outer_lb, &save_inner_ub) < 0)
 				return NULL;
-		}
-
-		if (ub_cmpval == 0)
-		{
-			/* Upper bounds of both the ranges match. */
-			Assert(overlap);
 
-			/* Move to the next pair of partitions. */
-			Assert(outer_lb_index < outer_bi->ndatums);
-			Assert(inner_lb_index < inner_bi->ndatums);
-			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
-															   outer_lb_index);
-			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
-															   inner_lb_index);
+			/*
+			 * A row from a non-overlapping portion (if any) of a partition
+			 * on one side might find its join partner in the default
+			 * partition (if any) on the other side, causing the same
+			 * situation as above; if so, give up.
+			 */
+			if ((outer_has_default && (lb_cmpval > 0 || ub_cmpval < 0)) ||
+				(inner_has_default && (lb_cmpval < 0 || ub_cmpval > 0)))
+				return NULL;
 		}
 		else if (ub_cmpval < 0)
 		{
 			/* Upper bound of inner range higher than that of the outer. */
 
-			if (overlap)
-			{
-				/* We have already dealt with overlapping ranges. */
-			}
-			else
-			{
-				if (inner_has_default || IS_OUTER_JOIN(jointype))
-				{
-					if (!process_outer_partition(&outer_map,
-												 &inner_map,
-												 outer_has_default,
-												 inner_has_default,
-												 outer_part,
-												 inner_default,
-												 jointype,
-												 outer_bi->strategy,
-												 &next_index,
-												 &default_index,
-												 &merged_index))
-						return NULL;
-				}
+			merged_lb = outer_lb;
+			merged_ub = outer_ub;
 
-				merged_lb = &outer_lb;
-				merged_ub = &outer_ub;
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				if (!process_outer_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 outer_part,
+											 inner_default,
+											 jointype,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
 			}
 
 			/* Move to the next partition on the outer side. */
-			Assert(outer_lb_index < outer_bi->ndatums);
-			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
-															   outer_lb_index);
+			outer_part = get_range_partition(outer_bi, &outer_lb_index,
+											 &outer_lb, &outer_ub);
 		}
 		else
 		{
+			/* Upper bound of outer range higher than that of the inner. */
 			Assert(ub_cmpval > 0);
 
-			/* Upper bound of outer range higher than that of the inner. */
-			if (overlap)
-			{
-				/* We have already dealt with overlapping ranges. */
-			}
-			else
-			{
-				if (outer_has_default || jointype == JOIN_FULL)
-				{
-					if (!process_inner_partition(&outer_map,
-												 &inner_map,
-												 outer_has_default,
-												 inner_has_default,
-												 inner_part,
-												 outer_default,
-												 jointype,
-												 outer_bi->strategy,
-												 &next_index,
-												 &default_index,
-												 &merged_index))
-						return NULL;
-				}
+			merged_lb = inner_lb;
+			merged_ub = inner_ub;
 
-				merged_lb = &inner_lb;
-				merged_ub = &inner_ub;
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				if (!process_inner_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 inner_part,
+											 outer_default,
+											 jointype,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
 			}
 
 			/* Move to the next partition on the inner side. */
-			Assert(inner_lb_index < inner_bi->ndatums);
-			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
-															   inner_lb_index);
+			inner_part = get_range_partition(inner_bi, &inner_lb_index,
+											 &inner_lb, &inner_ub);
 		}
 
-		if (merged_index < 0)
+		if (merged_index >= 0)
 		{
-			/* We didn't find a new merged partition. */
-			continue;
+			/* Add the range bounds of the merged partition. */
+			add_merged_range_bounds(partnatts, partsupfuncs, partcollations,
+									&merged_lb, &merged_ub, merged_index,
+									&merged_datums, &merged_kinds,
+									&merged_indexes);
 		}
-
-		/*
-		 * We have a valid partition index for the next partition of join. The
-		 * partition should have valid range.
-		 */
-		Assert(merged_lb && merged_ub);
-
-		/* Try merging new lower bound with the last upper bound. */
-		if (!partition_range_merge_next_lb(partnatts, partsupfuncs,
-										   partcollations,
-										   merged_lb->datums,
-										   merged_lb->kind, &merged_datums,
-										   &merged_kinds, &merged_indexes))
-			return NULL;
-
-		/* Add upper bound with the merged partition index. */
-		merged_datums = lappend(merged_datums, merged_ub->datums);
-		merged_kinds = lappend(merged_kinds, merged_ub->kind);
-		merged_indexes = lappend_int(merged_indexes, merged_index);
 	}
 
 	/* Merge default partitions if any. */
@@ -3696,6 +3664,10 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 	else
 		Assert(default_index == -1);
 
+	/*
+	 * Unlike the case for list partitioning, we wouldn't have re-merged
+	 * partitions, so did_remapping should be left alone.
+	 */
 	Assert(!outer_map.did_remapping && !inner_map.did_remapping);
 
 	/* Use maps to match partition from the joining relations. */
@@ -3856,7 +3828,6 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 											 o_index,
 											 inner_default,
 											 jointype,
-											 outer_bi->strategy,
 											 &next_index,
 											 &default_index,
 											 &merged_index))
@@ -3886,7 +3857,6 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 											 i_index,
 											 outer_default,
 											 jointype,
-											 outer_bi->strategy,
 											 &next_index,
 											 &default_index,
 											 &merged_index))
@@ -4150,7 +4120,6 @@ process_outer_partition(PartitionMap *outer_map,
 						int outer_index,
 						int inner_default,
 						JoinType jointype,
-						char strategy,
 						int *next_index,
 						int *default_index,
 						int *merged_index)
@@ -4203,22 +4172,8 @@ process_outer_partition(PartitionMap *outer_map,
 		Assert(IS_OUTER_JOIN(jointype));
 		Assert(jointype != JOIN_RIGHT);
 
-		/*
-		 * In range partitioning, if the given outer partition is already
-		 * merged (eg, because we found an overlapping range earlier), we know
-		 * where it fits in the join result; nothing to do in that case.  Else
-		 * create a new merged partition.
-		 */
 		if (outer_map->merged_indexes[outer_index] >= 0)
-		{
-			if (strategy == PARTITION_STRATEGY_LIST)
-				*merged_index = outer_map->merged_indexes[outer_index];
-			else
-			{
-				Assert(strategy == PARTITION_STRATEGY_RANGE);
-				*merged_index = -1;
-			}
-		}
+			*merged_index = outer_map->merged_indexes[outer_index];
 		else
 			*merged_index = merge_partition_with_dummy(outer_map, outer_index,
 													   next_index);
@@ -4242,7 +4197,6 @@ process_inner_partition(PartitionMap *outer_map,
 						int inner_index,
 						int outer_default,
 						JoinType jointype,
-						char strategy,
 						int *next_index,
 						int *default_index,
 						int *merged_index)
@@ -4295,22 +4249,8 @@ process_inner_partition(PartitionMap *outer_map,
 	{
 		Assert(jointype == JOIN_FULL);
 
-		/*
-		 * In range partitioning, if the given inner partition is already
-		 * merged (eg, because we found an overlapping range earlier), we know
-		 * where it fits in the join result; nothing to do in that case.  Else
-		 * create a new merged partition.
-		 */
 		if (inner_map->merged_indexes[inner_index] >= 0)
-		{
-			if (strategy == PARTITION_STRATEGY_LIST)
-				*merged_index = inner_map->merged_indexes[inner_index];
-			else
-			{
-				Assert(strategy == PARTITION_STRATEGY_RANGE);
-				*merged_index = -1;
-			}
-		}
+			*merged_index = inner_map->merged_indexes[inner_index];
 		else
 			*merged_index = merge_partition_with_dummy(inner_map, inner_index,
 													   next_index);
@@ -4628,7 +4568,6 @@ merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
 										 outer_bi->null_index,
 										 inner_bi->default_index,
 										 jointype,
-										 outer_bi->strategy,
 										 next_index,
 										 default_index,
 										 &merged_index))
@@ -4655,7 +4594,6 @@ merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
 										 inner_bi->null_index,
 										 outer_bi->default_index,
 										 jointype,
-										 outer_bi->strategy,
 										 next_index,
 										 default_index,
 										 &merged_index))
-- 
2.19.2

#105amul sul
sulamul@gmail.com
In reply to: Etsuro Fujita (#104)
5 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Wed, Nov 13, 2019 at 9:46 AM Etsuro Fujita <etsuro.fujita@gmail.com>
wrote:

Hi Amul,

On Wed, Nov 6, 2019 at 3:12 PM amul sul <sulamul@gmail.com> wrote:

Attached is the rebase atop the latest master head(a9056cc637f).

Thanks for that!

On Tue, Nov 5, 2019 at 6:44 PM amul sul <sulamul@gmail.com> wrote:

On Fri, Nov 1, 2019 at 3:58 PM Etsuro Fujita <etsuro.fujita@gmail.com>

wrote:

Done. Attached is a new version of the patch.

A query and comments for v25:

583 + * The function returns NULL if we can not find the matching pair

of

584 + * partitions. This happens if 1. multiple partitions on one side

match with

585 + * one partition on the other side. 2. a given partition on the

outer side

586 + * doesn't have a matching partition on the inner side. We can

not support the

587 + * first case since we don't have a way to represent multiple

partitions as a

588 + * single relation (RelOptInfo) and then perform join using the

ganged

589 + * relation. We can not support the second case since the missing

inner

590 + * partition needs to be represented as an empty relation and we

don't have a

591 + * way to introduce empty relation during join planning after

creating paths

592 + * for all the base relations.
593 + */
594 +PartitionBoundInfo
595 +partition_bounds_merge(int partnatts,

I think the second condition mention for partition_bounds_merge() looks
outdated, do you think we should remove that or am I missing something

here?

Yeah, I think so. In addition to that, the output arguments for that
function *outer_pars and *inner_parts don't store partition indexes
anymore, so I updated comments for that function totally. Does that
make sense?

Thanks, that's looks better.

[....]

I looked at partition_range_bounds_merge() more closely. Here are my
comments:

* When merging partition bounds from the outer/inner sides in the
while loop of that function, we only moves to the next partition on
one side when both partitions overlap (ie, overlap=true) and the upper
bounds of partitions are not the same (ie, ub_cmpval<0 or
ub_cmpval>0); but I think that's inefficient. Let me explain using an
example:

create table prt1 (a int, b int) partition by range (a);
create table prt1_p1 partition of prt1 for values from (0) to (50);
create table prt1_p2 partition of prt1 for values from (100) to (150);
create table prt2 (a int, b int) partition by range (a);
create table prt2_p1 partition of prt2 for values from (0) to (100);
create table prt2_p2 partition of prt2 for values from (100) to (200);
insert into prt1 select i, i from generate_series(0, 49) i;
insert into prt1 select i, i from generate_series(100, 149) i;
insert into prt2 select i, i from generate_series(0, 49) i;
insert into prt2 select i, i from generate_series(100, 149) i;
analyze prt1;
analyze prt2;
set enable_partitionwise_join to on;
explain verbose select * from prt1 t1 full join prt2 t2 on (t1.a = t2.a);
QUERY PLAN

--------------------------------------------------------------------------------------
Append (cost=2.12..9.12 rows=100 width=16)
-> Hash Full Join (cost=2.12..4.31 rows=50 width=16)
Output: t1.a, t1.b, t2.a, t2.b
Hash Cond: (t1.a = t2.a)
-> Seq Scan on public.prt1_p1 t1 (cost=0.00..1.50 rows=50
width=8)
Output: t1.a, t1.b
-> Hash (cost=1.50..1.50 rows=50 width=8)
Output: t2.a, t2.b
-> Seq Scan on public.prt2_p1 t2 (cost=0.00..1.50
rows=50 width=8)
Output: t2.a, t2.b
-> Hash Full Join (cost=2.12..4.31 rows=50 width=16)
Output: t1_1.a, t1_1.b, t2_1.a, t2_1.b
Hash Cond: (t1_1.a = t2_1.a)
-> Seq Scan on public.prt1_p2 t1_1 (cost=0.00..1.50 rows=50
width=8)
Output: t1_1.a, t1_1.b
-> Hash (cost=1.50..1.50 rows=50 width=8)
Output: t2_1.a, t2_1.b
-> Seq Scan on public.prt2_p2 t2_1 (cost=0.00..1.50
rows=50 width=8)
Output: t2_1.a, t2_1.b
(19 rows)

For this query, the merging would proceed as follow:

1) In the first iteration of the while loop, we have overlap=true, so
we merge prt1_p1 and prt2_p1 and move to the next partition on *only
the outer side* (prt1_p2) as we have ub_cmpval<0. 2) In the second
iteration, we have overlap=false and ub_cmpval>0, so we perform
process_inner_partition() to prt2_p1 and move to the next partition on
the inner side (prt2_p2). 3) In the third iteration, we have
overlap=true, so we merge prt1_p2 and prt2_p2 and move to the next
partition on *only the outer side* as we have ub_cmpval<0 (but can't
as the outer side is exhausted). 4) In the forth iteration, we have
overlap=false and ub_cmpval>0, so we perform process_inner_partition()
to prt2_p2 and move to the next partition on the inner side (but can't
as the inner side is exhausted). And we are done.

We do this in the process_inner_partition() call in the second and
forth iterations:

/*
* In range partitioning, if the given inner partition is already
* merged (eg, because we found an overlapping range earlier), we
know
* where it fits in the join result; nothing to do in that case.
Else
* create a new merged partition.
*/
if (inner_map->merged_indexes[inner_index] >= 0)
{
if (strategy == PARTITION_STRATEGY_LIST)
*merged_index = inner_map->merged_indexes[inner_index];
else
{
Assert(strategy == PARTITION_STRATEGY_RANGE);
*merged_index = -1;
}
}

What we do in that function is actually a no-op, so the second and
forth iterations are done only to move to the next partition on the
inner side, which I think is inefficient. To avoid that, why not move
to *the next pair of partitions* in the first and third iterations
like the attached? This needs an extra check to see if we can safely
move to the next pair of partitions in the first and third iterations,
but doesn't need meaningless iterations like the second and fourth
iterations in the example. This change also makes the code in
process_inner_partition() shown above unnecessary, so I removed it
from that function (and process_outer_partition()).

Make sense.

* I don't like this:

+           if ((lb_cmpval < 0 && inner_has_default) ||
+               /* Non-overlapping range on the lower side of outer range.
*/
+               (lb_cmpval > 0 && outer_has_default) ||
+               /* Non-overlapping range on the lower side of inner range.
*/
+               (ub_cmpval < 0 && outer_has_default) ||
+               /* Non-overlapping range on the upper side of inner range.
*/
+               (ub_cmpval > 0 && inner_has_default))
+               /* Non-overlapping range on the upper side of outer range.
*/
+               return NULL;

because it looks like that eg, the first comment "Non-overlapping
range on the lower side of outer range." explains the second condition
"(lb_cmpval > 0 && outer_has_default)", which isn't intended, I think.
Also, it seems a bit verbose to me to add a comment to each of the
four cases. How about simplifying the comments and moving them to
above the if block like the attached?

Check.

* In partition_range_merge_next_lb(), do we really need this bit?

+   /*
+    * The lower bound is lower than the last upper bound, thus does not
fit
+    * the bounds created so far and hence can not be merged with the
existing
+    * bounds.
+    */
+   if (cmpval < 0)
+       return false;

I think that if we have such a lower bound, we would error out before
calling that function. Also, I think it would be easier to understand
to add the upper bound as well within that function. So I modified
that function that way (and renamed it). I also added a bit of
comments to that function.

* Since this is called many times, I changed it to a macro:

+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+                         Oid *partcollations, PartitionRangeBound *bound1,
+                         PartitionRangeBound *bound2)
+{
+   return partition_rbound_cmp(partnatts, partsupfunc, partcollations,
+                               bound1->datums, bound1->kind,
bound1->lower,
+                               bound2);
+}

Check.

* This is just my taste, but how about renaming
partition_range_bound_cmp() and partition_range_cmp() to more common?
or readable? names eg, compare_range_bounds() and
compare_range_partitions(), respectively? Also, how about renaming
partition_range_merge() to eg, get_merged_range_bounds()?

Check.

Please find attached the delta patch for that. Any feedback welcome!

Thank you Fujita san for the patch & the enhancements. I am fine with your
delta patch. I would like to share some thought for other code:

File: partbounds.c:
3298 static void
3299 get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
3300 Oid *partcollations, JoinType jointype,
3301 PartitionRangeBound *left_lb,
3302 PartitionRangeBound *left_ub,
3303 PartitionRangeBound *right_lb,
3304 PartitionRangeBound *right_ub,
3305 PartitionRangeBound *merged_lb,
3306 PartitionRangeBound *merged_ub)

Can we rename these argument as inner_* & outer_* like we having for the
functions, like 0003 patch?
---

File: partbounds.c:
3322
3323 case JOIN_INNER:
3324 case JOIN_SEMI:
3325 if (compare_range_bounds(partnatts, partsupfuncs,
partcollations,
3326 left_ub, right_ub) < 0)
3327 *merged_ub = *left_ub;
3328 else
3329 *merged_ub = *right_ub;
3330
3331 if (compare_range_bounds(partnatts, partsupfuncs,
partcollations,
3332 left_lb, right_lb) > 0)
3333 *merged_lb = *left_lb;
3334 else
3335 *merged_lb = *right_lb;
3336 break;
3337

How about reusing ub_cmpval & lb_cmpval instead of calling
compare_range_bounds() inside get_merged_range_bounds(), like 0004 patch?
--

Apart from this, I would like to propose 0005 cleanup patch where I have
rearranged function arguments & code to make sure the arguments & the code
related to lower bound should come first and then the upper bound.

Regards,
Amul

P.S: As usual, reattaching 0001 & 0002 patches.

Attachments:

v27-0002-Modify-partition_range_bounds_merge.patchapplication/octet-stream; name=v27-0002-Modify-partition_range_bounds_merge.patchDownload
From 9d51a459c38d9e3cb7c39a09e6aa3e35461882a5 Mon Sep 17 00:00:00 2001
From: Etsuro Fujita <efujita@postgresql.org>
Date: Tue, 12 Nov 2019 21:58:34 +0900
Subject: [PATCH 2/5] Modify partition_range_bounds_merge()

---
 src/backend/partitioning/partbounds.c | 594 ++++++++++++--------------
 1 file changed, 266 insertions(+), 328 deletions(-)

diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index f1458518716..4aebc71ebb9 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -81,6 +81,12 @@ typedef struct PartitionMap
 								 * did_remapping */
 } PartitionMap;
 
+#define compare_range_bounds(partnatts, partsupfunc, partcollations, \
+							 bound1, bound2) \
+	(partition_rbound_cmp(partnatts, partsupfunc, partcollations, \
+						  (bound1)->datums, (bound1)->kind, (bound1)->lower, \
+						  bound2))
+
 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);
@@ -142,7 +148,6 @@ static bool process_outer_partition(PartitionMap *outer_map,
 						int outer_index,
 						int inner_default,
 						JoinType jointype,
-						char strategy,
 						int *next_index,
 						int *default_index,
 						int *merged_index);
@@ -153,7 +158,6 @@ static bool process_inner_partition(PartitionMap *outer_map,
 						int inner_index,
 						int outer_default,
 						JoinType jointype,
-						char strategy,
 						int *next_index,
 						int *default_index,
 						int *merged_index);
@@ -170,20 +174,31 @@ static PartitionBoundInfo build_merged_partition_bounds(char strategy,
 							  List *merged_datums, List *merged_indexes,
 							  List *merged_contents, int null_index,
 							  int default_index);
-static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
-						  Oid *collations, PartitionRangeBound *bound1,
-						  PartitionRangeBound *bound2);
-static bool partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
-						   Oid *collations, PartitionRangeBound *lower_bound1,
-						   PartitionRangeBound *upper_bound1,
-						   PartitionRangeBound *lower_bound2,
-						   PartitionRangeBound *upper_bound2, int *ub_cmpval,
-						   int *lb_cmpval);
-static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
-							  Oid *collations, Datum *next_lb_datums,
-							  PartitionRangeDatumKind *next_lb_kind,
-							  List **merged_datums, List **merged_kinds,
-							  List **merged_indexes);
+static int get_range_partition(PartitionBoundInfo bi, int *lb_pos,
+					PartitionRangeBound *lb, PartitionRangeBound *ub);
+static bool compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+						 Oid *partcollations,
+						 PartitionRangeBound *lower_bound1,
+						 PartitionRangeBound *upper_bound1,
+						 PartitionRangeBound *lower_bound2,
+						 PartitionRangeBound *upper_bound2,
+						 int *ub_cmpval, int *lb_cmpval);
+static void get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations, JoinType jointype,
+						PartitionRangeBound *left_lb,
+						PartitionRangeBound *left_ub,
+						PartitionRangeBound *right_lb,
+						PartitionRangeBound *right_ub,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub);
+static void add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub,
+						int merged_index,
+						List **merged_datums,
+						List **merged_kinds,
+						List **merged_indexes);
 static bool merge_default_partitions(PartitionMap *outer_map,
 						 PartitionMap *inner_map,
 						 bool outer_has_default,
@@ -3164,83 +3179,60 @@ partition_bounds_merge(int partnatts,
 }
 
 /*
- * partition_get_range_bounds
+ * get_range_partition
+ *		Returns the index of the range partition with the given lower bound
  *
- * Given the index of lower bound in datums array, return lower and upper
- * bounds and the index of the partition with that lower bound.
+ * *lb and *ub are set to the lower and upper bounds of the range partition
+ * respectively, and *lb_index is advanced to the next lower bound, if any.
  */
 static int
-partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
-						   PartitionRangeBound *lower,
-						   PartitionRangeBound *upper)
+get_range_partition(PartitionBoundInfo bi, int *lb_index,
+					PartitionRangeBound *lb, PartitionRangeBound *ub)
 {
-	int			part_index;
+	/* Return the index as -1 if we've exhausted all the lower bounds. */
+	if (*lb_index >= bi->ndatums)
+		return -1;
 
 	/* A lower bound should have at least one more bound after it. */
-	Assert(lb_index < bi->ndatums - 1);
-
-	/* The lower bound should correspond to a valid partition. */
-	part_index = bi->indexes[lb_index + 1];
-	Assert(part_index >= 0);
-
-	lower->kind = bi->kind[lb_index];
-	lower->datums = bi->datums[lb_index];
-	lower->lower = true;
-	upper->kind = bi->kind[lb_index + 1];
-	upper->datums = bi->datums[lb_index + 1];
-	upper->lower = false;
+	Assert(*lb_index + 1 < bi->ndatums);
 
-	return part_index;
-}
-
-/*
- * partition_range_get_next_lb_index
- *
- * Given the index of lower bound in datums array return the
- * index of lower bound of the next partition. When the given index corresponds
- * to the last partition, return number of datums (ndatums).
- */
-static int
-partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
-{
-	/* A lower bound should have at least one more bound after it. */
-	Assert(lb_index < bi->ndatums - 1);
+	lb->index = bi->indexes[*lb_index];
+	lb->kind = bi->kind[*lb_index];
+	lb->datums = bi->datums[*lb_index];
+	lb->lower = true;
+	ub->index = bi->indexes[*lb_index + 1];
+	ub->kind = bi->kind[*lb_index + 1];
+	ub->datums = bi->datums[*lb_index + 1];
+	ub->lower = false;
 
-	/* The partition index corresponding to the upper bound should be valid. */
-	Assert(bi->indexes[lb_index + 1] >= 0);
+	/* The partition index of an upper bound should be valid. */
+	Assert(ub->index >= 0);
 
 	/*
-	 * If there are no bounds left beyond the upper bound, we have reached the
-	 * last partition.
+	 * Advance the lower bound; if there are no bounds left beyond the upper
+	 * bound, we have reached the last lower bound.
 	 */
-	if (lb_index + 2 < bi->ndatums)
+	if (*lb_index + 2 >= bi->ndatums)
+		*lb_index = bi->ndatums;
+	else
 	{
 		/*
-		 * If the bound next to the upper bound corresponds to no partition,
-		 * that's the next lower bound of the next partition. Otherwise, the
-		 * current upper bound is the lower bound of the next partition.
+		 * If the index assigned to the bound next to the upper bound isn't
+		 * valid, that is the lower bound of the next range partition; else,
+		 * the upper bound of the current range partition is also the lower
+		 * bound of the next range partition.
 		 */
-		if (bi->indexes[lb_index + 2] < 0)
-			return lb_index + 2;
+		if (bi->indexes[*lb_index + 2] < 0)
+			*lb_index = *lb_index + 2;
 		else
-			return lb_index + 1;
+			*lb_index = *lb_index + 1;
 	}
-	else
-		return bi->ndatums;
-}
 
-static int32
-partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
-						  Oid *partcollations, PartitionRangeBound *bound1,
-						  PartitionRangeBound *bound2)
-{
-	return partition_rbound_cmp(partnatts, partsupfunc, partcollations,
-								bound1->datums, bound1->kind, bound1->lower,
-								bound2);
+	return ub->index;
 }
 
 /*
- * partition_range_cmp
+ * compare_range_partitions
  *
  * Compare the bounds of two range partitions. Set ub_cmpval <, = or > 0, if the
  * first partition's upper bound is lower than, equal to or higher than the
@@ -3251,12 +3243,13 @@ partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
  * Return true, if the ranges overlap, otherwise return false.
  */
 static bool
-partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
-					PartitionRangeBound *lower_bound1,
-					PartitionRangeBound *upper_bound1,
-					PartitionRangeBound *lower_bound2,
-					PartitionRangeBound *upper_bound2, int *ub_cmpval,
-					int *lb_cmpval)
+compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+						 Oid *partcollations,
+						 PartitionRangeBound *lower_bound1,
+						 PartitionRangeBound *upper_bound1,
+						 PartitionRangeBound *lower_bound2,
+						 PartitionRangeBound *upper_bound2,
+						 int *ub_cmpval, int *lb_cmpval)
 {
 	bool		overlap;
 
@@ -3266,15 +3259,15 @@ partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
 	 * the partitions are not overlapping. All other cases indicate overlapping
 	 * partitions.
 	 */
-	if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
-								  lower_bound1, upper_bound2) > 0)
+	if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+							 lower_bound1, upper_bound2) > 0)
 	{
 		overlap = false;
 		*ub_cmpval = 1;
 		*lb_cmpval = 1;
 	}
-	else if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
-									   lower_bound2, upper_bound1) > 0)
+	else if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								  lower_bound2, upper_bound1) > 0)
 	{
 		overlap = false;
 		*ub_cmpval = -1;
@@ -3283,35 +3276,34 @@ partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
 	else
 	{
 		overlap = true;
-		*ub_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
-											   partcollations, upper_bound1,
-											   upper_bound2);
-		*lb_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
-											   partcollations, lower_bound1,
-											   lower_bound2);
+		*ub_cmpval = compare_range_bounds(partnatts, partsupfuncs,
+										  partcollations, upper_bound1,
+										  upper_bound2);
+		*lb_cmpval = compare_range_bounds(partnatts, partsupfuncs,
+										  partcollations, lower_bound1,
+										  lower_bound2);
 	}
 
 	return overlap;
 }
 
 /*
- * partition_range_merge
- *
- * Merge the partition bounds of given two partitions such that the join
- * between the given two partitions fits merged bounds.
+ * get_merged_range_bounds
+ *		Given the bounds of range partitions to be join, determine the range
+ *		bounds of the merged partition produced from the range partitions
  *
- * "merged_upper" will be set to one of the given upper bounds and
- * "merged_lower" will be set to one of the given lower bounds.
+ * *merged_lb and *merged_ub are set to the lower and upper bounds of the
+ * merged partition.
  */
 static void
-partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
-					  Oid *partcollations, JoinType jointype,
-					  PartitionRangeBound *left_lb,
-					  PartitionRangeBound *left_ub,
-					  PartitionRangeBound *right_lb,
-					  PartitionRangeBound *right_ub,
-					  PartitionRangeBound **merged_lb,
-					  PartitionRangeBound **merged_ub)
+get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations, JoinType jointype,
+						PartitionRangeBound *left_lb,
+						PartitionRangeBound *left_ub,
+						PartitionRangeBound *right_lb,
+						PartitionRangeBound *right_ub,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub)
 {
 	/*
 	 * An outer join will have all the rows from the outer side, so merged
@@ -3324,41 +3316,37 @@ partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
 	{
 		case JOIN_LEFT:
 		case JOIN_ANTI:
-			*merged_ub = left_ub;
-			*merged_lb = left_lb;
+			*merged_ub = *left_ub;
+			*merged_lb = *left_lb;
 			break;
 
 		case JOIN_INNER:
 		case JOIN_SEMI:
-			if (partition_range_bound_cmp(partnatts, partsupfuncs,
-										  partcollations, left_ub,
-										  right_ub) < 0)
-				*merged_ub = left_ub;
+			if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 left_ub, right_ub) < 0)
+				*merged_ub = *left_ub;
 			else
-				*merged_ub = right_ub;
+				*merged_ub = *right_ub;
 
-			if (partition_range_bound_cmp(partnatts, partsupfuncs,
-										  partcollations, left_lb,
-										  right_lb) > 0)
-				*merged_lb = left_lb;
+			if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 left_lb, right_lb) > 0)
+				*merged_lb = *left_lb;
 			else
-				*merged_lb = right_lb;
+				*merged_lb = *right_lb;
 			break;
 
 		case JOIN_FULL:
-			if (partition_range_bound_cmp(partnatts, partsupfuncs,
-										  partcollations, left_ub,
-										  right_ub) > 0)
-				*merged_ub = left_ub;
+			if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 left_ub, right_ub) > 0)
+				*merged_ub = *left_ub;
 			else
-				*merged_ub = right_ub;
+				*merged_ub = *right_ub;
 
-			if (partition_range_bound_cmp(partnatts, partsupfuncs,
-										  partcollations, left_lb,
-										  right_lb) < 0)
-				*merged_lb = left_lb;
+			if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 left_lb, right_lb) < 0)
+				*merged_lb = *left_lb;
 			else
-				*merged_lb = right_lb;
+				*merged_lb = *right_lb;
 			break;
 
 		default:
@@ -3367,16 +3355,19 @@ partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
 }
 
 /*
- * Add the lower bound of the next range to the list of bounds, if the lower
- * bound is higher or equal to the previous upper bound. If successful return
- * true, otherwise false.
+ * add_merged_range_bounds
+ *		Add the range bounds of a merged partition to the lists of range
+ *		bounds
  */
-static bool
-partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
-							  Oid *partcollations, Datum *next_lb_datums,
-							  PartitionRangeDatumKind *next_lb_kind,
-							  List **merged_datums, List **merged_kinds,
-							  List **merged_indexes)
+static void
+add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub,
+						int merged_index,
+						List **merged_datums,
+						List **merged_kinds,
+						List **merged_indexes)
 {
 	int			cmpval;
 
@@ -3389,38 +3380,43 @@ partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
 	{
 		PartitionRangeBound	prev_ub;
 
-		prev_ub.datums = llast(*merged_datums);
-		prev_ub.kind = llast(*merged_kinds);
+		Assert(*merged_kinds && *merged_indexes);
+
+		/* Get the last upper bound. */
+		prev_ub.index = llast_int(*merged_indexes);
+		prev_ub.datums = (Datum *) llast(*merged_datums);
+		prev_ub.kind = (PartitionRangeDatumKind *) llast(*merged_kinds);
 		prev_ub.lower = false;
 
+		/*
+		 * We pass to partition_rbound_cmp() lower1 as false to prevent it
+		 * from considering the last upper bound to be smaller than the lower
+		 * bound of the merged partition when the values of the two range
+		 * bounds compare equal.
+		 */
 		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
-									  next_lb_datums, next_lb_kind, false,
-									  &prev_ub);
+									  merged_lb->datums, merged_lb->kind,
+									  false, &prev_ub);
+		Assert(cmpval >= 0);
 	}
 
 	/*
-	 * The lower bound is lower than the last upper bound, thus does not fit
-	 * the bounds created so far and hence can not be merged with the existing
-	 * bounds.
-	 */
-	if (cmpval < 0)
-		return false;
-
-	/*
-	 * Add bounds of the new merged partition. If the next lower bound is
-	 * higher than the last upper bound, add new range with index
-	 * corresponding to the lower bound as -1. If the merged lower bound
-	 * is same as the last merged upper bound, the last upper bound will be
-	 * reused as the lower bound of the next range.
+	 * If the lower bound is higher than the last upper bound, add the lower
+	 * bound with the index as -1 indicating that that is a lower bound; else,
+	 * the last upper bound will be reused as the lower bound of the merged
+	 * partition, so skip this.
 	 */
 	if (cmpval > 0)
 	{
-		*merged_datums = lappend(*merged_datums, next_lb_datums);
-		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_datums = lappend(*merged_datums, merged_lb->datums);
+		*merged_kinds = lappend(*merged_kinds, merged_lb->kind);
 		*merged_indexes = lappend_int(*merged_indexes, -1);
 	}
 
-	return true;
+	/* Add the upper bound and index of the merged partition. */
+	*merged_datums = lappend(*merged_datums, merged_ub->datums);
+	*merged_kinds = lappend(*merged_kinds, merged_ub->kind);
+	*merged_indexes = lappend_int(*merged_indexes, merged_index);
 }
 
 /*
@@ -3444,13 +3440,19 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 	int			inner_default = inner_bi->default_index;
 	PartitionMap outer_map;
 	PartitionMap inner_map;
+	int			outer_part;
+	int			inner_part;
+	int			outer_lb_index;
+	int			inner_lb_index;
+	PartitionRangeBound outer_lb;
+	PartitionRangeBound outer_ub;
+	PartitionRangeBound inner_lb;
+	PartitionRangeBound inner_ub;
 	int			next_index = 0;
 	int			default_index = -1;
 	List	   *merged_datums = NIL;
 	List	   *merged_kinds = NIL;
 	List	   *merged_indexes = NIL;
-	int			outer_lb_index;
-	int			inner_lb_index;
 
 	Assert(outer_bi->strategy == inner_bi->strategy &&
 		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
@@ -3471,29 +3473,25 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 	 * datum in PartitionBoundInfo::datums of that side.
 	 */
 	outer_lb_index = inner_lb_index = 0;
-	while (outer_lb_index < outer_bi->ndatums ||
-		   inner_lb_index < inner_bi->ndatums)
+	outer_part = get_range_partition(outer_bi, &outer_lb_index,
+									 &outer_lb, &outer_ub);
+	inner_part = get_range_partition(inner_bi, &inner_lb_index,
+									 &inner_lb, &inner_ub);
+	while (outer_part >= 0 || inner_part >= 0)
 	{
-		PartitionRangeBound *merged_lb = NULL;
-		PartitionRangeBound *merged_ub = NULL;
+		PartitionRangeBound merged_lb;
+		PartitionRangeBound merged_ub;
 		int			merged_index = -1;
-		PartitionRangeBound outer_lb;
-		PartitionRangeBound outer_ub;
-		PartitionRangeBound inner_lb;
-		PartitionRangeBound inner_ub;
-		int			outer_part = -1;
-		int			inner_part = -1;
 		bool		overlap;
 		int			ub_cmpval;
 		int			lb_cmpval;
 
-		/* Get the range bounds of the next pair of partitions. */
-		if (outer_lb_index < outer_bi->ndatums)
-			outer_part = partition_get_range_bounds(outer_bi, outer_lb_index,
-													&outer_lb, &outer_ub);
-		if (inner_lb_index < inner_bi->ndatums)
-			inner_part = partition_get_range_bounds(inner_bi, inner_lb_index,
-													&inner_lb, &inner_ub);
+		if (outer_part >= 0)
+			Assert(outer_map.merged_indexes[outer_part] == -1 &&
+				   outer_map.merged[outer_part] == false);
+		if (inner_part >= 0)
+			Assert(inner_map.merged_indexes[inner_part] == -1 &&
+				   inner_map.merged[inner_part] == false);
 
 		/*
 		 * We run this loop till both the sides finish. This allows to avoid
@@ -3505,25 +3503,30 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 		 * side. That way we advance the partitions on that side till all of
 		 * them are  exhausted.
 		 */
-		if (outer_lb_index >= outer_bi->ndatums)
+		if (outer_part == -1)
 		{
 			overlap = false;
 			ub_cmpval = 1;
 			lb_cmpval = 1;
 		}
-		else if (inner_lb_index >= inner_bi->ndatums)
+		else if (inner_part == -1)
 		{
 			overlap = false;
 			ub_cmpval = -1;
 			lb_cmpval = -1;
 		}
 		else
-			overlap = partition_range_cmp(partnatts, partsupfuncs, partcollations,
-										  &outer_lb, &outer_ub, &inner_lb,
-										  &inner_ub, &ub_cmpval, &lb_cmpval);
+			overlap = compare_range_partitions(partnatts, partsupfuncs,
+											   partcollations,
+											   &outer_lb, &outer_ub,
+											   &inner_lb, &inner_ub,
+											   &ub_cmpval, &lb_cmpval);
 
 		if (overlap)
 		{
+			PartitionRangeBound save_outer_ub;
+			PartitionRangeBound save_inner_ub;
+
 			/*
 			 * The rows from overlapping portion of ranges on both sides may
 			 * join, hence the corresponding pair of partitions form a joining
@@ -3531,152 +3534,117 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 			 * and its index by merging the bounds according to the type of
 			 * join.
 			 */
-			partition_range_merge(partnatts, partsupfuncs, partcollations,
-								  jointype, &outer_lb, &outer_ub, &inner_lb,
-								  &inner_ub, &merged_lb, &merged_ub);
+			get_merged_range_bounds(partnatts, partsupfuncs, partcollations,
+									jointype, &outer_lb, &outer_ub, &inner_lb,
+									&inner_ub, &merged_lb, &merged_ub);
 
+			/*
+			 * Both partitions are not merged yet, so they should be merged
+			 * successfully.
+			 */
 			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
 													outer_part, inner_part,
 													&next_index);
+			Assert(merged_index >= 0);
 
-			if (merged_index < 0)
-			{
-				/* Failed to match the partitions. */
-				return NULL;
-			}
+			/* Save the upper bounds of both partitions for use below. */
+			save_outer_ub = outer_ub;
+			save_inner_ub = inner_ub;
+
+			/* Move to the next pair of partitions. */
+			outer_part = get_range_partition(outer_bi, &outer_lb_index,
+											 &outer_lb, &outer_ub);
+			inner_part = get_range_partition(inner_bi, &inner_lb_index,
+											 &inner_lb, &inner_ub);
 
 			/*
-			 * If the ranges overlap but don't exactly match, a row from
-			 * non-overlapping portion of the range from one side of join may
-			 * find its join partner in the previous or next overlapping
-			 * partition or default partition on the other side , if such a
-			 * partition exists. All those cases, if true, will cause one
-			 * partition from that side to match at least two partitions on the
-			 * other side; a case that we do not support now. Previous
-			 * partition has been delt with in the previous iteration of this
-			 * loop, next partition will be delt in the next iteration. We will
-			 * deal with the default partition here.
+			 * If the range of a partition on one side overlaps the range of
+			 * the next partition on the other side, that will cause the
+			 * partition on one side to match at least two partitions on the
+			 * other side, which is the case that we currently don't support
+			 * partitioned join for; give up.
 			 */
-			if ((lb_cmpval < 0 && inner_has_default) ||
-				/* Non-overlapping range on the lower side of outer range. */
-				(lb_cmpval > 0 && outer_has_default) ||
-				/* Non-overlapping range on the lower side of inner range. */
-				(ub_cmpval < 0 && outer_has_default) ||
-				/* Non-overlapping range on the upper side of inner range. */
-				(ub_cmpval > 0 && inner_has_default))
-				/* Non-overlapping range on the upper side of outer range. */
+			if (ub_cmpval > 0 && inner_part >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &save_outer_ub, &inner_lb) > 0)
+				return NULL;
+			if (ub_cmpval < 0 && outer_part >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &outer_lb, &save_inner_ub) < 0)
 				return NULL;
-		}
-
-		if (ub_cmpval == 0)
-		{
-			/* Upper bounds of both the ranges match. */
-			Assert(overlap);
 
-			/* Move to the next pair of partitions. */
-			Assert(outer_lb_index < outer_bi->ndatums);
-			Assert(inner_lb_index < inner_bi->ndatums);
-			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
-															   outer_lb_index);
-			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
-															   inner_lb_index);
+			/*
+			 * A row from a non-overlapping portion (if any) of a partition
+			 * on one side might find its join partner in the default
+			 * partition (if any) on the other side, causing the same
+			 * situation as above; if so, give up.
+			 */
+			if ((outer_has_default && (lb_cmpval > 0 || ub_cmpval < 0)) ||
+				(inner_has_default && (lb_cmpval < 0 || ub_cmpval > 0)))
+				return NULL;
 		}
 		else if (ub_cmpval < 0)
 		{
 			/* Upper bound of inner range higher than that of the outer. */
 
-			if (overlap)
-			{
-				/* We have already dealt with overlapping ranges. */
-			}
-			else
-			{
-				if (inner_has_default || IS_OUTER_JOIN(jointype))
-				{
-					if (!process_outer_partition(&outer_map,
-												 &inner_map,
-												 outer_has_default,
-												 inner_has_default,
-												 outer_part,
-												 inner_default,
-												 jointype,
-												 outer_bi->strategy,
-												 &next_index,
-												 &default_index,
-												 &merged_index))
-						return NULL;
-				}
+			merged_lb = outer_lb;
+			merged_ub = outer_ub;
 
-				merged_lb = &outer_lb;
-				merged_ub = &outer_ub;
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				if (!process_outer_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 outer_part,
+											 inner_default,
+											 jointype,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
 			}
 
 			/* Move to the next partition on the outer side. */
-			Assert(outer_lb_index < outer_bi->ndatums);
-			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
-															   outer_lb_index);
+			outer_part = get_range_partition(outer_bi, &outer_lb_index,
+											 &outer_lb, &outer_ub);
 		}
 		else
 		{
+			/* Upper bound of outer range higher than that of the inner. */
 			Assert(ub_cmpval > 0);
 
-			/* Upper bound of outer range higher than that of the inner. */
-			if (overlap)
-			{
-				/* We have already dealt with overlapping ranges. */
-			}
-			else
-			{
-				if (outer_has_default || jointype == JOIN_FULL)
-				{
-					if (!process_inner_partition(&outer_map,
-												 &inner_map,
-												 outer_has_default,
-												 inner_has_default,
-												 inner_part,
-												 outer_default,
-												 jointype,
-												 outer_bi->strategy,
-												 &next_index,
-												 &default_index,
-												 &merged_index))
-						return NULL;
-				}
+			merged_lb = inner_lb;
+			merged_ub = inner_ub;
 
-				merged_lb = &inner_lb;
-				merged_ub = &inner_ub;
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				if (!process_inner_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 inner_part,
+											 outer_default,
+											 jointype,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
 			}
 
 			/* Move to the next partition on the inner side. */
-			Assert(inner_lb_index < inner_bi->ndatums);
-			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
-															   inner_lb_index);
+			inner_part = get_range_partition(inner_bi, &inner_lb_index,
+											 &inner_lb, &inner_ub);
 		}
 
-		if (merged_index < 0)
+		if (merged_index >= 0)
 		{
-			/* We didn't find a new merged partition. */
-			continue;
+			/* Add the range bounds of the merged partition. */
+			add_merged_range_bounds(partnatts, partsupfuncs, partcollations,
+									&merged_lb, &merged_ub, merged_index,
+									&merged_datums, &merged_kinds,
+									&merged_indexes);
 		}
-
-		/*
-		 * We have a valid partition index for the next partition of join. The
-		 * partition should have valid range.
-		 */
-		Assert(merged_lb && merged_ub);
-
-		/* Try merging new lower bound with the last upper bound. */
-		if (!partition_range_merge_next_lb(partnatts, partsupfuncs,
-										   partcollations,
-										   merged_lb->datums,
-										   merged_lb->kind, &merged_datums,
-										   &merged_kinds, &merged_indexes))
-			return NULL;
-
-		/* Add upper bound with the merged partition index. */
-		merged_datums = lappend(merged_datums, merged_ub->datums);
-		merged_kinds = lappend(merged_kinds, merged_ub->kind);
-		merged_indexes = lappend_int(merged_indexes, merged_index);
 	}
 
 	/* Merge default partitions if any. */
@@ -3696,6 +3664,10 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 	else
 		Assert(default_index == -1);
 
+	/*
+	 * Unlike the case for list partitioning, we wouldn't have re-merged
+	 * partitions, so did_remapping should be left alone.
+	 */
 	Assert(!outer_map.did_remapping && !inner_map.did_remapping);
 
 	/* Use maps to match partition from the joining relations. */
@@ -3856,7 +3828,6 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 											 o_index,
 											 inner_default,
 											 jointype,
-											 outer_bi->strategy,
 											 &next_index,
 											 &default_index,
 											 &merged_index))
@@ -3886,7 +3857,6 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 											 i_index,
 											 outer_default,
 											 jointype,
-											 outer_bi->strategy,
 											 &next_index,
 											 &default_index,
 											 &merged_index))
@@ -4150,7 +4120,6 @@ process_outer_partition(PartitionMap *outer_map,
 						int outer_index,
 						int inner_default,
 						JoinType jointype,
-						char strategy,
 						int *next_index,
 						int *default_index,
 						int *merged_index)
@@ -4203,22 +4172,8 @@ process_outer_partition(PartitionMap *outer_map,
 		Assert(IS_OUTER_JOIN(jointype));
 		Assert(jointype != JOIN_RIGHT);
 
-		/*
-		 * In range partitioning, if the given outer partition is already
-		 * merged (eg, because we found an overlapping range earlier), we know
-		 * where it fits in the join result; nothing to do in that case.  Else
-		 * create a new merged partition.
-		 */
 		if (outer_map->merged_indexes[outer_index] >= 0)
-		{
-			if (strategy == PARTITION_STRATEGY_LIST)
-				*merged_index = outer_map->merged_indexes[outer_index];
-			else
-			{
-				Assert(strategy == PARTITION_STRATEGY_RANGE);
-				*merged_index = -1;
-			}
-		}
+			*merged_index = outer_map->merged_indexes[outer_index];
 		else
 			*merged_index = merge_partition_with_dummy(outer_map, outer_index,
 													   next_index);
@@ -4242,7 +4197,6 @@ process_inner_partition(PartitionMap *outer_map,
 						int inner_index,
 						int outer_default,
 						JoinType jointype,
-						char strategy,
 						int *next_index,
 						int *default_index,
 						int *merged_index)
@@ -4295,22 +4249,8 @@ process_inner_partition(PartitionMap *outer_map,
 	{
 		Assert(jointype == JOIN_FULL);
 
-		/*
-		 * In range partitioning, if the given inner partition is already
-		 * merged (eg, because we found an overlapping range earlier), we know
-		 * where it fits in the join result; nothing to do in that case.  Else
-		 * create a new merged partition.
-		 */
 		if (inner_map->merged_indexes[inner_index] >= 0)
-		{
-			if (strategy == PARTITION_STRATEGY_LIST)
-				*merged_index = inner_map->merged_indexes[inner_index];
-			else
-			{
-				Assert(strategy == PARTITION_STRATEGY_RANGE);
-				*merged_index = -1;
-			}
-		}
+			*merged_index = inner_map->merged_indexes[inner_index];
 		else
 			*merged_index = merge_partition_with_dummy(inner_map, inner_index,
 													   next_index);
@@ -4628,7 +4568,6 @@ merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
 										 outer_bi->null_index,
 										 inner_bi->default_index,
 										 jointype,
-										 outer_bi->strategy,
 										 next_index,
 										 default_index,
 										 &merged_index))
@@ -4655,7 +4594,6 @@ merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
 										 inner_bi->null_index,
 										 outer_bi->default_index,
 										 jointype,
-										 outer_bi->strategy,
 										 next_index,
 										 default_index,
 										 &merged_index))
-- 
2.18.0

v27-0003-renage-get_merged_range_bounds-args.patchapplication/octet-stream; name=v27-0003-renage-get_merged_range_bounds-args.patchDownload
From 3674b909453e1f2cd85a4eff10b5279adaf79fe1 Mon Sep 17 00:00:00 2001
From: Amul Sul <amul.sul@enterprisedb.com>
Date: Thu, 14 Nov 2019 06:44:17 -0500
Subject: [PATCH 3/5] renage-get_merged_range_bounds-args

---
 src/backend/partitioning/partbounds.c | 47 ++++++++++++++-------------
 1 file changed, 24 insertions(+), 23 deletions(-)

diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 4aebc71ebb9..269b5c1f52f 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -183,12 +183,13 @@ static bool compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
 						 PartitionRangeBound *lower_bound2,
 						 PartitionRangeBound *upper_bound2,
 						 int *ub_cmpval, int *lb_cmpval);
-static void get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+static void
+get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
 						Oid *partcollations, JoinType jointype,
-						PartitionRangeBound *left_lb,
-						PartitionRangeBound *left_ub,
-						PartitionRangeBound *right_lb,
-						PartitionRangeBound *right_ub,
+						PartitionRangeBound *outer_lb,
+						PartitionRangeBound *outer_ub,
+						PartitionRangeBound *inner_lb,
+						PartitionRangeBound *inner_ub,
 						PartitionRangeBound *merged_lb,
 						PartitionRangeBound *merged_ub);
 static void add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
@@ -3298,10 +3299,10 @@ compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
 static void
 get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
 						Oid *partcollations, JoinType jointype,
-						PartitionRangeBound *left_lb,
-						PartitionRangeBound *left_ub,
-						PartitionRangeBound *right_lb,
-						PartitionRangeBound *right_ub,
+						PartitionRangeBound *outer_lb,
+						PartitionRangeBound *outer_ub,
+						PartitionRangeBound *inner_lb,
+						PartitionRangeBound *inner_ub,
 						PartitionRangeBound *merged_lb,
 						PartitionRangeBound *merged_ub)
 {
@@ -3316,37 +3317,37 @@ get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
 	{
 		case JOIN_LEFT:
 		case JOIN_ANTI:
-			*merged_ub = *left_ub;
-			*merged_lb = *left_lb;
+			*merged_ub = *outer_ub;
+			*merged_lb = *outer_lb;
 			break;
 
 		case JOIN_INNER:
 		case JOIN_SEMI:
 			if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
-									 left_ub, right_ub) < 0)
-				*merged_ub = *left_ub;
+									 outer_ub, inner_ub) < 0)
+				*merged_ub = *outer_ub;
 			else
-				*merged_ub = *right_ub;
+				*merged_ub = *inner_ub;
 
 			if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
-									 left_lb, right_lb) > 0)
-				*merged_lb = *left_lb;
+									 outer_lb, inner_lb) > 0)
+				*merged_lb = *outer_lb;
 			else
-				*merged_lb = *right_lb;
+				*merged_lb = *inner_lb;
 			break;
 
 		case JOIN_FULL:
 			if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
-									 left_ub, right_ub) > 0)
-				*merged_ub = *left_ub;
+									 outer_ub, inner_ub) > 0)
+				*merged_ub = *outer_ub;
 			else
-				*merged_ub = *right_ub;
+				*merged_ub = *inner_ub;
 
 			if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
-									 left_lb, right_lb) < 0)
-				*merged_lb = *left_lb;
+									 outer_lb, inner_lb) < 0)
+				*merged_lb = *outer_lb;
 			else
-				*merged_lb = *right_lb;
+				*merged_lb = *inner_lb;
 			break;
 
 		default:
-- 
2.18.0

v27-0004-pass-and-reuse-in-get_merged_range_bounds.patchapplication/octet-stream; name=v27-0004-pass-and-reuse-in-get_merged_range_bounds.patchDownload
From 6c7de3d0374397ce6a372cd3b0a9ec9e7c549230 Mon Sep 17 00:00:00 2001
From: Amul Sul <amul.sul@enterprisedb.com>
Date: Thu, 14 Nov 2019 23:12:08 -0500
Subject: [PATCH 4/5] pass and reuse in get_merged_range_bounds

---
 src/backend/partitioning/partbounds.c | 36 ++++++++++-----------------
 1 file changed, 13 insertions(+), 23 deletions(-)

diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 269b5c1f52f..9748dda8e2a 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -190,6 +190,7 @@ get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
 						PartitionRangeBound *outer_ub,
 						PartitionRangeBound *inner_lb,
 						PartitionRangeBound *inner_ub,
+						int	lb_cmpval, int ub_cmpval,
 						PartitionRangeBound *merged_lb,
 						PartitionRangeBound *merged_ub);
 static void add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
@@ -3303,9 +3304,15 @@ get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
 						PartitionRangeBound *outer_ub,
 						PartitionRangeBound *inner_lb,
 						PartitionRangeBound *inner_ub,
+						int	lb_cmpval, int ub_cmpval,
 						PartitionRangeBound *merged_lb,
 						PartitionRangeBound *merged_ub)
 {
+	Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								outer_lb, inner_lb) == lb_cmpval);
+	Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								outer_ub, inner_ub) == ub_cmpval);
+
 	/*
 	 * An outer join will have all the rows from the outer side, so merged
 	 * bounds will be same as the outer bounds. An inner join will have rows
@@ -3323,31 +3330,13 @@ get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
 
 		case JOIN_INNER:
 		case JOIN_SEMI:
-			if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
-									 outer_ub, inner_ub) < 0)
-				*merged_ub = *outer_ub;
-			else
-				*merged_ub = *inner_ub;
-
-			if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
-									 outer_lb, inner_lb) > 0)
-				*merged_lb = *outer_lb;
-			else
-				*merged_lb = *inner_lb;
+			*merged_lb = (lb_cmpval > 0)? *outer_lb : *inner_lb;
+			*merged_ub = (ub_cmpval < 0)? *outer_ub : *inner_ub;
 			break;
 
 		case JOIN_FULL:
-			if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
-									 outer_ub, inner_ub) > 0)
-				*merged_ub = *outer_ub;
-			else
-				*merged_ub = *inner_ub;
-
-			if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
-									 outer_lb, inner_lb) < 0)
-				*merged_lb = *outer_lb;
-			else
-				*merged_lb = *inner_lb;
+			*merged_lb = (lb_cmpval < 0)? *outer_lb : *inner_lb;
+			*merged_ub = (ub_cmpval > 0)? *outer_ub : *inner_ub;
 			break;
 
 		default:
@@ -3537,7 +3526,8 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 			 */
 			get_merged_range_bounds(partnatts, partsupfuncs, partcollations,
 									jointype, &outer_lb, &outer_ub, &inner_lb,
-									&inner_ub, &merged_lb, &merged_ub);
+									&inner_ub, lb_cmpval, ub_cmpval,
+									&merged_lb, &merged_ub);
 
 			/*
 			 * Both partitions are not merged yet, so they should be merged
-- 
2.18.0

v27-0005-cleanup-rearrage-arguments-minor-comment.patchapplication/octet-stream; name=v27-0005-cleanup-rearrage-arguments-minor-comment.patchDownload
From 81ff29d309cdee20cd4cf830b155754857ebbb89 Mon Sep 17 00:00:00 2001
From: Amul Sul <amul.sul@enterprisedb.com>
Date: Thu, 14 Nov 2019 23:59:21 -0500
Subject: [PATCH 5/5] cleanup - rearrage arguments & minor comment

---
 src/backend/partitioning/partbounds.c | 22 +++++++++++-----------
 1 file changed, 11 insertions(+), 11 deletions(-)

diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 9748dda8e2a..ba607e5c681 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -182,7 +182,7 @@ static bool compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
 						 PartitionRangeBound *upper_bound1,
 						 PartitionRangeBound *lower_bound2,
 						 PartitionRangeBound *upper_bound2,
-						 int *ub_cmpval, int *lb_cmpval);
+						 int *lb_cmpval, int *ub_cmpval);
 static void
 get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
 						Oid *partcollations, JoinType jointype,
@@ -3251,7 +3251,7 @@ compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
 						 PartitionRangeBound *upper_bound1,
 						 PartitionRangeBound *lower_bound2,
 						 PartitionRangeBound *upper_bound2,
-						 int *ub_cmpval, int *lb_cmpval)
+						 int *lb_cmpval, int *ub_cmpval)
 {
 	bool		overlap;
 
@@ -3265,25 +3265,25 @@ compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
 							 lower_bound1, upper_bound2) > 0)
 	{
 		overlap = false;
-		*ub_cmpval = 1;
 		*lb_cmpval = 1;
+		*ub_cmpval = 1;
 	}
 	else if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
 								  lower_bound2, upper_bound1) > 0)
 	{
 		overlap = false;
-		*ub_cmpval = -1;
 		*lb_cmpval = -1;
+		*ub_cmpval = -1;
 	}
 	else
 	{
 		overlap = true;
-		*ub_cmpval = compare_range_bounds(partnatts, partsupfuncs,
-										  partcollations, upper_bound1,
-										  upper_bound2);
 		*lb_cmpval = compare_range_bounds(partnatts, partsupfuncs,
 										  partcollations, lower_bound1,
 										  lower_bound2);
+		*ub_cmpval = compare_range_bounds(partnatts, partsupfuncs,
+										  partcollations, upper_bound1,
+										  upper_bound2);
 	}
 
 	return overlap;
@@ -3364,7 +3364,7 @@ add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
 	if (!*merged_datums)
 	{
 		Assert(!*merged_kinds && !*merged_indexes);
-		cmpval = 1;
+		cmpval = 1; /* First merged partition */
 	}
 	else
 	{
@@ -3496,21 +3496,21 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 		if (outer_part == -1)
 		{
 			overlap = false;
-			ub_cmpval = 1;
 			lb_cmpval = 1;
+			ub_cmpval = 1;
 		}
 		else if (inner_part == -1)
 		{
 			overlap = false;
-			ub_cmpval = -1;
 			lb_cmpval = -1;
+			ub_cmpval = -1;
 		}
 		else
 			overlap = compare_range_partitions(partnatts, partsupfuncs,
 											   partcollations,
 											   &outer_lb, &outer_ub,
 											   &inner_lb, &inner_ub,
-											   &ub_cmpval, &lb_cmpval);
+											   &lb_cmpval, &ub_cmpval);
 
 		if (overlap)
 		{
-- 
2.18.0

v27-0001-Improve-partition-matching-for-partitionwise-join.patchapplication/octet-stream; name=v27-0001-Improve-partition-matching-for-partitionwise-join.patchDownload
From 8a708840bb75e64f683f472941a6b74eaf12214c Mon Sep 17 00:00:00 2001
From: Etsuro Fujita <efujita@postgresql.org>
Date: Tue, 12 Nov 2019 21:56:44 +0900
Subject: [PATCH 1/5] Improve partition matching for partitionwise join

---
 src/backend/nodes/outfuncs.c                 |    2 +
 src/backend/optimizer/path/joinrels.c        |  246 +-
 src/backend/optimizer/util/inherit.c         |    2 +
 src/backend/optimizer/util/relnode.c         |   45 +-
 src/backend/partitioning/partbounds.c        | 1682 +++++++
 src/include/nodes/pathnodes.h                |    5 +
 src/include/partitioning/partbounds.h        |    7 +
 src/test/regress/expected/partition_join.out | 4259 +++++++++++++++---
 src/test/regress/sql/partition_join.sql      |  482 +-
 9 files changed, 6037 insertions(+), 693 deletions(-)

diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index b0dcd02ff68..b8151574024 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -2281,6 +2281,8 @@ _outRelOptInfo(StringInfo str, const RelOptInfo *node)
 	WRITE_BOOL_FIELD(has_eclass_joins);
 	WRITE_BOOL_FIELD(consider_partitionwise_join);
 	WRITE_BITMAPSET_FIELD(top_parent_relids);
+	WRITE_BOOL_FIELD(merged);
+	WRITE_BITMAPSET_FIELD(all_partrels);
 	WRITE_NODE_FIELD(partitioned_child_rels);
 }
 
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 6a480ab7644..ed7bc23c7be 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -48,6 +48,9 @@ static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 												Relids left_relids, Relids right_relids);
 static int	match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel,
 										 bool strict_op);
+static void get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2);
 
 
 /*
@@ -1357,25 +1360,30 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	bool		merged = false;
+	List	   *parts1 = NIL;
+	List	   *parts2 = NIL;
+	ListCell   *lcr1 = NULL;
+	ListCell   *lcr2 = NULL;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1390,34 +1398,107 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		   joinrel->part_scheme == rel2->part_scheme);
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * If we don't have the partition bounds for the join rel yet, try to
+	 * create it along with pairs of partitions to be joined; else generate
+	 * those using the partitioning info for the join rel we already have.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	if (joinrel->nparts == -1)
+	{
+		PartitionScheme part_scheme = joinrel->part_scheme;
+		PartitionBoundInfo boundinfo = NULL;
+		int			nparts = 0;
+
+		Assert(joinrel->boundinfo == NULL);
+		Assert(joinrel->part_rels == NULL);
+
+		/*
+		 * See if the partition bounds for inputs are exactly the same, in
+		 * which case we don't need to work hard: partitions with the same
+		 * partition indexes will form join pairs, and the join rel will have
+		 * the same partition bounds as inputs; otherwise try to merge the
+		 * partition bounds along with generating join pairs.
+		 *
+		 * Even if one or both inputs have merged partition bounds, it'd be
+		 * possible for the partition bounds to be exactly the same, but it
+		 * seems unlikely to be worth the cycles to check; do this check only
+		 * if both inputs have non-merged partition bounds.
+		 */
+		if (!rel1->merged &&
+			!rel2->merged &&
+			rel1->nparts == rel2->nparts &&
+			partition_bounds_equal(part_scheme->partnatts,
+								   part_scheme->parttyplen,
+								   part_scheme->parttypbyval,
+								   rel1->boundinfo, rel2->boundinfo))
+		{
+			boundinfo = rel1->boundinfo;
+			nparts = rel1->nparts;
+		}
+		else
+		{
+			boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											   part_scheme->parttyplen,
+											   part_scheme->parttypbyval,
+											   part_scheme->partsupfunc,
+											   part_scheme->partcollation,
+											   rel1, rel2,
+											   parent_sjinfo->jointype,
+											   &parts1, &parts2);
+			if (boundinfo == NULL)
+			{
+				joinrel->nparts = 0;
+				return;
+			}
+			nparts = list_length(parts1);
+			merged = true;
+		}
+
+		Assert(nparts > 0);
+		joinrel->boundinfo = boundinfo;
+		joinrel->merged = merged;
+		joinrel->nparts = nparts;
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+	}
+	else
+	{
+		Assert(joinrel->nparts > 0);
+		Assert(joinrel->boundinfo);
+		Assert(joinrel->part_rels);
+
+		/*
+		 * If the partition bounds for the join rel are not merged ones,
+		 * inputs are guaranteed to have the same partition bounds, so
+		 * partitions with the same partition indexes will form join pairs;
+		 * else let get_matching_part_pairs() do the work.
+		 */
+		if (joinrel->merged)
+		{
+			get_matching_part_pairs(root, joinrel, rel1, rel2,
+									&parts1, &parts2);
+			Assert(list_length(parts1) == joinrel->nparts);
+			Assert(list_length(parts2) == joinrel->nparts);
+			merged = true;
+		}
+	}
 
-	nparts = joinrel->nparts;
+	if (merged)
+	{
+		lcr1 = list_head(parts1);
+		lcr2 = list_head(parts2);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
-		bool		rel1_empty = (child_rel1 == NULL ||
-								  IS_DUMMY_REL(child_rel1));
-		bool		rel2_empty = (child_rel2 == NULL ||
-								  IS_DUMMY_REL(child_rel2));
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		bool		rel1_empty;
+		bool		rel2_empty;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1425,6 +1506,22 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		if (merged)
+		{
+			child_rel1 = lfirst_node(RelOptInfo, lcr1);
+			child_rel2 = lfirst_node(RelOptInfo, lcr2);
+			lcr1 = lnext(parts1, lcr1);
+			lcr2 = lnext(parts2, lcr2);
+		}
+		else
+		{
+			child_rel1 = rel1->part_rels[cnt_parts];
+			child_rel2 = rel2->part_rels[cnt_parts];
+		}
+
+		rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1));
+		rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2));
+
 		/*
 		 * Check for cases where we can prove that this segment of the join
 		 * returns no rows, due to one or both inputs being empty (including
@@ -1522,6 +1619,8 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 												 child_sjinfo,
 												 child_sjinfo->jointype);
 			joinrel->part_rels[cnt_parts] = child_joinrel;
+			joinrel->all_partrels = bms_add_members(joinrel->all_partrels,
+													child_joinrel->relids);
 		}
 
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
@@ -1529,6 +1628,7 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
+
 	}
 }
 
@@ -1738,3 +1838,99 @@ match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op)
 
 	return -1;
 }
+
+/*
+ * get_matching_part_pairs
+ *		Generate join pairs of partitions for the two inputs
+ */
+static void
+get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2)
+{
+	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
+	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
+	int 		cnt_parts;
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
+	{
+		RelOptInfo *child_joinrel = joinrel->part_rels[cnt_parts];
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		Relids		child_relids1;
+		Relids		child_relids2;
+
+		/*
+		 * If this segment of the join is empty, it means that this segment
+		 * was ignored when previously creating child-join paths for it in
+		 * try_partitionwise_join() as it would not contribute to the join
+		 * result, due to one or both inputs being empty; add NULL to each of
+		 * the given lists so that this segment will be ignored again in that
+		 * function.
+		 */
+		if (!child_joinrel)
+		{
+			*parts1 = lappend(*parts1, NULL);
+			*parts2 = lappend(*parts2, NULL);
+			continue;
+		}
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel1 side.
+		 */
+		child_relids1 = bms_intersect(child_joinrel->relids,
+									  rel1->all_partrels);
+		Assert(bms_num_members(child_relids1) == bms_num_members(rel1->relids));
+
+		/*
+		 * Get a child rel for rel1 with the relids.  Note that we should have
+		 * the child rel even if rel1 is a join rel, because in that case the
+		 * partitions specified in the relids would have matching/overlapping
+		 * boundaries, so those partitions should be considered as ones to be
+		 * joined even when planning partitionwise joins of rel1, meaning that
+		 * the child rel would have been built by the time we get here.
+		 */
+		if (rel1_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids1);
+
+			child_rel1 = find_base_rel(root, varno);
+		}
+		else
+			child_rel1 = find_join_rel(root, child_relids1);
+		Assert(child_rel1);
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel2 side.
+		 */
+		child_relids2 = bms_intersect(child_joinrel->relids,
+									  rel2->all_partrels);
+		Assert(bms_num_members(child_relids2) == bms_num_members(rel2->relids));
+
+		/*
+		 * Get a child rel for rel2 with the relids.  See above comments.
+		 */
+		if (rel2_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids2);
+
+			child_rel2 = find_base_rel(root, varno);
+		}
+		else
+			child_rel2 = find_join_rel(root, child_relids2);
+		Assert(child_rel2);
+
+		/*
+		 * The join of rel1 and rel2 is legal, so is the join of the child
+		 * rels obtained above; add them to the given lists as a join pair
+		 * producing this join segment.
+		 */
+		*parts1 = lappend(*parts1, child_rel1);
+		*parts2 = lappend(*parts2, child_rel2);
+	}
+}
diff --git a/src/backend/optimizer/util/inherit.c b/src/backend/optimizer/util/inherit.c
index 38bc61e6878..caf6039c106 100644
--- a/src/backend/optimizer/util/inherit.c
+++ b/src/backend/optimizer/util/inherit.c
@@ -376,6 +376,8 @@ expand_partitioned_rtentry(PlannerInfo *root, RelOptInfo *relinfo,
 		/* Create the otherrel RelOptInfo too. */
 		childrelinfo = build_simple_rel(root, childRTindex, relinfo);
 		relinfo->part_rels[i] = childrelinfo;
+		relinfo->all_partrels = bms_add_members(relinfo->all_partrels,
+												childrelinfo->relids);
 
 		/* If this child is itself partitioned, recurse */
 		if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 03e02423b2e..a1cf6329328 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -240,10 +240,12 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->has_eclass_joins = false;
 	rel->consider_partitionwise_join = false;	/* might get changed later */
 	rel->part_scheme = NULL;
-	rel->nparts = 0;
+	rel->nparts = -1;
 	rel->boundinfo = NULL;
+	rel->merged = false;
 	rel->partition_qual = NIL;
 	rel->part_rels = NULL;
+	rel->all_partrels = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->partitioned_child_rels = NIL;
@@ -653,10 +655,12 @@ build_join_rel(PlannerInfo *root,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -829,10 +833,12 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -1639,7 +1645,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1652,24 +1658,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1681,17 +1669,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index cfb44e23e29..f1458518716 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -25,6 +25,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/pathnodes.h"
 #include "parser/parse_coerce.h"
 #include "partitioning/partbounds.h"
 #include "partitioning/partdesc.h"
@@ -69,6 +70,17 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+typedef struct PartitionMap
+{
+	int			nparts;			/* number of partitions */
+	int		   *merged_indexes;	/* indexes of merged partitions */
+	bool	   *merged;			/* flags to indicate whether partitions are
+								 * merged with non-dummy partitions */
+	bool		did_remapping;	/* did we remap partitions? */
+	int		   *old_indexes;	/* old indexes of merged partitions if
+								 * did_remapping */
+} PartitionMap;
+
 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);
@@ -108,6 +120,85 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 									 Expr **keyCol,
 									 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static PartitionBoundInfo partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts);
+static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							JoinType jointype,
+							List **outer_parts, List **inner_parts);
+static void init_partition_map(RelOptInfo *rel, PartitionMap *map);
+static void free_partition_map(PartitionMap *map);
+static int map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_part, int inner_part, int *next_index);
+static int merge_partition_with_dummy(PartitionMap *map, int index,
+						   int *next_index);
+static bool process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index);
+static bool process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index);
+static void fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes);
+static void generate_matching_part_pairs(RelOptInfo *outer_rel,
+							 RelOptInfo *inner_rel,
+							 PartitionMap *outer_map,
+							 PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_part_list,
+							 List **inner_part_list);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int32 partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *collations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2);
+static bool partition_range_cmp(int partnatts, FmgrInfo *supfuncs,
+						   Oid *collations, PartitionRangeBound *lower_bound1,
+						   PartitionRangeBound *upper_bound1,
+						   PartitionRangeBound *lower_bound2,
+						   PartitionRangeBound *upper_bound2, int *ub_cmpval,
+						   int *lb_cmpval);
+static bool partition_range_merge_next_lb(int partnatts, FmgrInfo *supfuncs,
+							  Oid *collations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes);
+static bool merge_default_partitions(PartitionMap *outer_map,
+						 PartitionMap *inner_map,
+						 bool outer_has_default,
+						 int outer_default,
+						 bool inner_has_default,
+						 int inner_default,
+						 JoinType jointype,
+						 int *next_index,
+						 int *default_index);
+static bool merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_map, PartitionMap *inner_map,
+					  bool outer_has_default, bool inner_has_default,
+					  bool outer_has_null, bool inner_has_null,
+					  JoinType jointype, int *next_index,
+					  int *default_index, int *null_index);
 
 /*
  * get_qual_from_partbound
@@ -2995,3 +3086,1594 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * This function builds and returns the partition bounds for a join relation
+ * between input relations, creating two lists of partitions, which are
+ * returned to *outer_parts and *inner_parts respectively.  The lists contain
+ * the same number of partitions, and the partitions at the same positions in
+ * the lists indicate join pairs used for partitioned join.
+ *
+ * This function returns NULL, setting *outer_parts and *inner_parts to NIL,
+ * if a partition on one side matches multiple partitions on the other side,
+ * in which case we currently don't support partitioned join.
+ */
+PartitionBoundInfo
+partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
+{
+	PartitionBoundInfo 	merged_bounds;
+	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
+						inner_binfo = inner_rel->boundinfo;
+	char				strategy = outer_binfo->strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = NULL;
+
+			break;
+
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partsupfunc,
+														partcollation,
+														outer_rel,
+														inner_rel,
+														jointype,
+														outer_parts,
+														inner_parts);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(partnatts,
+														 partsupfunc,
+														 partcollation,
+														 outer_rel,
+														 inner_rel,
+														 jointype,
+														 outer_parts,
+														 inner_parts);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	return merged_bounds;
+}
+
+/*
+ * partition_get_range_bounds
+ *
+ * Given the index of lower bound in datums array, return lower and upper
+ * bounds and the index of the partition with that lower bound.
+ */
+static int
+partition_get_range_bounds(PartitionBoundInfo bi, int lb_index,
+						   PartitionRangeBound *lower,
+						   PartitionRangeBound *upper)
+{
+	int			part_index;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The lower bound should correspond to a valid partition. */
+	part_index = bi->indexes[lb_index + 1];
+	Assert(part_index >= 0);
+
+	lower->kind = bi->kind[lb_index];
+	lower->datums = bi->datums[lb_index];
+	lower->lower = true;
+	upper->kind = bi->kind[lb_index + 1];
+	upper->datums = bi->datums[lb_index + 1];
+	upper->lower = false;
+
+	return part_index;
+}
+
+/*
+ * partition_range_get_next_lb_index
+ *
+ * Given the index of lower bound in datums array return the
+ * index of lower bound of the next partition. When the given index corresponds
+ * to the last partition, return number of datums (ndatums).
+ */
+static int
+partition_range_get_next_lb_index(PartitionBoundInfo bi, int lb_index)
+{
+	/* A lower bound should have at least one more bound after it. */
+	Assert(lb_index < bi->ndatums - 1);
+
+	/* The partition index corresponding to the upper bound should be valid. */
+	Assert(bi->indexes[lb_index + 1] >= 0);
+
+	/*
+	 * If there are no bounds left beyond the upper bound, we have reached the
+	 * last partition.
+	 */
+	if (lb_index + 2 < bi->ndatums)
+	{
+		/*
+		 * If the bound next to the upper bound corresponds to no partition,
+		 * that's the next lower bound of the next partition. Otherwise, the
+		 * current upper bound is the lower bound of the next partition.
+		 */
+		if (bi->indexes[lb_index + 2] < 0)
+			return lb_index + 2;
+		else
+			return lb_index + 1;
+	}
+	else
+		return bi->ndatums;
+}
+
+static int32
+partition_range_bound_cmp(int partnatts, FmgrInfo *partsupfunc,
+						  Oid *partcollations, PartitionRangeBound *bound1,
+						  PartitionRangeBound *bound2)
+{
+	return partition_rbound_cmp(partnatts, partsupfunc, partcollations,
+								bound1->datums, bound1->kind, bound1->lower,
+								bound2);
+}
+
+/*
+ * partition_range_cmp
+ *
+ * Compare the bounds of two range partitions. Set ub_cmpval <, = or > 0, if the
+ * first partition's upper bound is lower than, equal to or higher than the
+ * second partition's upper bound resp. Similarly set lb_cmpval <, =  or > 0,
+ * if the first partition's lower bound is lower than, equal to or higher than
+ * the second partition's lower bound resp.
+ *
+ * Return true, if the ranges overlap, otherwise return false.
+ */
+static bool
+partition_range_cmp(int partnatts, FmgrInfo *partsupfuncs, Oid *partcollations,
+					PartitionRangeBound *lower_bound1,
+					PartitionRangeBound *upper_bound1,
+					PartitionRangeBound *lower_bound2,
+					PartitionRangeBound *upper_bound2, int *ub_cmpval,
+					int *lb_cmpval)
+{
+	bool		overlap;
+
+	/*
+	 * Compare upper bound of the first partition with the lower bound of the
+	 * second and vice-versa. If lower bound is higher than the upper bound,
+	 * the partitions are not overlapping. All other cases indicate overlapping
+	 * partitions.
+	 */
+	if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+								  lower_bound1, upper_bound2) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = 1;
+		*lb_cmpval = 1;
+	}
+	else if (partition_range_bound_cmp(partnatts, partsupfuncs, partcollations,
+									   lower_bound2, upper_bound1) > 0)
+	{
+		overlap = false;
+		*ub_cmpval = -1;
+		*lb_cmpval = -1;
+	}
+	else
+	{
+		overlap = true;
+		*ub_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, upper_bound1,
+											   upper_bound2);
+		*lb_cmpval = partition_range_bound_cmp(partnatts, partsupfuncs,
+											   partcollations, lower_bound1,
+											   lower_bound2);
+	}
+
+	return overlap;
+}
+
+/*
+ * partition_range_merge
+ *
+ * Merge the partition bounds of given two partitions such that the join
+ * between the given two partitions fits merged bounds.
+ *
+ * "merged_upper" will be set to one of the given upper bounds and
+ * "merged_lower" will be set to one of the given lower bounds.
+ */
+static void
+partition_range_merge(int partnatts, FmgrInfo *partsupfuncs,
+					  Oid *partcollations, JoinType jointype,
+					  PartitionRangeBound *left_lb,
+					  PartitionRangeBound *left_ub,
+					  PartitionRangeBound *right_lb,
+					  PartitionRangeBound *right_ub,
+					  PartitionRangeBound **merged_lb,
+					  PartitionRangeBound **merged_ub)
+{
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = left_ub;
+			*merged_lb = left_lb;
+			break;
+
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) < 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) > 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		case JOIN_FULL:
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_ub,
+										  right_ub) > 0)
+				*merged_ub = left_ub;
+			else
+				*merged_ub = right_ub;
+
+			if (partition_range_bound_cmp(partnatts, partsupfuncs,
+										  partcollations, left_lb,
+										  right_lb) < 0)
+				*merged_lb = left_lb;
+			else
+				*merged_lb = right_lb;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+}
+
+/*
+ * Add the lower bound of the next range to the list of bounds, if the lower
+ * bound is higher or equal to the previous upper bound. If successful return
+ * true, otherwise false.
+ */
+static bool
+partition_range_merge_next_lb(int partnatts, FmgrInfo *partsupfuncs,
+							  Oid *partcollations, Datum *next_lb_datums,
+							  PartitionRangeDatumKind *next_lb_kind,
+							  List **merged_datums, List **merged_kinds,
+							  List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		prev_ub.datums = llast(*merged_datums);
+		prev_ub.kind = llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  next_lb_datums, next_lb_kind, false,
+									  &prev_ub);
+	}
+
+	/*
+	 * The lower bound is lower than the last upper bound, thus does not fit
+	 * the bounds created so far and hence can not be merged with the existing
+	 * bounds.
+	 */
+	if (cmpval < 0)
+		return false;
+
+	/*
+	 * Add bounds of the new merged partition. If the next lower bound is
+	 * higher than the last upper bound, add new range with index
+	 * corresponding to the lower bound as -1. If the merged lower bound
+	 * is same as the last merged upper bound, the last upper bound will be
+	 * reused as the lower bound of the next range.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, next_lb_datums);
+		*merged_kinds = lappend(*merged_kinds, next_lb_kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	return true;
+}
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			next_index = 0;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_kinds = NIL;
+	List	   *merged_indexes = NIL;
+	int			outer_lb_index;
+	int			inner_lb_index;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = inner_lb_index = 0;
+	while (outer_lb_index < outer_bi->ndatums ||
+		   inner_lb_index < inner_bi->ndatums)
+	{
+		PartitionRangeBound *merged_lb = NULL;
+		PartitionRangeBound *merged_ub = NULL;
+		int			merged_index = -1;
+		PartitionRangeBound outer_lb;
+		PartitionRangeBound outer_ub;
+		PartitionRangeBound inner_lb;
+		PartitionRangeBound inner_ub;
+		int			outer_part = -1;
+		int			inner_part = -1;
+		bool		overlap;
+		int			ub_cmpval;
+		int			lb_cmpval;
+
+		/* Get the range bounds of the next pair of partitions. */
+		if (outer_lb_index < outer_bi->ndatums)
+			outer_part = partition_get_range_bounds(outer_bi, outer_lb_index,
+													&outer_lb, &outer_ub);
+		if (inner_lb_index < inner_bi->ndatums)
+			inner_part = partition_get_range_bounds(inner_bi, inner_lb_index,
+													&inner_lb, &inner_ub);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (outer_lb_index >= outer_bi->ndatums)
+		{
+			overlap = false;
+			ub_cmpval = 1;
+			lb_cmpval = 1;
+		}
+		else if (inner_lb_index >= inner_bi->ndatums)
+		{
+			overlap = false;
+			ub_cmpval = -1;
+			lb_cmpval = -1;
+		}
+		else
+			overlap = partition_range_cmp(partnatts, partsupfuncs, partcollations,
+										  &outer_lb, &outer_ub, &inner_lb,
+										  &inner_ub, &ub_cmpval, &lb_cmpval);
+
+		if (overlap)
+		{
+			/*
+			 * The rows from overlapping portion of ranges on both sides may
+			 * join, hence the corresponding pair of partitions form a joining
+			 * pair. Match them and produce the bounds of the joint partition
+			 * and its index by merging the bounds according to the type of
+			 * join.
+			 */
+			partition_range_merge(partnatts, partsupfuncs, partcollations,
+								  jointype, &outer_lb, &outer_ub, &inner_lb,
+								  &inner_ub, &merged_lb, &merged_ub);
+
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
+													outer_part, inner_part,
+													&next_index);
+
+			if (merged_index < 0)
+			{
+				/* Failed to match the partitions. */
+				return NULL;
+			}
+
+			/*
+			 * If the ranges overlap but don't exactly match, a row from
+			 * non-overlapping portion of the range from one side of join may
+			 * find its join partner in the previous or next overlapping
+			 * partition or default partition on the other side , if such a
+			 * partition exists. All those cases, if true, will cause one
+			 * partition from that side to match at least two partitions on the
+			 * other side; a case that we do not support now. Previous
+			 * partition has been delt with in the previous iteration of this
+			 * loop, next partition will be delt in the next iteration. We will
+			 * deal with the default partition here.
+			 */
+			if ((lb_cmpval < 0 && inner_has_default) ||
+				/* Non-overlapping range on the lower side of outer range. */
+				(lb_cmpval > 0 && outer_has_default) ||
+				/* Non-overlapping range on the lower side of inner range. */
+				(ub_cmpval < 0 && outer_has_default) ||
+				/* Non-overlapping range on the upper side of inner range. */
+				(ub_cmpval > 0 && inner_has_default))
+				/* Non-overlapping range on the upper side of outer range. */
+				return NULL;
+		}
+
+		if (ub_cmpval == 0)
+		{
+			/* Upper bounds of both the ranges match. */
+			Assert(overlap);
+
+			/* Move to the next pair of partitions. */
+			Assert(outer_lb_index < outer_bi->ndatums);
+			Assert(inner_lb_index < inner_bi->ndatums);
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				if (inner_has_default || IS_OUTER_JOIN(jointype))
+				{
+					if (!process_outer_partition(&outer_map,
+												 &inner_map,
+												 outer_has_default,
+												 inner_has_default,
+												 outer_part,
+												 inner_default,
+												 jointype,
+												 outer_bi->strategy,
+												 &next_index,
+												 &default_index,
+												 &merged_index))
+						return NULL;
+				}
+
+				merged_lb = &outer_lb;
+				merged_ub = &outer_ub;
+			}
+
+			/* Move to the next partition on the outer side. */
+			Assert(outer_lb_index < outer_bi->ndatums);
+			outer_lb_index = partition_range_get_next_lb_index(outer_bi,
+															   outer_lb_index);
+		}
+		else
+		{
+			Assert(ub_cmpval > 0);
+
+			/* Upper bound of outer range higher than that of the inner. */
+			if (overlap)
+			{
+				/* We have already dealt with overlapping ranges. */
+			}
+			else
+			{
+				if (outer_has_default || jointype == JOIN_FULL)
+				{
+					if (!process_inner_partition(&outer_map,
+												 &inner_map,
+												 outer_has_default,
+												 inner_has_default,
+												 inner_part,
+												 outer_default,
+												 jointype,
+												 outer_bi->strategy,
+												 &next_index,
+												 &default_index,
+												 &merged_index))
+						return NULL;
+				}
+
+				merged_lb = &inner_lb;
+				merged_ub = &inner_ub;
+			}
+
+			/* Move to the next partition on the inner side. */
+			Assert(inner_lb_index < inner_bi->ndatums);
+			inner_lb_index = partition_range_get_next_lb_index(inner_bi,
+															   inner_lb_index);
+		}
+
+		if (merged_index < 0)
+		{
+			/* We didn't find a new merged partition. */
+			continue;
+		}
+
+		/*
+		 * We have a valid partition index for the next partition of join. The
+		 * partition should have valid range.
+		 */
+		Assert(merged_lb && merged_ub);
+
+		/* Try merging new lower bound with the last upper bound. */
+		if (!partition_range_merge_next_lb(partnatts, partsupfuncs,
+										   partcollations,
+										   merged_lb->datums,
+										   merged_lb->kind, &merged_datums,
+										   &merged_kinds, &merged_indexes))
+			return NULL;
+
+		/* Add upper bound with the merged partition index. */
+		merged_datums = lappend(merged_datums, merged_ub->datums);
+		merged_kinds = lappend(merged_kinds, merged_ub->kind);
+		merged_indexes = lappend_int(merged_indexes, merged_index);
+	}
+
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+	{
+		if (!merge_default_partitions(&outer_map,
+									  &inner_map,
+									  outer_has_default,
+									  outer_default,
+									  inner_has_default,
+									  inner_default,
+									  jointype,
+									  &next_index,
+									  &default_index))
+			return NULL;
+	}
+	else
+		Assert(default_index == -1);
+
+	Assert(!outer_map.did_remapping && !inner_map.did_remapping);
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 &outer_map, &inner_map,
+								 next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes,
+													  merged_kinds,
+													  -1, default_index);
+	}
+
+	/* Free any memory we used in this function. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							JoinType jointype,
+							List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			next_index = 0;
+	int			null_index = -1;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			cnto;
+	int			cnti;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *merged_datum = NULL;
+		int			merged_index = -1;
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			cmpval;
+
+		/* Get the list datums of the next pair of partitions. */
+		odatums = cnto < outer_bi->ndatums ? outer_bi->datums[cnto] : NULL;
+		idatums = cnti < inner_bi->ndatums ? inner_bi->datums[cnti] : NULL;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+			cmpval = 1;
+		else if (cnti >= inner_bi->ndatums)
+			cmpval = -1;
+		else
+		{
+			Assert(odatums != NULL && idatums != NULL);
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0],
+													 idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			int			o_index = outer_bi->indexes[cnto];
+			int			i_index = inner_bi->indexes[cnti];
+
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			Assert(o_index >= 0 && i_index >= 0);
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
+													o_index, i_index,
+													&next_index);
+
+			if (merged_index < 0)
+				return NULL;
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			Assert(cnto < outer_bi->ndatums);
+
+			/* A datum missing from the inner side. */
+			merged_datum = odatums;
+
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				int			o_index = outer_bi->indexes[cnto];
+
+				Assert(o_index >= 0);
+				if (!process_outer_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 o_index,
+											 inner_default,
+											 jointype,
+											 outer_bi->strategy,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next datum on the outer side. */
+			cnto++;
+		}
+		else
+		{
+			Assert(cmpval > 0);
+			Assert(cnti < inner_bi->ndatums);
+
+			/* A datum missing from the outer side. */
+			merged_datum = idatums;
+
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				int			i_index = inner_bi->indexes[cnti];
+
+				Assert(i_index >= 0);
+				if (!process_inner_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 i_index,
+											 outer_default,
+											 jointype,
+											 outer_bi->strategy,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next datum on the inner side. */
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	/* Merge null partitions if any. */
+	if (outer_has_null || inner_has_null)
+	{
+		if (!merge_null_partitions(outer_bi, inner_bi,
+								   &outer_map, &inner_map,
+								   outer_has_default, inner_has_default,
+								   outer_has_null, inner_has_null,
+								   jointype, &next_index, &default_index,
+								   &null_index))
+			return NULL;
+	}
+	else
+		Assert(null_index == -1);
+
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+	{
+		if (!merge_default_partitions(&outer_map, &inner_map,
+									  outer_has_default, outer_default,
+									  inner_has_default, inner_default,
+									  jointype, &next_index, &default_index))
+			return NULL;
+	}
+	else
+		Assert(default_index == -1);
+
+	/* Fix the merged_indexes list if necessary. */
+	if (outer_map.did_remapping || inner_map.did_remapping)
+	{
+		Assert(jointype == JOIN_FULL);
+		fix_merged_indexes(&outer_map, &inner_map,
+						   next_index, merged_indexes);
+	}
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 &outer_map, &inner_map,
+								 next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) <= next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes, NIL,
+													  null_index, default_index);
+	}
+
+	/* Free up all extra memory before returning from this function. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+	list_free(merged_datums);
+	list_free(merged_indexes);
+
+	return merged_bounds;
+}
+
+/*
+ * init_partition_map
+ *
+ * Initialize a PartitionMap struct for given relation.
+ */
+static void
+init_partition_map(RelOptInfo *rel, PartitionMap *map)
+{
+	int			nparts = rel->nparts;
+	int			i;
+
+	map->nparts = nparts;
+	map->merged_indexes = (int *) palloc(sizeof(int) * nparts);
+	map->merged = (bool *) palloc(sizeof(bool) * nparts);
+	map->did_remapping = false;
+	map->old_indexes = (int *) palloc(sizeof(int) * nparts);
+	for (i = 0; i < nparts; i++)
+	{
+		map->merged_indexes[i] = map->old_indexes[i] = -1;
+		map->merged[i] = false;
+	}
+}
+
+/*
+ * free_partition_map
+ */
+static void
+free_partition_map(PartitionMap *map)
+{
+	pfree(map->merged_indexes);
+	pfree(map->merged);
+	pfree(map->old_indexes);
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given partitions.
+ */
+static int
+map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_index, int inner_index, int *next_index)
+{
+	int 		outer_merged_index;
+	bool 		outer_merged;
+	int 		inner_merged_index;
+	bool 		inner_merged;
+
+	Assert(outer_index >= 0 && outer_index < outer_map->nparts);
+	outer_merged_index = outer_map->merged_indexes[outer_index];
+	outer_merged = outer_map->merged[outer_index];
+	Assert(inner_index >= 0 && inner_index < inner_map->nparts);
+	inner_merged_index = inner_map->merged_indexes[inner_index];
+	inner_merged = inner_map->merged[inner_index];
+
+	/*
+	 * Handle cases where both partitions are mapped to merged partitions.
+	 */
+	if (outer_merged_index >= 0 && inner_merged_index >= 0)
+	{
+		/*
+		 * If the mereged partitions are the same, no need to do anything;
+		 * return the index of the merged partition.  Otherwise, if both
+		 * partitions are merged with dummy partitions, re-merge them; map
+		 * them to the merged partition with the smaller of the two merged
+		 * indexes and return the smaller index.  Otherwise they can't be
+		 * merged, so return -1.
+		 */
+		if (outer_merged_index == inner_merged_index)
+		{
+			Assert(outer_merged);
+			Assert(inner_merged);
+			return outer_merged_index;
+		}
+		if (!outer_merged && !inner_merged)
+		{
+			/*
+			 * Note that we will fix the larger index that have been added to
+			 * the merged_indexes list so far in fix_merged_indexes().
+			 */
+			if (outer_merged_index < inner_merged_index)
+			{
+				outer_map->merged[outer_index] = true;
+				inner_map->merged_indexes[inner_index] = outer_merged_index;
+				inner_map->merged[inner_index] = true;
+				inner_map->did_remapping = true;
+				inner_map->old_indexes[inner_index] = inner_merged_index;
+				return outer_merged_index;
+			}
+			else
+			{
+				inner_map->merged[inner_index] = true;
+				outer_map->merged_indexes[outer_index] = inner_merged_index;
+				outer_map->merged[outer_index] = true;
+				outer_map->did_remapping = true;
+				outer_map->old_indexes[outer_index] = outer_merged_index;
+				return inner_merged_index;
+			}
+		}
+		return -1;
+	}
+
+	/* At least one partition isn't mapped to a merged partition. */
+	Assert(outer_merged_index == -1 || inner_merged_index == -1);
+
+	/*
+	 * If neither of partitions isn't mapped, assign them a new merged
+	 * partition and return the index of the merged partition.  Otherwise, if
+	 * one of partitions is merged with a dummy relation (and the other isn't
+	 * merged), re-merge it with the other, with the same index, and return
+	 * the index.  Otherwise they can't be merged, so return -1.
+	 */
+	if (outer_merged_index == -1 && inner_merged_index == -1)
+	{
+		int 		merged_index = *next_index;
+
+		Assert(!outer_merged);
+		Assert(!inner_merged);
+		outer_map->merged_indexes[outer_index] = merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged_indexes[inner_index] = merged_index;
+		inner_map->merged[inner_index] = true;
+		*next_index = *next_index + 1;
+		return merged_index;
+	}
+	if (outer_merged_index >= 0 && !outer_map->merged[outer_index])
+	{
+		Assert(inner_merged_index == -1);
+		Assert(!inner_merged);
+		inner_map->merged_indexes[inner_index] = outer_merged_index;
+		inner_map->merged[inner_index] = true;
+		outer_map->merged[outer_index] = true;
+		return outer_merged_index;
+	}
+	if (inner_merged_index >= 0 && !inner_map->merged[inner_index])
+	{
+		Assert(outer_merged_index == -1);
+		Assert(!outer_merged);
+		outer_map->merged_indexes[outer_index] = inner_merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged[inner_index] = true;
+		return inner_merged_index;
+	}
+	return -1;
+}
+
+/*
+ * merge_partition_with_dummy
+ *
+ * *next_index is incremented.
+ */
+static int
+merge_partition_with_dummy(PartitionMap *map, int index, int *next_index)
+{
+	int 		merged_index = *next_index;
+
+	Assert(index >= 0 && index < map->nparts);
+	Assert(map->merged_indexes[index] == -1);
+	Assert(!map->merged[index]);
+	map->merged_indexes[index] = merged_index;
+	/* Leave the merged flag alone! */
+	*next_index = *next_index + 1;
+	return merged_index;
+}
+
+/*
+ * process_outer_partition
+ *
+ * Determine the merged partition associated with the given outer partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given outer partition.
+ */
+static bool
+process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index)
+{
+	Assert(outer_index >= 0);
+
+	/*
+	 * If the inner side has the default partition, the outer partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an outer join, in which case the outer partition has to be
+	 * scanned all the way anyway; if the outer partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the outer partition with a dummy partition.
+	 */
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0);
+
+		/*
+		 * If the outer side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (outer_has_default)
+			return false;
+
+		*merged_index = map_and_merge_partitions(outer_map, inner_map,
+												 outer_index, inner_default,
+												 next_index);
+		if (*merged_index == -1)
+			return false;
+
+		/*
+		 * If this is a FULL join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (*default_index == -1)
+				*default_index = *merged_index;
+			else
+				Assert(*merged_index == *default_index);
+			/* Don't add this index to the list of merged indexes. */
+			*merged_index = -1;
+		}
+	}
+	else
+	{
+		Assert(IS_OUTER_JOIN(jointype));
+		Assert(jointype != JOIN_RIGHT);
+
+		/*
+		 * In range partitioning, if the given outer partition is already
+		 * merged (eg, because we found an overlapping range earlier), we know
+		 * where it fits in the join result; nothing to do in that case.  Else
+		 * create a new merged partition.
+		 */
+		if (outer_map->merged_indexes[outer_index] >= 0)
+		{
+			if (strategy == PARTITION_STRATEGY_LIST)
+				*merged_index = outer_map->merged_indexes[outer_index];
+			else
+			{
+				Assert(strategy == PARTITION_STRATEGY_RANGE);
+				*merged_index = -1;
+			}
+		}
+		else
+			*merged_index = merge_partition_with_dummy(outer_map, outer_index,
+													   next_index);
+	}
+	return true;
+}
+
+/*
+ * process_inner_partition
+ *
+ * Determine the merged partition associated with the given inner partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given inner partition.
+ */
+static bool
+process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						char strategy,
+						int *next_index,
+						int *default_index,
+						int *merged_index)
+{
+	Assert(inner_index >= 0);
+
+	/*
+	 * If the outer side has the default partition, the inner partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an FULL join, in which case the inner partition has to be
+	 * scanned all the way anyway; if the inner partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the inner partition with a dummy partition.
+	 */
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0);
+
+		/*
+		 * If the inner side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (inner_has_default)
+			return false;
+
+		*merged_index = map_and_merge_partitions(outer_map, inner_map,
+												 outer_default, inner_index,
+												 next_index);
+		if (*merged_index == -1)
+			return false;
+
+		/*
+		 * If this is an outer join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			if (*default_index == -1)
+				*default_index = *merged_index;
+			else
+				Assert(*merged_index == *default_index);
+			/* Don't add this index to the list of merged indexes. */
+			*merged_index = -1;
+		}
+	}
+	else
+	{
+		Assert(jointype == JOIN_FULL);
+
+		/*
+		 * In range partitioning, if the given inner partition is already
+		 * merged (eg, because we found an overlapping range earlier), we know
+		 * where it fits in the join result; nothing to do in that case.  Else
+		 * create a new merged partition.
+		 */
+		if (inner_map->merged_indexes[inner_index] >= 0)
+		{
+			if (strategy == PARTITION_STRATEGY_LIST)
+				*merged_index = inner_map->merged_indexes[inner_index];
+			else
+			{
+				Assert(strategy == PARTITION_STRATEGY_RANGE);
+				*merged_index = -1;
+			}
+		}
+		else
+			*merged_index = merge_partition_with_dummy(inner_map, inner_index,
+													   next_index);
+	}
+	return true;
+}
+
+/*
+ * fix_merged_indexes
+ */
+static void
+fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes)
+{
+	int		   *new_indexes;
+	int			merged_index;
+	int			i;
+	ListCell   *lc;
+
+	new_indexes = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		new_indexes[i] = -1;
+
+	/* Build the mapping of old merged indexes to new merged indexes. */
+	if (outer_map->did_remapping)
+	{
+		for (i = 0; i < outer_map->nparts; i++)
+		{
+			merged_index = outer_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = outer_map->merged_indexes[i];
+		}
+	}
+	if (inner_map->did_remapping)
+	{
+		for (i = 0; i < inner_map->nparts; i++)
+		{
+			merged_index = inner_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = inner_map->merged_indexes[i];
+		}
+	}
+
+	/* Fix the merged_indexes list using the mapping. */
+	foreach(lc, merged_indexes)
+	{
+		merged_index = lfirst_int(lc);
+		Assert(merged_index >= 0);
+		if (new_indexes[merged_index] >= 0)
+			lfirst_int(lc) = new_indexes[merged_index];
+	}
+
+	pfree(new_indexes);
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * This function produces the list pairs of partitions which produce merged
+ * partitions in the order of merged partition indexes.
+ *
+ * nmerged is the number of merged partitions.
+ *
+ * If successful, the list pairs are returned as two separate lists,
+ * outer_part_list and inner_part_list, one for each side.  Otherwise, those
+ * lists will be set to NIL.
+ */
+static void
+generate_matching_part_pairs(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 PartitionMap *outer_map, PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_part_list, List **inner_part_list)
+{
+	int			outer_nparts = outer_map->nparts;
+	int			inner_nparts = inner_map->nparts;
+	int		   *outer_part_array;
+	int		   *inner_part_array;
+	int			max_nparts;
+	int 		i;
+
+	Assert(outer_nparts == outer_rel->nparts);
+	Assert(inner_nparts == inner_rel->nparts);
+
+	outer_part_array = (int *) palloc(sizeof(int) * nmerged);
+	inner_part_array = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		outer_part_array[i] = inner_part_array[i] = -1;
+
+	/* Set pairs of matching partitions. */
+	max_nparts = Max(outer_nparts, inner_nparts);
+	for (i = 0; i < max_nparts; i++)
+	{
+		if (i < outer_nparts)
+		{
+			int 		merged_index = outer_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				outer_part_array[merged_index] = i;
+			}
+		}
+		if (i < inner_nparts)
+		{
+			int 		merged_index = inner_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				inner_part_array[merged_index] = i;
+			}
+		}
+	}
+
+	*outer_part_list = NIL;
+	*inner_part_list = NIL;
+
+	/* Build the list pairs. */
+	for (i = 0; i < nmerged; i++)
+	{
+		int			outer_part = outer_part_array[i];
+		int			inner_part = inner_part_array[i];
+
+		/*
+		 * If both partitions are dummy, it means the merged partition was
+		 * removed by partition re-merging in map_and_merge_partitions();
+		 * ignore the merged partition
+		 */
+		if (outer_part == -1 && inner_part == -1)
+			continue;
+
+		*outer_part_list = lappend(*outer_part_list, outer_part >= 0 ?
+								   outer_rel->part_rels[outer_part] : NULL);
+		*inner_part_list = lappend(*inner_part_list, inner_part >= 0 ?
+								   inner_rel->part_rels[inner_part] : NULL);
+	}
+
+	pfree(outer_part_array);
+	pfree(inner_part_array);
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * Merge default partitions from both sides, if any, and assign the default
+ * partition for the join result, if necessary.
+ *
+ * If both the relations have default partitions, try mapping those to each
+ * other. If the mapping succeeds corresponding merged partition will act as
+ * the default partition of the join result.
+ *
+ * If inner side of the join has default but not the outer side, rows in it
+ * won't appear in the join result. So don't create a default partition. If
+ * outer side of the join has default but not the inner side, rows in it will
+ * appear in the join result, so create a default merged partition.
+ */
+static bool
+merge_default_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 bool outer_has_default, int outer_default,
+						 bool inner_has_default, int inner_default,
+						 JoinType jointype, int *next_index,
+						 int *default_index)
+{
+	Assert(outer_has_default || inner_has_default);
+
+	if (outer_has_default && !inner_has_default)
+	{
+		if (IS_OUTER_JOIN(jointype))
+		{
+			int			merged_index;
+
+			Assert(jointype != JOIN_RIGHT);
+			Assert(outer_default >= 0 && outer_default < outer_map->nparts);
+			merged_index = outer_map->merged_indexes[outer_default];
+			if (merged_index == -1)
+			{
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(outer_map,
+															outer_default,
+															next_index);
+			}
+			else
+				Assert(*default_index == merged_index);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else if (!outer_has_default && inner_has_default)
+	{
+		if (jointype == JOIN_FULL)
+		{
+			int			merged_index;
+
+			Assert(inner_default >= 0 && inner_default < inner_map->nparts);
+			merged_index = inner_map->merged_indexes[inner_default];
+			if (merged_index == -1)
+			{
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(inner_map,
+															inner_default,
+															next_index);
+			}
+			else
+				Assert(*default_index == merged_index);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		*default_index = map_and_merge_partitions(outer_map,
+												  inner_map,
+												  outer_default,
+												  inner_default,
+												  next_index);
+		if (*default_index == -1)
+			return false;
+	}
+
+	return true;
+}
+
+/*
+ * merge_null_partitions
+ *
+ * Merge NULL partitions, i.e. a partition that can hold NULL values for a list
+ * partitioned table, if any. Find the index of merged partition to which the
+ * NULL values would belong in the join result. If one joining relation has a
+ * NULL partition but not the other, try matching it with the default partition
+ * from the other relation since the default partition may have rows with NULL
+ * partition key. We can eliminate a NULL partition when it appears only on the
+ * inner side of the join and the outer side doesn't have a default partition.
+ *
+ * When the equality operator used for join is strict, two NULL values will not
+ * be considered as equal, and thus a NULL partition can be eliminated for an
+ * inner join. But we don't check the strictness operator here.
+ */
+static bool
+merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_map, PartitionMap *inner_map,
+					  bool outer_has_default, bool inner_has_default,
+					  bool outer_has_null, bool inner_has_null,
+					  JoinType jointype, int *next_index,
+					  int *default_index, int *null_index)
+{
+	Assert(outer_has_null || inner_has_null);
+	Assert(*null_index == -1);
+
+	if (outer_has_null && !inner_has_null)
+	{
+		int			merged_index = -1;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join to which the NULL partition matches will
+		 * contain the NULL values and thus become the NULL partition of the
+		 * the join.
+		 */
+		if (inner_has_default || IS_OUTER_JOIN(jointype))
+		{
+			if (!process_outer_partition(outer_map,
+										 inner_map,
+										 outer_has_default,
+										 inner_has_default,
+										 outer_bi->null_index,
+										 inner_bi->default_index,
+										 jointype,
+										 outer_bi->strategy,
+										 next_index,
+										 default_index,
+										 &merged_index))
+				return false;
+		}
+		*null_index = merged_index;
+	}
+	else if (!outer_has_null && inner_has_null)
+	{
+		int			merged_index = -1;
+
+		/*
+		 * If the NULL partition was missing from the outer side of the join,
+		 * the partition of the join to which the NULL partition matches will
+		 * contain the NULL values and thus become the NULL partition of the
+		 * the join.
+		 */
+		if (outer_has_default || jointype == JOIN_FULL)
+		{
+			if (!process_inner_partition(outer_map,
+										 inner_map,
+										 outer_has_default,
+										 inner_has_default,
+										 inner_bi->null_index,
+										 outer_bi->default_index,
+										 jointype,
+										 outer_bi->strategy,
+										 next_index,
+										 default_index,
+										 &merged_index))
+				return false;
+		}
+		*null_index = merged_index;
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		*null_index = map_and_merge_partitions(outer_map,
+											   inner_map,
+											   outer_bi->null_index,
+											   inner_bi->null_index,
+											   next_index);
+		if (*null_index == -1)
+			return false;
+	}
+
+	return true;
+}
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index 23a06d718e3..e06eb9aaae5 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -577,8 +577,10 @@ typedef struct PartitionSchemeData *PartitionScheme;
  *		part_scheme - Partitioning scheme of the relation
  *		nparts - Number of partitions
  *		boundinfo - Partition bounds
+ *		merged - true if partition bounds are merged ones
  *		partition_qual - Partition constraint if not the root
  *		part_rels - RelOptInfos for each partition
+ *		all_partrels - Relids set of all partition relids
  *		partexprs, nullable_partexprs - Partition key expressions
  *		partitioned_child_rels - RT indexes of unpruned partitions of
  *								 this relation that are partitioned tables
@@ -718,9 +720,12 @@ typedef struct RelOptInfo
 	PartitionScheme part_scheme;	/* Partitioning scheme. */
 	int			nparts;			/* number of partitions */
 	struct PartitionBoundInfoData *boundinfo;	/* Partition bounds */
+	bool		merged;			/* true if partition bounds were created by
+								 * partition_bounds_merge() */
 	List	   *partition_qual; /* partition constraint */
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of partitions,
 									 * stored in the same order of bounds */
+	Relids		all_partrels;	/* Relids set of all partition relids */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 	List	   *partitioned_child_rels; /* List of RT indexes. */
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index 0d0fd42b181..9292aa11e5e 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -16,6 +16,7 @@
 #include "nodes/pg_list.h"
 #include "partitioning/partdefs.h"
 #include "utils/relcache.h"
+struct RelOptInfo;				/* avoid including pathnodes.h here */
 
 
 /*
@@ -108,5 +109,11 @@ extern int	partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 										  int nvalues, Datum *values, bool *is_equal);
 extern int	partition_hash_bsearch(PartitionBoundInfo boundinfo,
 								   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   struct RelOptInfo *outer_rel, struct RelOptInfo *inner_rel,
+					   JoinType jointype,
+					   List **outer_parts, List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 975bf6765ca..1675abb1e75 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -8,59 +8,86 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-(4 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+(7 rows)
 
 -- left outer join, with whole-row reference; partitionwise join does not apply
 EXPLAIN (COSTS OFF)
@@ -72,35 +99,50 @@ SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER
    ->  Hash Right Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(22 rows)
 
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-      t1      |      t2      
---------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
- (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
- (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
- (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
- (550,0,0550) | 
-(12 rows)
+       t1       |       t2       
+----------------+----------------
+ (-250,0,-0250) | 
+ (-200,0,-0200) | 
+ (-150,0,-0150) | (0,-150,-0150)
+ (-100,0,-0100) | 
+ (-50,0,-0050)  | 
+ (0,0,0000)     | (0,0,0000)
+ (50,0,0050)    | 
+ (100,0,0100)   | 
+ (150,0,0150)   | (0,150,0150)
+ (200,0,0200)   | 
+ (250,0,0250)   | 
+ (300,0,0300)   | (0,300,0300)
+ (350,0,0350)   | 
+ (400,0,0400)   | 
+ (450,0,0450)   | (0,450,0450)
+ (500,0,0500)   | 
+ (550,0,0550)   | 
+ (600,0,0600)   | (0,600,0600)
+ (650,0,0650)   | 
+ (700,0,0700)   | 
+ (750,0,0750)   | (0,750,0750)
+(21 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
@@ -112,35 +154,53 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
    ->  Append
          ->  Hash Right Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Right Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t2_2.b)
-(20 rows)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+      |       | -225 | -0225
+      |       |  -75 | -0075
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(14 rows)
 
 -- full outer join, with placeholder vars
 EXPLAIN (COSTS OFF)
@@ -148,8 +208,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                             QUERY PLAN                            
 ------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               Filter: (((50) = prt1_p0.a) OR ((75) = prt2_p0.b))
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
@@ -174,7 +242,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               Filter: (((50) = prt1_p4.a) OR ((75) = prt2_p4.b))
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
@@ -211,35 +287,44 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Hash Right Join
-         Hash Cond: (prt2_p2.b = prt1_p1.a)
+         Hash Cond: (prt2_p2.b = prt1_p0.a)
          ->  Append
                ->  Seq Scan on prt2_p2
                      Filter: (b > 250)
                ->  Seq Scan on prt2_p3
                      Filter: (b > 250)
+               ->  Seq Scan on prt2_p4
+                     Filter: (b > 250)
          ->  Hash
                ->  Append
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p1
                            Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p2
                            Filter: ((a < 450) AND (b = 0))
-(15 rows)
+(19 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-(9 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+(14 rows)
 
 -- Currently we can't do partitioned join if nullable-side partitions are pruned
 EXPLAIN (COSTS OFF)
@@ -247,11 +332,13 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                       QUERY PLAN                      
 ------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Hash Full Join
-         Hash Cond: (prt1_p1.a = prt2_p2.b)
-         Filter: ((prt1_p1.b = 0) OR (prt2_p2.a = 0))
+         Hash Cond: (prt1_p0.a = prt2_p2.b)
+         Filter: ((prt1_p0.b = 0) OR (prt2_p2.a = 0))
          ->  Append
+               ->  Seq Scan on prt1_p0
+                     Filter: (a < 450)
                ->  Seq Scan on prt1_p1
                      Filter: (a < 450)
                ->  Seq Scan on prt1_p2
@@ -262,64 +349,147 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                            Filter: (b > 250)
                      ->  Seq Scan on prt2_p3
                            Filter: (b > 250)
-(16 rows)
+                     ->  Seq Scan on prt2_p4
+                           Filter: (b > 250)
+(20 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
-(12 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+      |       | 375 | 0375
+      |       | 450 | 0450
+      |       | 525 | 0525
+      |       | 600 | 0600
+      |       | 675 | 0675
+      |       | 750 | 0750
+(20 rows)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Semi Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Semi Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                            Filter: (a = 0)
-         ->  Nested Loop Semi Join
-               Join Filter: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
-               ->  Materialize
-                     ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
-(24 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt1_p3 t2_3
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+                           Filter: (b = 0)
+(37 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
 
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
@@ -330,27 +500,82 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
    ->  Append
          ->  Hash Anti Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Hash Anti Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Hash Anti Join
                Hash Cond: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
-(17 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(27 rows)
 
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
-  sum  |         avg          | sum  |         avg         
--------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+  sum  |         avg          | sum  |        avg         
+-------+----------------------+------+--------------------
+ 95550 | 273.0000000000000000 | 2200 | 6.2857142857142857
 (1 row)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Append
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p0 t1
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+               Index Cond: (a = t1.b)
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.b = t2_1.a)
+         ->  Seq Scan on prt2_p1 t1_1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p1 t2_1
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p2 t1_2
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+               Index Cond: (a = t1_2.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p3 t1_3
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+               Index Cond: (a = t1_3.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p4 t1_4
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+               Index Cond: (a = t1_4.b)
+(27 rows)
+
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+  b   |   c   
+------+-------
+ -225 | -0225
+  -75 | -0075
+   75 | 0075
+  225 | 0225
+  375 | 0375
+  525 | 0525
+  675 | 0675
+(7 rows)
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -362,49 +587,74 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
+                     ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
                            Index Cond: (a = t1.a)
-                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t3
                            Index Cond: (b = t2.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_1
+                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
                            Index Cond: (a = t1_1.a)
-                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3_1
                            Index Cond: (b = t2_1.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_2
+                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
                            Index Cond: (a = t1_2.a)
-                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_2
                            Index Cond: (b = t2_2.a)
-(27 rows)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                           Index Cond: (a = t1_3.a)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_3
+                           Index Cond: (b = t2_3.a)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                           Index Cond: (a = t1_4.a)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t3_4
+                           Index Cond: (b = t2_4.a)
+(43 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   | t2a | t3a | least 
------+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
- 100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
- 250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
- 400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
- 550 | 0 | 0550 |     |     |      
-(12 rows)
+  a   | b |   c   | t2a  | t3a | least 
+------+---+-------+------+-----+-------
+ -250 | 0 | -0250 |      |     |      
+ -200 | 0 | -0200 |      |     |      
+ -150 | 0 | -0150 | -150 |   0 |  -150
+ -100 | 0 | -0100 |      |     |      
+  -50 | 0 | -0050 |      |     |      
+    0 | 0 | 0000  |    0 |   0 |     0
+   50 | 0 | 0050  |      |     |      
+  100 | 0 | 0100  |      |     |      
+  150 | 0 | 0150  |  150 |   0 |   150
+  200 | 0 | 0200  |      |     |      
+  250 | 0 | 0250  |      |     |      
+  300 | 0 | 0300  |  300 |   0 |   300
+  350 | 0 | 0350  |      |     |      
+  400 | 0 | 0400  |      |     |      
+  450 | 0 | 0450  |  450 |   0 |   450
+  500 | 0 | 0500  |      |     |      
+  550 | 0 | 0550  |      |     |      
+  600 | 0 | 0600  |  600 |   0 |   600
+  650 | 0 | 0650  |      |     |      
+  700 | 0 | 0700  |      |     |      
+  750 | 0 | 0750  |  750 |   0 |   750
+(21 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
@@ -418,46 +668,67 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
          Hash Cond: ((t1.c)::text = (t2.c)::text)
          Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Seq Scan on prt1_p1 t1
-               ->  Seq Scan on prt1_p2 t1_1
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
          ->  Hash
                ->  Append
                      ->  Hash Join
                            Hash Cond: (t2.a = t3.b)
-                           ->  Seq Scan on prt1_p1 t2
+                           ->  Seq Scan on prt1_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t3
+                                 ->  Seq Scan on prt2_p0 t3
                      ->  Hash Join
                            Hash Cond: (t2_1.a = t3_1.b)
-                           ->  Seq Scan on prt1_p2 t2_1
+                           ->  Seq Scan on prt1_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t3_1
+                                 ->  Seq Scan on prt2_p1 t3_1
                      ->  Hash Join
                            Hash Cond: (t2_2.a = t3_2.b)
-                           ->  Seq Scan on prt1_p3 t2_2
+                           ->  Seq Scan on prt1_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p3 t3_2
-(26 rows)
+                                 ->  Seq Scan on prt2_p2 t3_2
+                     ->  Hash Join
+                           Hash Cond: (t2_3.a = t3_3.b)
+                           ->  Seq Scan on prt1_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t3_3
+                     ->  Hash Join
+                           Hash Cond: (t2_4.a = t3_4.b)
+                           ->  Seq Scan on prt1_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t3_4
+(38 rows)
 
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, t2.b t2b, t2.c t2c, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
-  a  | t2a | t2c  
------+-----+------
-   0 |   0 | 0000
-  50 |     | 
- 100 |     | 
- 150 | 150 | 0150
- 200 |     | 
- 250 |     | 
- 300 | 300 | 0300
- 350 |     | 
- 400 |     | 
- 450 | 450 | 0450
- 500 |     | 
- 550 |     | 
-(12 rows)
+  a   | t2a  |  t2c  
+------+------+-------
+ -250 |      | 
+ -200 |      | 
+ -150 | -150 | -0150
+ -100 |      | 
+  -50 |      | 
+    0 |    0 | 0000
+   50 |      | 
+  100 |      | 
+  150 |  150 | 0150
+  200 |      | 
+  250 |      | 
+  300 |  300 | 0300
+  350 |      | 
+  400 |      | 
+  450 |  450 | 0450
+  500 |      | 
+  550 |      | 
+  600 |  600 | 0600
+  650 |      | 
+  700 |      | 
+  750 |  750 | 0750
+(21 rows)
 
 -- bug with inadequate sort key representation
 SET enable_partitionwise_aggregate TO true;
@@ -469,49 +740,75 @@ SELECT a, b FROM prt1 FULL JOIN prt2 p2(b,a,c) USING(a,b)
                                                     QUERY PLAN                                                     
 -------------------------------------------------------------------------------------------------------------------
  Group
-   Group Key: (COALESCE(prt1_p1.a, p2.a)), (COALESCE(prt1_p1.b, p2.b))
+   Group Key: (COALESCE(prt1_p0.a, p2.a)), (COALESCE(prt1_p0.b, p2.b))
    ->  Merge Append
-         Sort Key: (COALESCE(prt1_p1.a, p2.a)), (COALESCE(prt1_p1.b, p2.b))
+         Sort Key: (COALESCE(prt1_p0.a, p2.a)), (COALESCE(prt1_p0.b, p2.b))
          ->  Group
-               Group Key: (COALESCE(prt1_p1.a, p2.a)), (COALESCE(prt1_p1.b, p2.b))
+               Group Key: (COALESCE(prt1_p0.a, p2.a)), (COALESCE(prt1_p0.b, p2.b))
                ->  Sort
-                     Sort Key: (COALESCE(prt1_p1.a, p2.a)), (COALESCE(prt1_p1.b, p2.b))
+                     Sort Key: (COALESCE(prt1_p0.a, p2.a)), (COALESCE(prt1_p0.b, p2.b))
                      ->  Merge Full Join
-                           Merge Cond: ((prt1_p1.a = p2.a) AND (prt1_p1.b = p2.b))
-                           Filter: ((COALESCE(prt1_p1.a, p2.a) >= 490) AND (COALESCE(prt1_p1.a, p2.a) <= 510))
+                           Merge Cond: ((prt1_p0.a = p2.a) AND (prt1_p0.b = p2.b))
+                           Filter: ((COALESCE(prt1_p0.a, p2.a) >= 490) AND (COALESCE(prt1_p0.a, p2.a) <= 510))
+                           ->  Sort
+                                 Sort Key: prt1_p0.a, prt1_p0.b
+                                 ->  Seq Scan on prt1_p0
+                           ->  Sort
+                                 Sort Key: p2.a, p2.b
+                                 ->  Seq Scan on prt2_p0 p2
+         ->  Group
+               Group Key: (COALESCE(prt1_p1.a, p2_1.a)), (COALESCE(prt1_p1.b, p2_1.b))
+               ->  Sort
+                     Sort Key: (COALESCE(prt1_p1.a, p2_1.a)), (COALESCE(prt1_p1.b, p2_1.b))
+                     ->  Merge Full Join
+                           Merge Cond: ((prt1_p1.a = p2_1.a) AND (prt1_p1.b = p2_1.b))
+                           Filter: ((COALESCE(prt1_p1.a, p2_1.a) >= 490) AND (COALESCE(prt1_p1.a, p2_1.a) <= 510))
                            ->  Sort
                                  Sort Key: prt1_p1.a, prt1_p1.b
                                  ->  Seq Scan on prt1_p1
                            ->  Sort
-                                 Sort Key: p2.a, p2.b
-                                 ->  Seq Scan on prt2_p1 p2
+                                 Sort Key: p2_1.a, p2_1.b
+                                 ->  Seq Scan on prt2_p1 p2_1
          ->  Group
-               Group Key: (COALESCE(prt1_p2.a, p2_1.a)), (COALESCE(prt1_p2.b, p2_1.b))
+               Group Key: (COALESCE(prt1_p2.a, p2_2.a)), (COALESCE(prt1_p2.b, p2_2.b))
                ->  Sort
-                     Sort Key: (COALESCE(prt1_p2.a, p2_1.a)), (COALESCE(prt1_p2.b, p2_1.b))
+                     Sort Key: (COALESCE(prt1_p2.a, p2_2.a)), (COALESCE(prt1_p2.b, p2_2.b))
                      ->  Merge Full Join
-                           Merge Cond: ((prt1_p2.a = p2_1.a) AND (prt1_p2.b = p2_1.b))
-                           Filter: ((COALESCE(prt1_p2.a, p2_1.a) >= 490) AND (COALESCE(prt1_p2.a, p2_1.a) <= 510))
+                           Merge Cond: ((prt1_p2.a = p2_2.a) AND (prt1_p2.b = p2_2.b))
+                           Filter: ((COALESCE(prt1_p2.a, p2_2.a) >= 490) AND (COALESCE(prt1_p2.a, p2_2.a) <= 510))
                            ->  Sort
                                  Sort Key: prt1_p2.a, prt1_p2.b
                                  ->  Seq Scan on prt1_p2
                            ->  Sort
-                                 Sort Key: p2_1.a, p2_1.b
-                                 ->  Seq Scan on prt2_p2 p2_1
+                                 Sort Key: p2_2.a, p2_2.b
+                                 ->  Seq Scan on prt2_p2 p2_2
          ->  Group
-               Group Key: (COALESCE(prt1_p3.a, p2_2.a)), (COALESCE(prt1_p3.b, p2_2.b))
+               Group Key: (COALESCE(prt1_p3.a, p2_3.a)), (COALESCE(prt1_p3.b, p2_3.b))
                ->  Sort
-                     Sort Key: (COALESCE(prt1_p3.a, p2_2.a)), (COALESCE(prt1_p3.b, p2_2.b))
+                     Sort Key: (COALESCE(prt1_p3.a, p2_3.a)), (COALESCE(prt1_p3.b, p2_3.b))
                      ->  Merge Full Join
-                           Merge Cond: ((prt1_p3.a = p2_2.a) AND (prt1_p3.b = p2_2.b))
-                           Filter: ((COALESCE(prt1_p3.a, p2_2.a) >= 490) AND (COALESCE(prt1_p3.a, p2_2.a) <= 510))
+                           Merge Cond: ((prt1_p3.a = p2_3.a) AND (prt1_p3.b = p2_3.b))
+                           Filter: ((COALESCE(prt1_p3.a, p2_3.a) >= 490) AND (COALESCE(prt1_p3.a, p2_3.a) <= 510))
                            ->  Sort
                                  Sort Key: prt1_p3.a, prt1_p3.b
                                  ->  Seq Scan on prt1_p3
                            ->  Sort
-                                 Sort Key: p2_2.a, p2_2.b
-                                 ->  Seq Scan on prt2_p3 p2_2
-(43 rows)
+                                 Sort Key: p2_3.a, p2_3.b
+                                 ->  Seq Scan on prt2_p3 p2_3
+         ->  Group
+               Group Key: (COALESCE(prt1_p4.a, p2_4.a)), (COALESCE(prt1_p4.b, p2_4.b))
+               ->  Sort
+                     Sort Key: (COALESCE(prt1_p4.a, p2_4.a)), (COALESCE(prt1_p4.b, p2_4.b))
+                     ->  Merge Full Join
+                           Merge Cond: ((prt1_p4.a = p2_4.a) AND (prt1_p4.b = p2_4.b))
+                           Filter: ((COALESCE(prt1_p4.a, p2_4.a) >= 490) AND (COALESCE(prt1_p4.a, p2_4.a) <= 510))
+                           ->  Sort
+                                 Sort Key: prt1_p4.a, prt1_p4.b
+                                 ->  Seq Scan on prt1_p4
+                           ->  Sort
+                                 Sort Key: p2_4.a, p2_4.b
+                                 ->  Seq Scan on prt2_p4 p2_4
+(69 rows)
 
 SELECT a, b FROM prt1 FULL JOIN prt2 p2(b,a,c) USING(a,b)
   WHERE a BETWEEN 490 AND 510
@@ -540,19 +837,29 @@ RESET enable_hashjoin;
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
@@ -563,32 +870,49 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
    ->  Append
          ->  Hash Join
                Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
-               ->  Seq Scan on prt2_e_p1 t2
+               ->  Seq Scan on prt2_e_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 t1
+                     ->  Seq Scan on prt1_e_p0 t1
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
-               ->  Seq Scan on prt2_e_p2 t2_1
+               Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+               ->  Seq Scan on prt1_e_p1 t1_1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_e_p1 t2_1
+         ->  Hash Join
+               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
+               ->  Seq Scan on prt2_e_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 t1_1
+                     ->  Seq Scan on prt1_e_p2 t1_2
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
-               ->  Seq Scan on prt2_e_p3 t2_2
+               Hash Cond: (((t2_3.b + t2_3.a) / 2) = ((t1_3.a + t1_3.b) / 2))
+               ->  Seq Scan on prt2_e_p3 t2_3
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 t1_2
+                     ->  Seq Scan on prt1_e_p3 t1_3
                            Filter: (c = 0)
-(21 rows)
+         ->  Hash Join
+               Hash Cond: (((t2_4.b + t2_4.a) / 2) = ((t1_4.a + t1_4.b) / 2))
+               ->  Seq Scan on prt2_e_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4 t1_4
+                           Filter: (c = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
- 150 | 0 | 150 | 0
- 300 | 0 | 300 | 0
- 450 | 0 | 450 | 0
-(4 rows)
+  a   | c |  b   | c 
+------+---+------+---
+ -250 | 0 | -250 | 0
+ -100 | 0 | -100 | 0
+    0 | 0 |    0 | 0
+    0 | 0 |    0 | 0
+  150 | 0 |  150 | 0
+  300 | 0 |  300 | 0
+  450 | 0 |  450 | 0
+  600 | 0 |  600 | 0
+  750 | 0 |  750 | 0
+(9 rows)
 
 --
 -- N-way join
@@ -601,154 +925,232 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = ((t3.a + t3.b) / 2))
+               Join Filter: (t1.a = t2.b)
                ->  Hash Join
-                     Hash Cond: (t2.b = t1.a)
-                     ->  Seq Scan on prt2_p1 t2
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on prt1_e_p0 t3
                      ->  Hash
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                     Index Cond: (((a + b) / 2) = t2.b)
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
+                     Index Cond: (b = ((t3.a + t3.b) / 2))
          ->  Nested Loop
                Join Filter: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_1.b = t1_1.a)
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
+               ->  Index Scan using iprt1_e_p4_ab2 on prt1_e_p1 t3_1
                      Index Cond: (((a + b) / 2) = t2_1.b)
          ->  Nested Loop
                Join Filter: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_2.b = t1_2.a)
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
+               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_2
                      Index Cond: (((a + b) / 2) = t2_2.b)
-(33 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = ((t3_3.a + t3_3.b) / 2))
+               ->  Nested Loop
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t2_3.b)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t2_4.b)
+               ->  Hash Join
+                     Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                     ->  Seq Scan on prt1_e_p4 t3_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_p4 t1_4
+                                 Filter: (b = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = ((t3_4.a + t3_4.b) / 2))
+(52 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
-(4 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(8 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                          QUERY PLAN                          
---------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Hash Right Join
                Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
-               ->  Seq Scan on prt1_e_p1 t3
+               ->  Seq Scan on prt1_e_p0 t3
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2.b = t1.a)
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_1.a)
-               ->  Seq Scan on prt1_e_p2 t3_1
+               ->  Seq Scan on prt1_e_p1 t3_1
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_1.b = t1_1.a)
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_2.a)
-               ->  Seq Scan on prt1_e_p3 t3_2
+               ->  Seq Scan on prt1_e_p2 t3_2
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_2.b = t1_2.a)
-                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                                        Filter: (b = 0)
-(33 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+               ->  Seq Scan on prt1_e_p4 t3_4
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_4.b = t1_4.a)
+                           ->  Seq Scan on prt2_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_p4 t1_4
+                                       Filter: (b = 0)
+(51 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                            QUERY PLAN                             
--------------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p1 t3
+                           ->  Seq Scan on prt1_e_p0 t3
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p1_b on prt2_p1 t2
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
                      Index Cond: (b = t1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p2 t3_1
+                           ->  Seq Scan on prt1_e_p1 t3_1
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
+               ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
                      Index Cond: (b = t1_1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p3 t3_2
+                           ->  Seq Scan on prt1_e_p2 t3_2
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
                      Index Cond: (b = t1_2.a)
-(30 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_e_p3 t3_3
+                           Filter: (c = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                           Index Cond: (a = ((t3_3.a + t3_3.b) / 2))
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Nested Loop Left Join
+               ->  Hash Right Join
+                     Hash Cond: (t1_4.a = ((t3_4.a + t3_4.b) / 2))
+                     ->  Seq Scan on prt1_p4 t1_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p4 t3_4
+                                 Filter: (c = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = t1_4.a)
+(47 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -757,21 +1159,34 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                                                    QUERY PLAN                                                   
 ----------------------------------------------------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   Sort Key: prt1_p0.a, prt2_p0.b, ((prt1_e_p0.a + prt1_e_p0.b))
    ->  Append
          ->  Hash Full Join
-               Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
-               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               Hash Cond: (prt1_p0.a = ((prt1_e_p0.a + prt1_e_p0.b) / 2))
+               Filter: ((prt1_p0.a = (50)) OR (prt2_p0.b = (75)) OR (((prt1_e_p0.a + prt1_e_p0.b) / 2) = (50)))
                ->  Hash Full Join
-                     Hash Cond: (prt1_p1.a = prt2_p1.b)
-                     ->  Seq Scan on prt1_p1
+                     Hash Cond: (prt1_p0.a = prt2_p0.b)
+                     ->  Seq Scan on prt1_p0
                            Filter: (b = 0)
                      ->  Hash
-                           ->  Seq Scan on prt2_p1
+                           ->  Seq Scan on prt2_p0
                                  Filter: (a = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1
+                     ->  Seq Scan on prt1_e_p0
                            Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = prt1_p1.a)
+               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               ->  Seq Scan on prt1_e_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on prt1_p1
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1
+                                       Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
                Filter: ((prt1_p2.a = (50)) OR (prt2_p2.b = (75)) OR (((prt1_e_p2.a + prt1_e_p2.b) / 2) = (50)))
@@ -798,7 +1213,20 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                ->  Hash
                      ->  Seq Scan on prt1_e_p3
                            Filter: (c = 0)
-(42 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = ((prt1_e_p4.a + prt1_e_p4.b) / 2))
+               Filter: ((prt1_p4.a = (50)) OR (prt2_p4.b = (75)) OR (((prt1_e_p4.a + prt1_e_p4.b) / 2) = (50)))
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = prt2_p4.b)
+                     ->  Seq Scan on prt1_p4
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4
+                                 Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4
+                           Filter: (c = 0)
+(68 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
@@ -816,172 +1244,260 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = t1_3.b)
+               Join Filter: (t1.a = t1_5.b)
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Join
-                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
-                           ->  Seq Scan on prt1_e_p1 t2
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t1_3
+                                 ->  Seq Scan on prt2_p0 t1_5
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
                      Index Cond: (a = ((t2.a + t2.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_1.a = t1_4.b)
+               Join Filter: (t1_1.a = t1_6.b)
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Join
-                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
-                           ->  Seq Scan on prt1_e_p2 t2_1
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_6.b)
+                           ->  Seq Scan on prt1_e_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t1_4
+                                 ->  Seq Scan on prt2_p1 t1_6
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
                      Index Cond: (a = ((t2_1.a + t2_1.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_2.a = t1_5.b)
+               Join Filter: (t1_2.a = t1_7.b)
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
+                           ->  Seq Scan on prt2_p2 t1_7
                                  Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
+                           ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t2_2
+                                 Index Cond: (((a + b) / 2) = t1_7.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t1_8.b)
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Nested Loop
+                           ->  Seq Scan on prt2_p3 t1_8
+                                 Filter: (a = 0)
+                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_3
+                                 Index Cond: (((a + b) / 2) = t1_8.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = ((t2_3.a + t2_3.b) / 2))
+                     Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t1_9.b)
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Join
+                           Hash Cond: (((t2_4.a + t2_4.b) / 2) = t1_9.b)
+                           ->  Seq Scan on prt1_e_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t1_9
+                                       Filter: (a = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = ((t2_4.a + t2_4.b) / 2))
+                     Filter: (b = 0)
+(66 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                               QUERY PLAN                                
--------------------------------------------------------------------------
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
-                           ->  Seq Scan on prt2_p1 t1_3
+                           Hash Cond: (t1_5.b = ((t1_10.a + t1_10.b) / 2))
+                           ->  Seq Scan on prt2_p0 t1_5
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t1_6
+                                 ->  Seq Scan on prt1_e_p0 t1_10
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
-                     Index Cond: (a = t1_3.b)
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t1_5.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
-                           ->  Seq Scan on prt2_p2 t1_4
+                           Hash Cond: (t1_6.b = ((t1_11.a + t1_11.b) / 2))
+                           ->  Seq Scan on prt2_p1 t1_6
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t1_7
+                                 ->  Seq Scan on prt1_e_p1 t1_11
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
-                     Index Cond: (a = t1_4.b)
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
+                     Index Cond: (a = t1_6.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
-                           ->  Seq Scan on prt2_p3 t1_5
+                           Hash Cond: (t1_7.b = ((t1_12.a + t1_12.b) / 2))
+                           ->  Seq Scan on prt2_p2 t1_7
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p3 t1_8
+                                 ->  Seq Scan on prt1_e_p2 t1_12
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t1_5.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t1_7.b)
                      Filter: (b = 0)
-(39 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_8.b = ((t1_13.a + t1_13.b) / 2))
+                           ->  Seq Scan on prt2_p3 t1_8
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p3 t1_13
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t1_8.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_9.b = ((t1_14.a + t1_14.b) / 2))
+                           ->  Seq Scan on prt2_p4 t1_9
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t1_14
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = t1_9.b)
+                     Filter: (b = 0)
+(63 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 -- test merge joins
 SET enable_hashjoin TO off;
 SET enable_nestloop TO off;
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                           QUERY PLAN                           
-----------------------------------------------------------------
+                            QUERY PLAN                            
+------------------------------------------------------------------
  Merge Append
    Sort Key: t1.a
    ->  Merge Semi Join
-         Merge Cond: (t1.a = t1_3.b)
+         Merge Cond: (t1.a = t1_5.b)
          ->  Sort
                Sort Key: t1.a
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+               Merge Cond: (t1_5.b = (((t1_10.a + t1_10.b) / 2)))
                ->  Sort
-                     Sort Key: t1_3.b
-                     ->  Seq Scan on prt2_p1 t1_3
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on prt2_p0 t1_5
                ->  Sort
-                     Sort Key: (((t1_6.a + t1_6.b) / 2))
-                     ->  Seq Scan on prt1_e_p1 t1_6
+                     Sort Key: (((t1_10.a + t1_10.b) / 2))
+                     ->  Seq Scan on prt1_e_p0 t1_10
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_1.a = t1_4.b)
+         Merge Cond: (t1_1.a = t1_6.b)
          ->  Sort
                Sort Key: t1_1.a
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+               Merge Cond: (t1_6.b = (((t1_11.a + t1_11.b) / 2)))
                ->  Sort
-                     Sort Key: t1_4.b
-                     ->  Seq Scan on prt2_p2 t1_4
+                     Sort Key: t1_6.b
+                     ->  Seq Scan on prt2_p1 t1_6
                ->  Sort
-                     Sort Key: (((t1_7.a + t1_7.b) / 2))
-                     ->  Seq Scan on prt1_e_p2 t1_7
+                     Sort Key: (((t1_11.a + t1_11.b) / 2))
+                     ->  Seq Scan on prt1_e_p1 t1_11
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_2.a = t1_5.b)
+         Merge Cond: (t1_2.a = t1_7.b)
          ->  Sort
                Sort Key: t1_2.a
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+               Merge Cond: (t1_7.b = (((t1_12.a + t1_12.b) / 2)))
                ->  Sort
-                     Sort Key: t1_5.b
-                     ->  Seq Scan on prt2_p3 t1_5
+                     Sort Key: t1_7.b
+                     ->  Seq Scan on prt2_p2 t1_7
                ->  Sort
-                     Sort Key: (((t1_8.a + t1_8.b) / 2))
-                     ->  Seq Scan on prt1_e_p3 t1_8
+                     Sort Key: (((t1_12.a + t1_12.b) / 2))
+                     ->  Seq Scan on prt1_e_p2 t1_12
                            Filter: (c = 0)
-(47 rows)
+   ->  Merge Semi Join
+         Merge Cond: (t1_3.a = t1_8.b)
+         ->  Sort
+               Sort Key: t1_3.a
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_8.b = (((t1_13.a + t1_13.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_8.b
+                     ->  Seq Scan on prt2_p3 t1_8
+               ->  Sort
+                     Sort Key: (((t1_13.a + t1_13.b) / 2))
+                     ->  Seq Scan on prt1_e_p3 t1_13
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_4.a = t1_9.b)
+         ->  Sort
+               Sort Key: t1_4.a
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_9.b = (((t1_14.a + t1_14.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_9.b
+                     ->  Seq Scan on prt2_p4 t1_9
+               ->  Sort
+                     Sort Key: (((t1_14.a + t1_14.b) / 2))
+                     ->  Seq Scan on prt1_e_p4 t1_14
+                           Filter: (c = 0)
+(77 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
@@ -998,14 +1514,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
                            ->  Sort
                                  Sort Key: (((t3.a + t3.b) / 2))
-                                 ->  Seq Scan on prt1_e_p1 t3
+                                 ->  Seq Scan on prt1_e_p0 t3
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1.a
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                ->  Sort
                      Sort Key: t2.b
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Merge Left Join
                Merge Cond: (t1_1.a = t2_1.b)
                ->  Sort
@@ -1014,14 +1530,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
                            ->  Sort
                                  Sort Key: (((t3_1.a + t3_1.b) / 2))
-                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                 ->  Seq Scan on prt1_e_p1 t3_1
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_1.a
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                ->  Sort
                      Sort Key: t2_1.b
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Merge Left Join
                Merge Cond: (t1_2.a = t2_2.b)
                ->  Sort
@@ -1030,32 +1546,74 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
                            ->  Sort
                                  Sort Key: (((t3_2.a + t3_2.b) / 2))
-                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                 ->  Seq Scan on prt1_e_p2 t3_2
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_2.a
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                ->  Sort
                      Sort Key: t2_2.b
-                     ->  Seq Scan on prt2_p3 t2_2
-(51 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Merge Left Join
+               Merge Cond: (t1_3.a = t2_3.b)
+               ->  Sort
+                     Sort Key: t1_3.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_3.a + t3_3.b) / 2)) = t1_3.a)
+                           ->  Sort
+                                 Sort Key: (((t3_3.a + t3_3.b) / 2))
+                                 ->  Seq Scan on prt1_e_p3 t3_3
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_3.a
+                                 ->  Seq Scan on prt1_p3 t1_3
+               ->  Sort
+                     Sort Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Merge Left Join
+               Merge Cond: (t1_4.a = t2_4.b)
+               ->  Sort
+                     Sort Key: t1_4.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_4.a + t3_4.b) / 2)) = t1_4.a)
+                           ->  Sort
+                                 Sort Key: (((t3_4.a + t3_4.b) / 2))
+                                 ->  Seq Scan on prt1_e_p4 t3_4
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_4.a
+                                 ->  Seq Scan on prt1_p4 t1_4
+               ->  Sort
+                     Sort Key: t2_4.b
+                     ->  Seq Scan on prt2_p4 t2_4
+(83 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- MergeAppend on nullable column
 -- This should generate a partitionwise join, but currently fails to
@@ -1064,12 +1622,14 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Merge Left Join
-         Merge Cond: (prt1_p1.a = prt2_p2.b)
+         Merge Cond: (prt1_p0.a = prt2_p2.b)
          ->  Sort
-               Sort Key: prt1_p1.a
+               Sort Key: prt1_p0.a
                ->  Append
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p1
                            Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p2
@@ -1081,21 +1641,28 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                            Filter: (b > 250)
                      ->  Seq Scan on prt2_p3
                            Filter: (b > 250)
-(18 rows)
+                     ->  Seq Scan on prt2_p4
+                           Filter: (b > 250)
+(22 rows)
 
 SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  b  
------+-----
-   0 |    
-  50 |    
- 100 |    
- 150 |    
- 200 |    
- 250 |    
- 300 | 300
- 350 |    
- 400 |    
-(9 rows)
+  a   |  b  
+------+-----
+ -250 |    
+ -200 |    
+ -150 |    
+ -100 |    
+  -50 |    
+    0 |    
+   50 |    
+  100 |    
+  150 |    
+  200 |    
+  250 |    
+  300 | 300
+  350 |    
+  400 |    
+(14 rows)
 
 -- merge join when expression with whole-row reference needs to be sorted;
 -- partitionwise join does not apply
@@ -1109,30 +1676,788 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
          Sort Key: t1.a, ((((t1.*)::prt1))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
    ->  Sort
          Sort Key: t2.b, ((((t2.*)::prt2))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt2_p1 t2
-                     ->  Seq Scan on prt2_p2 t2_1
-                     ->  Seq Scan on prt2_p3 t2_2
-(16 rows)
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+(20 rows)
 
 SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.b ORDER BY t1.a;
- a  | b  
-----+----
-  0 |  0
-  6 |  6
- 12 | 12
- 18 | 18
- 24 | 24
-(5 rows)
+  a  |  b  
+-----+-----
+ -24 | -24
+ -18 | -18
+ -12 | -12
+  -6 |  -6
+   0 |   0
+   6 |   6
+  12 |  12
+  18 |  18
+  24 |  24
+(9 rows)
 
 RESET enable_hashjoin;
 RESET enable_nestloop;
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches prt2_p3 and
+-- prt2_p4 partition from prt2.
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_4
+                           Filter: (b = 0)
+(22 rows)
+
+-- partitionwise join should be possible when we drop prt2_p4 from prt2.
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+(26 rows)
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ANALYZE prt2;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -150 | -0150 | 0 | -0150
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+(7 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 |   | 
+ -200 | -0200 |   | 
+ -150 | -0150 | 0 | -0150
+ -100 | -0100 |   | 
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+(21 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -200 | 0 | -0200
+ -100 | 0 | -0100
+  -50 | 0 | -0050
+   50 | 0 | 0050
+  100 | 0 | 0100
+  200 | 0 | 0200
+  250 | 0 | 0250
+  350 | 0 | 0350
+  400 | 0 | 0400
+  500 | 0 | 0500
+  550 | 0 | 0550
+  650 | 0 | 0650
+  700 | 0 | 0700
+(14 rows)
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+                             QUERY PLAN                              
+---------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t3.c
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p0_a on prt1_p0 t3
+                           Index Cond: (a = t2.b)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t2.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_1.a = t2_1.b)
+                           ->  Seq Scan on prt1_p1 t3_1
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1 t2_1
+                                       Filter: (a = 0)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p2_a on prt1_p2 t3_2
+                           Index Cond: (a = t2_2.b)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t3_3
+                           Index Cond: (a = t2_3.b)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_4.a = t2_4.b)
+                           ->  Seq Scan on prt1_p4 t3_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t2_4
+                                       Filter: (a = 0)
+(47 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+  a   | a |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.b
+   ->  Hash Right Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p5 t2_5
+                           Filter: (a = 0)
+(24 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: (t1.a = t2.b)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.a, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
+                     Filter: (a = 0)
+(22 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
 --
 -- partitioned by multiple columns
 --
@@ -1202,82 +2527,1552 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 -- test partition matching with N-way join
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                   QUERY PLAN                                   
---------------------------------------------------------------------------------
+                                           QUERY PLAN                                           
+------------------------------------------------------------------------------------------------
  GroupAggregate
    Group Key: t1.c, t2.c, t3.c
    ->  Sort
          Sort Key: t1.c, t3.c
          ->  Append
                ->  Hash Join
-                     Hash Cond: (t1.c = ltrim(t3.c, 'A'::text))
+                     Hash Cond: ((t1.c)::text = ltrim(t3.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1.b = t2.b) AND (t1.c = t2.c))
-                           ->  Seq Scan on plt1_p1 t1
+                           Hash Cond: ((t2.b = t1.b) AND ((t2.c)::text = (t1.c)::text))
+                           ->  Seq Scan on plt2_p4 t2
                            ->  Hash
-                                 ->  Seq Scan on plt2_p1 t2
+                                 ->  Seq Scan on plt1_p4 t1
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p1 t3
+                           ->  Seq Scan on plt1_e_p4 t3
                ->  Hash Join
-                     Hash Cond: (t1_1.c = ltrim(t3_1.c, 'A'::text))
+                     Hash Cond: ((t1_1.c)::text = ltrim(t3_1.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
-                           ->  Seq Scan on plt1_p2 t1_1
+                           Hash Cond: ((t1_1.b = t2_1.b) AND ((t1_1.c)::text = (t2_1.c)::text))
+                           ->  Seq Scan on plt1_p1 t1_1
                            ->  Hash
-                                 ->  Seq Scan on plt2_p2 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p2 t3_1
+                           ->  Seq Scan on plt1_e_p1 t3_1
                ->  Hash Join
-                     Hash Cond: (t1_2.c = ltrim(t3_2.c, 'A'::text))
+                     Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
-                           ->  Seq Scan on plt1_p3 t1_2
+                           Hash Cond: ((t1_2.b = t2_2.b) AND ((t1_2.c)::text = (t2_2.c)::text))
+                           ->  Seq Scan on plt1_p2 t1_2
                            ->  Hash
-                                 ->  Seq Scan on plt2_p3 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p3 t3_2
-(32 rows)
+                           ->  Seq Scan on plt1_e_p2 t3_2
+               ->  Hash Join
+                     Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2_3.b = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text))
+                           ->  Seq Scan on plt2_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p3 t3_3
+(41 rows)
 
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  75.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 123.0000000000000000 | 123.0000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 225.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 273.0000000000000000 | 273.0000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 375.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 423.0000000000000000 | 423.0000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 525.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 573.0000000000000000 | 573.0000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p5 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_3
+                                 ->  Seq Scan on plt2_p3 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Seq Scan on plt2_p2 t2_2
+                                 ->  Seq Scan on plt2_p3 t2_3
+(22 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1302,22 +4097,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
 --------------------------------------------------
  Hash Left Join
    Hash Cond: (t2.b = a)
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t3.a = t2.b)
-               ->  Seq Scan on prt1_p1 t3
-               ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
-         ->  Hash Join
-               Hash Cond: (t3_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t3_1
-               ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
-         ->  Hash Join
-               Hash Cond: (t3_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t3_2
-               ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
+   ->  Hash Join
+         Hash Cond: (t3.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t3
+               ->  Seq Scan on prt1_p1 t3_1
+               ->  Seq Scan on prt1_p2 t3_2
+               ->  Seq Scan on prt1_p3 t3_3
+               ->  Seq Scan on prt1_p4 t3_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
    ->  Hash
          ->  Result
                One-Time Filter: false
@@ -1332,16 +4127,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 --
 -- tests for hash partitioned tables.
@@ -1417,41 +4218,9 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  273.0000000000000000 | 273.0000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
 (6 rows)
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
- Sort
-   Sort Key: t1.a
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
-               ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
-               ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
-                           Filter: (b = 0)
-(21 rows)
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
@@ -1466,26 +4235,24 @@ SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c
    Sort Key: t1.c
    ->  HashAggregate
          Group Key: t1.c, t2.c
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t2.c = t1.c)
-                     ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p1 t1
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on plt1_p4 t1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_1.c = t1_1.c)
-                     ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_p2 t1_1
+                           ->  Seq Scan on plt1_p1 t1_1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_2.c = t1_2.c)
-                     ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Seq Scan on plt1_p2 t1_2
                                  Filter: ((a % 25) = 0)
-(23 rows)
+                           ->  Seq Scan on plt1_p3 t1_3
+                                 Filter: ((a % 25) = 0)
+(21 rows)
 
 --
 -- multiple levels of partitioning
@@ -1881,64 +4648,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
  Hash Join
    Hash Cond: (t1.a = t2.a)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt4_n_p1 t2
                ->  Seq Scan on prt4_n_p2 t2_1
                ->  Seq Scan on prt4_n_p3 t2_2
-(11 rows)
+(13 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a and t1.a = t3.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                        QUERY PLAN                        
+----------------------------------------------------------
  Hash Join
-   Hash Cond: (t2.a = t1.a)
+   Hash Cond: (t3.b = t1.a)
    ->  Append
-         ->  Seq Scan on prt4_n_p1 t2
-         ->  Seq Scan on prt4_n_p2 t2_1
-         ->  Seq Scan on prt4_n_p3 t2_2
+         ->  Seq Scan on prt2_p0 t3
+         ->  Seq Scan on prt2_p1 t3_1
+         ->  Seq Scan on prt2_p2 t3_2
+         ->  Seq Scan on prt2_p3 t3_3
+         ->  Seq Scan on prt2_p4 t3_4
+         ->  Seq Scan on prt2_p5 t3_5
    ->  Hash
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.a = t3_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_2.b)
-                     ->  Seq Scan on prt1_p3 t1_2
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3 t3_2
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt4_n_p1 t2
+                           ->  Seq Scan on prt4_n_p2 t2_1
+                           ->  Seq Scan on prt4_n_p3 t2_2
 (23 rows)
 
 -- partitionwise join can not be applied if there are no equi-join conditions
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (b > t1.a)
-(12 rows)
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+(16 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partitionwise join
@@ -2024,16 +4797,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
-                     ->  Seq Scan on plt1_p1 t3
-                     ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3
+                     ->  Seq Scan on plt1_p1 t3_1
+                     ->  Seq Scan on plt1_p2 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partitionwise join can not be applied for a join between list and range
 -- partitioned tables
@@ -2044,14 +4818,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
  Hash Full Join
    Hash Cond: ((t2.c)::text = (t1.c)::text)
    ->  Append
-         ->  Seq Scan on prt1_p1 t2
-         ->  Seq Scan on prt1_p2 t2_1
-         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p0 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p2 t2_2
+         ->  Seq Scan on prt1_p3 t2_3
+         ->  Seq Scan on prt1_p4 t2_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt1_n_p1 t1
                ->  Seq Scan on prt1_n_p2 t1_1
-(10 rows)
+(12 rows)
 
 -- partitionwise join can not be applied if only one of joining tables has
 -- default partition
@@ -2067,16 +4843,279 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
    ->  Hash Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007');
+INSERT INTO plt1 SELECT i, i, to_char(i % 8, 'FM0000') FROM generate_series(0, 39) i WHERE i % 8 IN (0, 2, 3, 4, 6, 7);
+ANALYZE plt1;
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0005', '0007');
+INSERT INTO plt2 SELECT i, i, to_char(i % 8, 'FM0000') FROM generate_series(0, 39) i WHERE i % 8 IN (1, 2, 3, 4, 5, 7);
+ANALYZE plt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 8 != 2 AND COALESCE(t1.a, 0) % 8 != 3 AND COALESCE(t1.a, 0) % 8 != 4 AND COALESCE(t1.a, 0) % 8 != 7 ORDER BY t1.c, t2.c, t1.a, t2.a;
+                                                                               QUERY PLAN                                                                                
+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t2.c, t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.c = t2.c)
+               Filter: (((COALESCE(t1.a, 0) % 8) <> 2) AND ((COALESCE(t1.a, 0) % 8) <> 3) AND ((COALESCE(t1.a, 0) % 8) <> 4) AND ((COALESCE(t1.a, 0) % 8) <> 7))
+               ->  Seq Scan on plt1_p1 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t2_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 8) <> 2) AND ((COALESCE(t1_1.a, 0) % 8) <> 3) AND ((COALESCE(t1_1.a, 0) % 8) <> 4) AND ((COALESCE(t1_1.a, 0) % 8) <> 7))
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_1
+         ->  Hash Full Join
+               Hash Cond: (t1_2.c = t2_2.c)
+               Filter: (((COALESCE(t1_2.a, 0) % 8) <> 2) AND ((COALESCE(t1_2.a, 0) % 8) <> 3) AND ((COALESCE(t1_2.a, 0) % 8) <> 4) AND ((COALESCE(t1_2.a, 0) % 8) <> 7))
+               ->  Seq Scan on plt1_p3 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_2
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 8 != 2 AND COALESCE(t1.a, 0) % 8 != 3 AND COALESCE(t1.a, 0) % 8 != 4 AND COALESCE(t1.a, 0) % 8 != 7 ORDER BY t1.c, t2.c, t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+  0 | 0000 |    | 
+  8 | 0000 |    | 
+ 16 | 0000 |    | 
+ 24 | 0000 |    | 
+ 32 | 0000 |    | 
+  6 | 0006 |    | 
+ 14 | 0006 |    | 
+ 22 | 0006 |    | 
+ 30 | 0006 |    | 
+ 38 | 0006 |    | 
+    |      |  1 | 0001
+    |      |  9 | 0001
+    |      | 17 | 0001
+    |      | 25 | 0001
+    |      | 33 | 0001
+    |      |  5 | 0005
+    |      | 13 | 0005
+    |      | 21 | 0005
+    |      | 29 | 0005
+    |      | 37 | 0005
+(20 rows)
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN (NULL, '0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
+ANALYZE plt1;
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 DEFAULT;
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+ANALYZE plt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t2_1.c, t1.a, t2_1.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.c = t2_1.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 2) AND ((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt1_p1 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t2.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 2) AND ((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+  0 | 0000 |    | 
+  5 | 0000 |    | 
+ 10 | 0000 |    | 
+ 15 | 0000 |    | 
+ 20 | 0000 |    | 
+    |      |  1 | 0001
+    |      |  6 | 0001
+    |      | 11 | 0001
+    |      | 16 | 0001
+    |      | 21 | 0001
+(10 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2 t2 FULL JOIN plt1 t1 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t2_1.c, t1.a, t2_1.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t2_1.c = t1.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 2) AND ((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1
+         ->  Hash Full Join
+               Hash Cond: (t2.c = t1_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 2) AND ((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt2_p2 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_1
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2 t2 FULL JOIN plt1 t1 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+  0 | 0000 |    | 
+  5 | 0000 |    | 
+ 10 | 0000 |    | 
+ 15 | 0000 |    | 
+ 20 | 0000 |    | 
+    |      |  1 | 0001
+    |      |  6 | 0001
+    |      | 11 | 0001
+    |      | 16 | 0001
+    |      | 21 | 0001
+(10 rows)
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1;
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2;
+CREATE TABLE plt3 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt3_p1 PARTITION OF plt3 FOR VALUES IN ('0001');
+CREATE TABLE plt3_p2 PARTITION OF plt3 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2.a, t3.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.c = t3.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1.c = t2.c)
+                     ->  Seq Scan on plt1_p1 t1
+                     ->  Hash
+                           ->  Seq Scan on plt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on plt3_p1 t3
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t3_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_1.c = t2_1.c)
+                     ->  Seq Scan on plt1_p2 t1_1
+                     ->  Hash
+                           ->  Seq Scan on plt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt3_p2 t3_1
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+ a  |  c   | a  |  c   | a  |  c   
+----+------+----+------+----+------
+  0 | 0000 |    |      |    | 
+  5 | 0000 |    |      |    | 
+ 10 | 0000 |    |      |    | 
+ 15 | 0000 |    |      |    | 
+ 20 | 0000 |    |      |    | 
+  1 | 0001 |    |      |  1 | 0001
+  1 | 0001 |    |      |  6 | 0001
+  1 | 0001 |    |      | 11 | 0001
+  1 | 0001 |    |      | 16 | 0001
+  1 | 0001 |    |      | 21 | 0001
+  6 | 0001 |    |      |  1 | 0001
+  6 | 0001 |    |      |  6 | 0001
+  6 | 0001 |    |      | 11 | 0001
+  6 | 0001 |    |      | 16 | 0001
+  6 | 0001 |    |      | 21 | 0001
+ 11 | 0001 |    |      |  1 | 0001
+ 11 | 0001 |    |      |  6 | 0001
+ 11 | 0001 |    |      | 11 | 0001
+ 11 | 0001 |    |      | 16 | 0001
+ 11 | 0001 |    |      | 21 | 0001
+ 16 | 0001 |    |      |  1 | 0001
+ 16 | 0001 |    |      |  6 | 0001
+ 16 | 0001 |    |      | 11 | 0001
+ 16 | 0001 |    |      | 16 | 0001
+ 16 | 0001 |    |      | 21 | 0001
+ 21 | 0001 |    |      |  1 | 0001
+ 21 | 0001 |    |      |  6 | 0001
+ 21 | 0001 |    |      | 11 | 0001
+ 21 | 0001 |    |      | 16 | 0001
+ 21 | 0001 |    |      | 21 | 0001
+  2 | 0002 |  2 | 0002 |    | 
+  2 | 0002 |  7 | 0002 |    | 
+  2 | 0002 | 12 | 0002 |    | 
+  2 | 0002 | 17 | 0002 |    | 
+  2 | 0002 | 22 | 0002 |    | 
+  7 | 0002 |  2 | 0002 |    | 
+  7 | 0002 |  7 | 0002 |    | 
+  7 | 0002 | 12 | 0002 |    | 
+  7 | 0002 | 17 | 0002 |    | 
+  7 | 0002 | 22 | 0002 |    | 
+ 12 | 0002 |  2 | 0002 |    | 
+ 12 | 0002 |  7 | 0002 |    | 
+ 12 | 0002 | 12 | 0002 |    | 
+ 12 | 0002 | 17 | 0002 |    | 
+ 12 | 0002 | 22 | 0002 |    | 
+ 17 | 0002 |  2 | 0002 |    | 
+ 17 | 0002 |  7 | 0002 |    | 
+ 17 | 0002 | 12 | 0002 |    | 
+ 17 | 0002 | 17 | 0002 |    | 
+ 17 | 0002 | 22 | 0002 |    | 
+ 22 | 0002 |  2 | 0002 |    | 
+ 22 | 0002 |  7 | 0002 |    | 
+ 22 | 0002 | 12 | 0002 |    | 
+ 22 | 0002 | 17 | 0002 |    | 
+ 22 | 0002 | 22 | 0002 |    | 
+(55 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 92994b479bb..d8434f6b1ad 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -10,25 +10,39 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -69,11 +83,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -110,20 +132,30 @@ RESET enable_hashjoin;
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 
 EXPLAIN (COSTS OFF)
@@ -187,6 +219,114 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches prt2_p3 and
+-- prt2_p4 partition from prt2.
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- partitionwise join should be possible when we drop prt2_p4 from prt2.
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ANALYZE prt2;
+
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -211,28 +351,79 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
 -- test partition matching with N-way join
@@ -240,6 +431,175 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
@@ -285,27 +645,18 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
 ALTER TABLE plt2 DETACH PARTITION plt2_p3;
 ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
 ANALYZE plt2;
-
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.a % 25 = 0 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
 --
 -- multiple levels of partitioning
 --
@@ -450,3 +801,72 @@ ANALYZE prt2;
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007');
+INSERT INTO plt1 SELECT i, i, to_char(i % 8, 'FM0000') FROM generate_series(0, 39) i WHERE i % 8 IN (0, 2, 3, 4, 6, 7);
+ANALYZE plt1;
+
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0005', '0007');
+INSERT INTO plt2 SELECT i, i, to_char(i % 8, 'FM0000') FROM generate_series(0, 39) i WHERE i % 8 IN (1, 2, 3, 4, 5, 7);
+ANALYZE plt2;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 8 != 2 AND COALESCE(t1.a, 0) % 8 != 3 AND COALESCE(t1.a, 0) % 8 != 4 AND COALESCE(t1.a, 0) % 8 != 7 ORDER BY t1.c, t2.c, t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 8 != 2 AND COALESCE(t1.a, 0) % 8 != 3 AND COALESCE(t1.a, 0) % 8 != 4 AND COALESCE(t1.a, 0) % 8 != 7 ORDER BY t1.c, t2.c, t1.a, t2.a;
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN (NULL, '0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
+ANALYZE plt1;
+
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 DEFAULT;
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+ANALYZE plt2;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2 t2 FULL JOIN plt1 t1 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2 t2 FULL JOIN plt1 t1 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1;
+
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2;
+
+CREATE TABLE plt3 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt3_p1 PARTITION OF plt3 FOR VALUES IN ('0001');
+CREATE TABLE plt3_p2 PARTITION OF plt3 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
-- 
2.18.0

#106Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: amul sul (#105)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Hi Amul,

On Fri, Nov 15, 2019 at 2:21 PM amul sul <sulamul@gmail.com> wrote:

Thank you Fujita san for the patch & the enhancements. I am fine with your
delta patch.

OK, I'll merge the delta patch with the main one in the next version,
if no objections from others.

I would like to share some thought for other code:

File: partbounds.c:
3298 static void
3299 get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
3300 Oid *partcollations, JoinType jointype,
3301 PartitionRangeBound *left_lb,
3302 PartitionRangeBound *left_ub,
3303 PartitionRangeBound *right_lb,
3304 PartitionRangeBound *right_ub,
3305 PartitionRangeBound *merged_lb,
3306 PartitionRangeBound *merged_ub)

Can we rename these argument as inner_* & outer_* like we having for the
functions, like 0003 patch?

+1 (Actually, I too was thinking about that.)

File: partbounds.c:
3322
3323 case JOIN_INNER:
3324 case JOIN_SEMI:
3325 if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
3326 left_ub, right_ub) < 0)
3327 *merged_ub = *left_ub;
3328 else
3329 *merged_ub = *right_ub;
3330
3331 if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
3332 left_lb, right_lb) > 0)
3333 *merged_lb = *left_lb;
3334 else
3335 *merged_lb = *right_lb;
3336 break;
3337

How about reusing ub_cmpval & lb_cmpval instead of calling
compare_range_bounds() inside get_merged_range_bounds(), like 0004 patch?

Good idea! So, +1

Apart from this, I would like to propose 0005 cleanup patch where I have
rearranged function arguments & code to make sure the arguments & the code
related to lower bound should come first and then the upper bound.

+1

I'll merge these changes as well into the main patch, if no objections
of others.

Thanks for the review and patches!

Best regards,
Etsuro Fujita

#107Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Etsuro Fujita (#106)
2 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Fri, Nov 15, 2019 at 6:10 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

On Fri, Nov 15, 2019 at 2:21 PM amul sul <sulamul@gmail.com> wrote:

Thank you Fujita san for the patch & the enhancements. I am fine with your
delta patch.

OK, I'll merge the delta patch with the main one in the next version,
if no objections from others.

I would like to share some thought for other code:

File: partbounds.c:
3298 static void
3299 get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
3300 Oid *partcollations, JoinType jointype,
3301 PartitionRangeBound *left_lb,
3302 PartitionRangeBound *left_ub,
3303 PartitionRangeBound *right_lb,
3304 PartitionRangeBound *right_ub,
3305 PartitionRangeBound *merged_lb,
3306 PartitionRangeBound *merged_ub)

Can we rename these argument as inner_* & outer_* like we having for the
functions, like 0003 patch?

+1 (Actually, I too was thinking about that.)

File: partbounds.c:
3322
3323 case JOIN_INNER:
3324 case JOIN_SEMI:
3325 if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
3326 left_ub, right_ub) < 0)
3327 *merged_ub = *left_ub;
3328 else
3329 *merged_ub = *right_ub;
3330
3331 if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
3332 left_lb, right_lb) > 0)
3333 *merged_lb = *left_lb;
3334 else
3335 *merged_lb = *right_lb;
3336 break;
3337

How about reusing ub_cmpval & lb_cmpval instead of calling
compare_range_bounds() inside get_merged_range_bounds(), like 0004 patch?

Good idea! So, +1

Apart from this, I would like to propose 0005 cleanup patch where I have
rearranged function arguments & code to make sure the arguments & the code
related to lower bound should come first and then the upper bound.

+1

I'll merge these changes as well into the main patch, if no objections
of others.

Done. I modified compare_range_partitions() as well to match its the
variable names with others. Attached is a new version of the patch.

I reviewed the rest of the partbounds.c changes. Here are my review comments:

* I don't think this analysis is correct:

+/*
+ * merge_null_partitions
+ *
+ * Merge NULL partitions, i.e. a partition that can hold NULL values for a lis\
t
+ * partitioned table, if any. Find the index of merged partition to which the
+ * NULL values would belong in the join result. If one joining relation has a
+ * NULL partition but not the other, try matching it with the default partitio\
n
+ * from the other relation since the default partition may have rows with NULL
+ * partition key. We can eliminate a NULL partition when it appears only on th\
e
+ * inner side of the join and the outer side doesn't have a default partition.
+ *
+ * When the equality operator used for join is strict, two NULL values will no\
t
+ * be considered as equal, and thus a NULL partition can be eliminated for an
+ * inner join. But we don't check the strictness operator here.
+ */

First of all, I think we can assume here that the equality operator is
*strict*, because 1) have_partkey_equi_join(), which is executed
before calling merge_null_partitions(), requires that the
corresponding clause is mergejoinable, and 2) currently, we assume
that mergejoinable operators are strict (see MJEvalOuterValues() and
MJEvalInnerValues()). So I don't think it's needed to try merging a
NULL partition on one side with the default partition on the other
side as above. (merge_null_partitions() tries merging NULL partitions
as well, but in some cases I don't think we need to do that, either.)
So I rewrote merge_null_partitions() completely. Another change is
that I eliminated the NULL partition of the joinrel in more cases.
Attached is a patch (v28-0002-modify-merge_null_partitions.patch) for
that created on top of the main patch. I might be missing something,
though.

Other changes in that patch:

* I fixed memory leaks in partition_list_bounds_merge() and
partition_range_bounds_merge().

* I noticed this in merge_default_partitions():

+       Assert(outer_has_default && inner_has_default);
+
+       *default_index = map_and_merge_partitions(outer_map,
+                                                 inner_map,
+                                                 outer_default,
+                                                 inner_default,
+                                                 next_index);
+       if (*default_index == -1)
+           return false;

I think the merging should be done successfully, because of 1) this in
process_outer_partition():

+   if (inner_has_default)
+   {
+       Assert(inner_default >= 0);
+
+       /*
+        * If the outer side has the default partition as well, we need to
+        * merge the default partitions (see merge_default_partitions()); give
+        * up on it.
+        */
+       if (outer_has_default)
+           return false;

and 2) this in process_inner_partition():

+   if (outer_has_default)
+   {
+       Assert(outer_default >= 0);
+
+       /*
+        * If the inner side has the default partition as well, we need to
+        * merge the default partitions (see merge_default_partitions()); give
+        * up on it.
+        */
+       if (inner_has_default)
+           return false;

So I removed the above if test (ie, *default_index == -1). I also
modified that function a bit further, including comments.

* I simplified process_outer_partition() and process_inner_partition()
a bit, changing the APIs to match that of map_and_merge_partitions().
Also, I think this in these functions is a bit ugly;

+           /* Don't add this index to the list of merged indexes. */
+           *merged_index = -1;

so I removed it, and modified the condition on whether or not we add
merged bounds to the lists in partition_list_bounds_merge() and
partition_range_bounds_merge(), instead.

That's it. Sorry for the delay.

Best regards,
Etsuro Fujita

Attachments:

v28-0001-Improve-partition-matching-for-partitionwise-join.patchapplication/octet-stream; name=v28-0001-Improve-partition-matching-for-partitionwise-join.patchDownload
From 89af632922c18e43e542b98fd540ba792f584013 Mon Sep 17 00:00:00 2001
From: Etsuro Fujita <efujita@postgresql.org>
Date: Fri, 22 Nov 2019 21:46:26 +0900
Subject: [PATCH 1/2] Improve partition matching for partitionwise join

---
 src/backend/nodes/outfuncs.c                 |    2 +
 src/backend/optimizer/path/joinrels.c        |  246 +-
 src/backend/optimizer/util/inherit.c         |    2 +
 src/backend/optimizer/util/relnode.c         |   45 +-
 src/backend/partitioning/partbounds.c        | 1607 +++++++
 src/include/nodes/pathnodes.h                |    5 +
 src/include/partitioning/partbounds.h        |    7 +
 src/test/regress/expected/partition_join.out | 4429 +++++++++++++++---
 src/test/regress/sql/partition_join.sql      |  482 +-
 9 files changed, 6047 insertions(+), 778 deletions(-)

diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index b0dcd02ff6..b815157402 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -2281,6 +2281,8 @@ _outRelOptInfo(StringInfo str, const RelOptInfo *node)
 	WRITE_BOOL_FIELD(has_eclass_joins);
 	WRITE_BOOL_FIELD(consider_partitionwise_join);
 	WRITE_BITMAPSET_FIELD(top_parent_relids);
+	WRITE_BOOL_FIELD(merged);
+	WRITE_BITMAPSET_FIELD(all_partrels);
 	WRITE_NODE_FIELD(partitioned_child_rels);
 }
 
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 6a480ab764..ed7bc23c7b 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -48,6 +48,9 @@ static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 												Relids left_relids, Relids right_relids);
 static int	match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel,
 										 bool strict_op);
+static void get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2);
 
 
 /*
@@ -1357,25 +1360,30 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	bool		merged = false;
+	List	   *parts1 = NIL;
+	List	   *parts2 = NIL;
+	ListCell   *lcr1 = NULL;
+	ListCell   *lcr2 = NULL;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1390,34 +1398,107 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		   joinrel->part_scheme == rel2->part_scheme);
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * If we don't have the partition bounds for the join rel yet, try to
+	 * create it along with pairs of partitions to be joined; else generate
+	 * those using the partitioning info for the join rel we already have.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	if (joinrel->nparts == -1)
+	{
+		PartitionScheme part_scheme = joinrel->part_scheme;
+		PartitionBoundInfo boundinfo = NULL;
+		int			nparts = 0;
+
+		Assert(joinrel->boundinfo == NULL);
+		Assert(joinrel->part_rels == NULL);
+
+		/*
+		 * See if the partition bounds for inputs are exactly the same, in
+		 * which case we don't need to work hard: partitions with the same
+		 * partition indexes will form join pairs, and the join rel will have
+		 * the same partition bounds as inputs; otherwise try to merge the
+		 * partition bounds along with generating join pairs.
+		 *
+		 * Even if one or both inputs have merged partition bounds, it'd be
+		 * possible for the partition bounds to be exactly the same, but it
+		 * seems unlikely to be worth the cycles to check; do this check only
+		 * if both inputs have non-merged partition bounds.
+		 */
+		if (!rel1->merged &&
+			!rel2->merged &&
+			rel1->nparts == rel2->nparts &&
+			partition_bounds_equal(part_scheme->partnatts,
+								   part_scheme->parttyplen,
+								   part_scheme->parttypbyval,
+								   rel1->boundinfo, rel2->boundinfo))
+		{
+			boundinfo = rel1->boundinfo;
+			nparts = rel1->nparts;
+		}
+		else
+		{
+			boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											   part_scheme->parttyplen,
+											   part_scheme->parttypbyval,
+											   part_scheme->partsupfunc,
+											   part_scheme->partcollation,
+											   rel1, rel2,
+											   parent_sjinfo->jointype,
+											   &parts1, &parts2);
+			if (boundinfo == NULL)
+			{
+				joinrel->nparts = 0;
+				return;
+			}
+			nparts = list_length(parts1);
+			merged = true;
+		}
+
+		Assert(nparts > 0);
+		joinrel->boundinfo = boundinfo;
+		joinrel->merged = merged;
+		joinrel->nparts = nparts;
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+	}
+	else
+	{
+		Assert(joinrel->nparts > 0);
+		Assert(joinrel->boundinfo);
+		Assert(joinrel->part_rels);
+
+		/*
+		 * If the partition bounds for the join rel are not merged ones,
+		 * inputs are guaranteed to have the same partition bounds, so
+		 * partitions with the same partition indexes will form join pairs;
+		 * else let get_matching_part_pairs() do the work.
+		 */
+		if (joinrel->merged)
+		{
+			get_matching_part_pairs(root, joinrel, rel1, rel2,
+									&parts1, &parts2);
+			Assert(list_length(parts1) == joinrel->nparts);
+			Assert(list_length(parts2) == joinrel->nparts);
+			merged = true;
+		}
+	}
 
-	nparts = joinrel->nparts;
+	if (merged)
+	{
+		lcr1 = list_head(parts1);
+		lcr2 = list_head(parts2);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
-		bool		rel1_empty = (child_rel1 == NULL ||
-								  IS_DUMMY_REL(child_rel1));
-		bool		rel2_empty = (child_rel2 == NULL ||
-								  IS_DUMMY_REL(child_rel2));
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		bool		rel1_empty;
+		bool		rel2_empty;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1425,6 +1506,22 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		if (merged)
+		{
+			child_rel1 = lfirst_node(RelOptInfo, lcr1);
+			child_rel2 = lfirst_node(RelOptInfo, lcr2);
+			lcr1 = lnext(parts1, lcr1);
+			lcr2 = lnext(parts2, lcr2);
+		}
+		else
+		{
+			child_rel1 = rel1->part_rels[cnt_parts];
+			child_rel2 = rel2->part_rels[cnt_parts];
+		}
+
+		rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1));
+		rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2));
+
 		/*
 		 * Check for cases where we can prove that this segment of the join
 		 * returns no rows, due to one or both inputs being empty (including
@@ -1522,6 +1619,8 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 												 child_sjinfo,
 												 child_sjinfo->jointype);
 			joinrel->part_rels[cnt_parts] = child_joinrel;
+			joinrel->all_partrels = bms_add_members(joinrel->all_partrels,
+													child_joinrel->relids);
 		}
 
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
@@ -1529,6 +1628,7 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
+
 	}
 }
 
@@ -1738,3 +1838,99 @@ match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op)
 
 	return -1;
 }
+
+/*
+ * get_matching_part_pairs
+ *		Generate join pairs of partitions for the two inputs
+ */
+static void
+get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2)
+{
+	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
+	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
+	int 		cnt_parts;
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
+	{
+		RelOptInfo *child_joinrel = joinrel->part_rels[cnt_parts];
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		Relids		child_relids1;
+		Relids		child_relids2;
+
+		/*
+		 * If this segment of the join is empty, it means that this segment
+		 * was ignored when previously creating child-join paths for it in
+		 * try_partitionwise_join() as it would not contribute to the join
+		 * result, due to one or both inputs being empty; add NULL to each of
+		 * the given lists so that this segment will be ignored again in that
+		 * function.
+		 */
+		if (!child_joinrel)
+		{
+			*parts1 = lappend(*parts1, NULL);
+			*parts2 = lappend(*parts2, NULL);
+			continue;
+		}
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel1 side.
+		 */
+		child_relids1 = bms_intersect(child_joinrel->relids,
+									  rel1->all_partrels);
+		Assert(bms_num_members(child_relids1) == bms_num_members(rel1->relids));
+
+		/*
+		 * Get a child rel for rel1 with the relids.  Note that we should have
+		 * the child rel even if rel1 is a join rel, because in that case the
+		 * partitions specified in the relids would have matching/overlapping
+		 * boundaries, so those partitions should be considered as ones to be
+		 * joined even when planning partitionwise joins of rel1, meaning that
+		 * the child rel would have been built by the time we get here.
+		 */
+		if (rel1_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids1);
+
+			child_rel1 = find_base_rel(root, varno);
+		}
+		else
+			child_rel1 = find_join_rel(root, child_relids1);
+		Assert(child_rel1);
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel2 side.
+		 */
+		child_relids2 = bms_intersect(child_joinrel->relids,
+									  rel2->all_partrels);
+		Assert(bms_num_members(child_relids2) == bms_num_members(rel2->relids));
+
+		/*
+		 * Get a child rel for rel2 with the relids.  See above comments.
+		 */
+		if (rel2_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids2);
+
+			child_rel2 = find_base_rel(root, varno);
+		}
+		else
+			child_rel2 = find_join_rel(root, child_relids2);
+		Assert(child_rel2);
+
+		/*
+		 * The join of rel1 and rel2 is legal, so is the join of the child
+		 * rels obtained above; add them to the given lists as a join pair
+		 * producing this join segment.
+		 */
+		*parts1 = lappend(*parts1, child_rel1);
+		*parts2 = lappend(*parts2, child_rel2);
+	}
+}
diff --git a/src/backend/optimizer/util/inherit.c b/src/backend/optimizer/util/inherit.c
index 38bc61e687..caf6039c10 100644
--- a/src/backend/optimizer/util/inherit.c
+++ b/src/backend/optimizer/util/inherit.c
@@ -376,6 +376,8 @@ expand_partitioned_rtentry(PlannerInfo *root, RelOptInfo *relinfo,
 		/* Create the otherrel RelOptInfo too. */
 		childrelinfo = build_simple_rel(root, childRTindex, relinfo);
 		relinfo->part_rels[i] = childrelinfo;
+		relinfo->all_partrels = bms_add_members(relinfo->all_partrels,
+												childrelinfo->relids);
 
 		/* If this child is itself partitioned, recurse */
 		if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 03e02423b2..a1cf632932 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -240,10 +240,12 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->has_eclass_joins = false;
 	rel->consider_partitionwise_join = false;	/* might get changed later */
 	rel->part_scheme = NULL;
-	rel->nparts = 0;
+	rel->nparts = -1;
 	rel->boundinfo = NULL;
+	rel->merged = false;
 	rel->partition_qual = NIL;
 	rel->part_rels = NULL;
+	rel->all_partrels = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->partitioned_child_rels = NIL;
@@ -653,10 +655,12 @@ build_join_rel(PlannerInfo *root,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -829,10 +833,12 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -1639,7 +1645,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1652,24 +1658,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1681,17 +1669,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index cfb44e23e2..a362998e57 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -25,6 +25,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/pathnodes.h"
 #include "parser/parse_coerce.h"
 #include "partitioning/partbounds.h"
 #include "partitioning/partdesc.h"
@@ -69,6 +70,23 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+typedef struct PartitionMap
+{
+	int			nparts;			/* number of partitions */
+	int		   *merged_indexes;	/* indexes of merged partitions */
+	bool	   *merged;			/* flags to indicate whether partitions are
+								 * merged with non-dummy partitions */
+	bool		did_remapping;	/* did we remap partitions? */
+	int		   *old_indexes;	/* old indexes of merged partitions if
+								 * did_remapping */
+} PartitionMap;
+
+#define compare_range_bounds(partnatts, partsupfunc, partcollations, \
+							 bound1, bound2) \
+	(partition_rbound_cmp(partnatts, partsupfunc, partcollations, \
+						  (bound1)->datums, (bound1)->kind, (bound1)->lower, \
+						  bound2))
+
 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);
@@ -108,6 +126,95 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 									 Expr **keyCol,
 									 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static PartitionBoundInfo partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts);
+static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							JoinType jointype,
+							List **outer_parts, List **inner_parts);
+static void init_partition_map(RelOptInfo *rel, PartitionMap *map);
+static void free_partition_map(PartitionMap *map);
+static int map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_part, int inner_part, int *next_index);
+static int merge_partition_with_dummy(PartitionMap *map, int index,
+						   int *next_index);
+static bool process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index,
+						int *merged_index);
+static bool process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index,
+						int *merged_index);
+static void fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes);
+static void generate_matching_part_pairs(RelOptInfo *outer_rel,
+							 RelOptInfo *inner_rel,
+							 PartitionMap *outer_map,
+							 PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_part_list,
+							 List **inner_part_list);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int get_range_partition(PartitionBoundInfo bi, int *lb_pos,
+					PartitionRangeBound *lb, PartitionRangeBound *ub);
+static bool compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+						 Oid *partcollations,
+						 PartitionRangeBound *outer_lb,
+						 PartitionRangeBound *outer_ub,
+						 PartitionRangeBound *inner_lb,
+						 PartitionRangeBound *inner_ub,
+						 int *lb_cmpval, int *ub_cmpval);
+static void get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations, JoinType jointype,
+						PartitionRangeBound *outer_lb,
+						PartitionRangeBound *outer_ub,
+						PartitionRangeBound *inner_lb,
+						PartitionRangeBound *inner_ub,
+						int	lb_cmpval, int ub_cmpval,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub);
+static void add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub,
+						int merged_index,
+						List **merged_datums,
+						List **merged_kinds,
+						List **merged_indexes);
+static bool merge_default_partitions(PartitionMap *outer_map,
+						 PartitionMap *inner_map,
+						 bool outer_has_default,
+						 int outer_default,
+						 bool inner_has_default,
+						 int inner_default,
+						 JoinType jointype,
+						 int *next_index,
+						 int *default_index);
+static bool merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_map, PartitionMap *inner_map,
+					  bool outer_has_default, bool inner_has_default,
+					  bool outer_has_null, bool inner_has_null,
+					  JoinType jointype, int *next_index,
+					  int *default_index, int *null_index);
 
 /*
  * get_qual_from_partbound
@@ -2995,3 +3102,1503 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * This function builds and returns the partition bounds for a join relation
+ * between input relations, creating two lists of partitions, which are
+ * returned to *outer_parts and *inner_parts respectively.  The lists contain
+ * the same number of partitions, and the partitions at the same positions in
+ * the lists indicate join pairs used for partitioned join.
+ *
+ * This function returns NULL, setting *outer_parts and *inner_parts to NIL,
+ * if a partition on one side matches multiple partitions on the other side,
+ * in which case we currently don't support partitioned join.
+ */
+PartitionBoundInfo
+partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
+{
+	PartitionBoundInfo 	merged_bounds;
+	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
+						inner_binfo = inner_rel->boundinfo;
+	char				strategy = outer_binfo->strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = NULL;
+
+			break;
+
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partsupfunc,
+														partcollation,
+														outer_rel,
+														inner_rel,
+														jointype,
+														outer_parts,
+														inner_parts);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(partnatts,
+														 partsupfunc,
+														 partcollation,
+														 outer_rel,
+														 inner_rel,
+														 jointype,
+														 outer_parts,
+														 inner_parts);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	return merged_bounds;
+}
+
+/*
+ * get_range_partition
+ *		Returns the index of the range partition with the given lower bound
+ *
+ * *lb and *ub are set to the lower and upper bounds of the range partition
+ * respectively, and *lb_index is advanced to the next lower bound, if any.
+ */
+static int
+get_range_partition(PartitionBoundInfo bi, int *lb_index,
+					PartitionRangeBound *lb, PartitionRangeBound *ub)
+{
+	/* Return the index as -1 if we've exhausted all the lower bounds. */
+	if (*lb_index >= bi->ndatums)
+		return -1;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(*lb_index + 1 < bi->ndatums);
+
+	lb->index = bi->indexes[*lb_index];
+	lb->kind = bi->kind[*lb_index];
+	lb->datums = bi->datums[*lb_index];
+	lb->lower = true;
+	ub->index = bi->indexes[*lb_index + 1];
+	ub->kind = bi->kind[*lb_index + 1];
+	ub->datums = bi->datums[*lb_index + 1];
+	ub->lower = false;
+
+	/* The partition index of an upper bound should be valid. */
+	Assert(ub->index >= 0);
+
+	/*
+	 * Advance the lower bound; if there are no bounds left beyond the upper
+	 * bound, we have reached the last lower bound.
+	 */
+	if (*lb_index + 2 >= bi->ndatums)
+		*lb_index = bi->ndatums;
+	else
+	{
+		/*
+		 * If the index assigned to the bound next to the upper bound isn't
+		 * valid, that is the lower bound of the next range partition; else,
+		 * the upper bound of the current range partition is also the lower
+		 * bound of the next range partition.
+		 */
+		if (bi->indexes[*lb_index + 2] < 0)
+			*lb_index = *lb_index + 2;
+		else
+			*lb_index = *lb_index + 1;
+	}
+
+	return ub->index;
+}
+
+/*
+ * compare_range_partitions
+ *		Compares the bounds of two range partitions, and returns true if the
+ *		ranges of the partitions overlap, false otherwise
+ *
+ * *lb_cmpval is set to -1, 0 or 1 if the outer partition's lower bound is
+ * lower than, equal to or higher than the inner partition's lower bound
+ * respectively.  Likewise, *ub_cmpval is set to -1, 0 or 1 if the outer
+ * partition's upper bound is lower than, equal to or higher than the inner
+ * partition's upper bound respectively.
+ */
+static bool
+compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+						 Oid *partcollations,
+						 PartitionRangeBound *outer_lb,
+						 PartitionRangeBound *outer_ub,
+						 PartitionRangeBound *inner_lb,
+						 PartitionRangeBound *inner_ub,
+						 int *lb_cmpval, int *ub_cmpval)
+{
+	/*
+	 * Check to see if the upper bound of the outer partition is lower than
+	 * the lower bound of the inner partition; in which case the partitions
+	 * aren't overlapping.
+	 */
+	if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+							 outer_ub, inner_lb) < 0)
+	{
+		*lb_cmpval = -1;
+		*ub_cmpval = -1;
+		return false;
+	}
+
+	/*
+	 * Check to see if the lower bound of the outer partition is higher than
+	 * the upper bound of the inner partition; in which case the partitions
+	 * aren't overlapping.
+	 */
+	if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+							 outer_lb, inner_ub) > 0)
+	{
+		*lb_cmpval = 1;
+		*ub_cmpval = 1;
+		return false;
+	}
+
+	/* All other cases indicate overlapping partitions. */
+	*lb_cmpval = compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									  outer_lb, inner_lb);
+	*ub_cmpval = compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									  outer_ub, inner_ub);
+	return true;
+}
+
+/*
+ * get_merged_range_bounds
+ *		Given the bounds of range partitions to be join, determine the range
+ *		bounds of the merged partition produced from the range partitions
+ *
+ * *merged_lb and *merged_ub are set to the lower and upper bounds of the
+ * merged partition.
+ */
+static void
+get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations, JoinType jointype,
+						PartitionRangeBound *outer_lb,
+						PartitionRangeBound *outer_ub,
+						PartitionRangeBound *inner_lb,
+						PartitionRangeBound *inner_ub,
+						int	lb_cmpval, int ub_cmpval,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub)
+{
+	Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								outer_lb, inner_lb) == lb_cmpval);
+	Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								outer_ub, inner_ub) == ub_cmpval);
+
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			*merged_lb = (lb_cmpval > 0) ? *outer_lb : *inner_lb;
+			*merged_ub = (ub_cmpval < 0) ? *outer_ub : *inner_ub;
+			break;
+
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = *outer_ub;
+			*merged_lb = *outer_lb;
+			break;
+
+		case JOIN_FULL:
+			*merged_lb = (lb_cmpval < 0) ? *outer_lb : *inner_lb;
+			*merged_ub = (ub_cmpval > 0) ? *outer_ub : *inner_ub;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+}
+
+/*
+ * add_merged_range_bounds
+ *		Add the range bounds of a merged partition to the lists of range
+ *		bounds
+ */
+static void
+add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub,
+						int merged_index,
+						List **merged_datums,
+						List **merged_kinds,
+						List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		/* First merged partition */
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		Assert(*merged_kinds && *merged_indexes);
+
+		/* Get the last upper bound. */
+		prev_ub.index = llast_int(*merged_indexes);
+		prev_ub.datums = (Datum *) llast(*merged_datums);
+		prev_ub.kind = (PartitionRangeDatumKind *) llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		/*
+		 * We pass to partition_rbound_cmp() lower1 as false to prevent it
+		 * from considering the last upper bound to be smaller than the lower
+		 * bound of the merged partition when the values of the two range
+		 * bounds compare equal.
+		 */
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  merged_lb->datums, merged_lb->kind,
+									  false, &prev_ub);
+		Assert(cmpval >= 0);
+	}
+
+	/*
+	 * If the lower bound is higher than the last upper bound, add the lower
+	 * bound with the index as -1 indicating that that is a lower bound; else,
+	 * the last upper bound will be reused as the lower bound of the merged
+	 * partition, so skip this.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, merged_lb->datums);
+		*merged_kinds = lappend(*merged_kinds, merged_lb->kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	/* Add the upper bound and index of the merged partition. */
+	*merged_datums = lappend(*merged_datums, merged_ub->datums);
+	*merged_kinds = lappend(*merged_kinds, merged_ub->kind);
+	*merged_indexes = lappend_int(*merged_indexes, merged_index);
+}
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			outer_part;
+	int			inner_part;
+	int			outer_lb_index;
+	int			inner_lb_index;
+	PartitionRangeBound outer_lb;
+	PartitionRangeBound outer_ub;
+	PartitionRangeBound inner_lb;
+	PartitionRangeBound inner_ub;
+	int			next_index = 0;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_kinds = NIL;
+	List	   *merged_indexes = NIL;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = inner_lb_index = 0;
+	outer_part = get_range_partition(outer_bi, &outer_lb_index,
+									 &outer_lb, &outer_ub);
+	inner_part = get_range_partition(inner_bi, &inner_lb_index,
+									 &inner_lb, &inner_ub);
+	while (outer_part >= 0 || inner_part >= 0)
+	{
+		PartitionRangeBound merged_lb;
+		PartitionRangeBound merged_ub;
+		int			merged_index = -1;
+		bool		overlap;
+		int			ub_cmpval;
+		int			lb_cmpval;
+
+		if (outer_part >= 0)
+			Assert(outer_map.merged_indexes[outer_part] == -1 &&
+				   outer_map.merged[outer_part] == false);
+		if (inner_part >= 0)
+			Assert(inner_map.merged_indexes[inner_part] == -1 &&
+				   inner_map.merged[inner_part] == false);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (outer_part == -1)
+		{
+			overlap = false;
+			lb_cmpval = 1;
+			ub_cmpval = 1;
+		}
+		else if (inner_part == -1)
+		{
+			overlap = false;
+			lb_cmpval = -1;
+			ub_cmpval = -1;
+		}
+		else
+			overlap = compare_range_partitions(partnatts, partsupfuncs,
+											   partcollations,
+											   &outer_lb, &outer_ub,
+											   &inner_lb, &inner_ub,
+											   &lb_cmpval, &ub_cmpval);
+
+		if (overlap)
+		{
+			/* The ranges of partitions overlap; form a join pair of them. */
+
+			PartitionRangeBound save_outer_ub;
+			PartitionRangeBound save_inner_ub;
+
+			/* Get the bounds of the merged partition. */
+			get_merged_range_bounds(partnatts, partsupfuncs,
+									partcollations, jointype,
+									&outer_lb, &outer_ub,
+									&inner_lb, &inner_ub,
+									lb_cmpval, ub_cmpval,
+									&merged_lb, &merged_ub);
+
+			/*
+			 * Get the index of the merged partition.  Both partitions aren't
+			 * merged yet, so the partitions should be merged successfully.
+			 */
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
+													outer_part, inner_part,
+													&next_index);
+			Assert(merged_index >= 0);
+
+			/* Save the upper bounds of both partitions for use below. */
+			save_outer_ub = outer_ub;
+			save_inner_ub = inner_ub;
+
+			/* Move to the next pair of partitions. */
+			outer_part = get_range_partition(outer_bi, &outer_lb_index,
+											 &outer_lb, &outer_ub);
+			inner_part = get_range_partition(inner_bi, &inner_lb_index,
+											 &inner_lb, &inner_ub);
+
+			/*
+			 * If the range of a partition on one side overlaps the range of
+			 * the next partition on the other side, that will cause the
+			 * partition on one side to match at least two partitions on the
+			 * other side, which is the case that we currently don't support
+			 * partitioned join for; give up.
+			 */
+			if (ub_cmpval > 0 && inner_part >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &save_outer_ub, &inner_lb) > 0)
+				return NULL;
+			if (ub_cmpval < 0 && outer_part >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &outer_lb, &save_inner_ub) < 0)
+				return NULL;
+
+			/*
+			 * A row from a non-overlapping portion (if any) of a partition
+			 * on one side might find its join partner in the default
+			 * partition (if any) on the other side, causing the same
+			 * situation as above; if so, give up.
+			 */
+			if ((outer_has_default && (lb_cmpval > 0 || ub_cmpval < 0)) ||
+				(inner_has_default && (lb_cmpval < 0 || ub_cmpval > 0)))
+				return NULL;
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			merged_lb = outer_lb;
+			merged_ub = outer_ub;
+
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				if (!process_outer_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 outer_part,
+											 inner_default,
+											 jointype,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the outer side. */
+			outer_part = get_range_partition(outer_bi, &outer_lb_index,
+											 &outer_lb, &outer_ub);
+		}
+		else
+		{
+			/* Upper bound of outer range higher than that of the inner. */
+			Assert(ub_cmpval > 0);
+
+			merged_lb = inner_lb;
+			merged_ub = inner_ub;
+
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				if (!process_inner_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 inner_part,
+											 outer_default,
+											 jointype,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the inner side. */
+			inner_part = get_range_partition(inner_bi, &inner_lb_index,
+											 &inner_lb, &inner_ub);
+		}
+
+		if (merged_index >= 0)
+		{
+			/* Add the range bounds of the merged partition. */
+			add_merged_range_bounds(partnatts, partsupfuncs, partcollations,
+									&merged_lb, &merged_ub, merged_index,
+									&merged_datums, &merged_kinds,
+									&merged_indexes);
+		}
+	}
+
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+	{
+		if (!merge_default_partitions(&outer_map,
+									  &inner_map,
+									  outer_has_default,
+									  outer_default,
+									  inner_has_default,
+									  inner_default,
+									  jointype,
+									  &next_index,
+									  &default_index))
+			return NULL;
+	}
+	else
+		Assert(default_index == -1);
+
+	/*
+	 * Unlike the case for list partitioning, we wouldn't have re-merged
+	 * partitions, so did_remapping should be left alone.
+	 */
+	Assert(!outer_map.did_remapping && !inner_map.did_remapping);
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 &outer_map, &inner_map,
+								 next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes,
+													  merged_kinds,
+													  -1, default_index);
+	}
+
+	/* Free any memory we used in this function. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							JoinType jointype,
+							List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			next_index = 0;
+	int			null_index = -1;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			cnto;
+	int			cnti;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *merged_datum = NULL;
+		int			merged_index = -1;
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			cmpval;
+
+		/* Get the list datums of the next pair of partitions. */
+		odatums = cnto < outer_bi->ndatums ? outer_bi->datums[cnto] : NULL;
+		idatums = cnti < inner_bi->ndatums ? inner_bi->datums[cnti] : NULL;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+			cmpval = 1;
+		else if (cnti >= inner_bi->ndatums)
+			cmpval = -1;
+		else
+		{
+			Assert(odatums != NULL && idatums != NULL);
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0],
+													 idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			int			o_index = outer_bi->indexes[cnto];
+			int			i_index = inner_bi->indexes[cnti];
+
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			Assert(o_index >= 0 && i_index >= 0);
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
+													o_index, i_index,
+													&next_index);
+
+			if (merged_index < 0)
+				return NULL;
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			Assert(cnto < outer_bi->ndatums);
+
+			/* A datum missing from the inner side. */
+			merged_datum = odatums;
+
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				int			o_index = outer_bi->indexes[cnto];
+
+				Assert(o_index >= 0);
+				if (!process_outer_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 o_index,
+											 inner_default,
+											 jointype,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next datum on the outer side. */
+			cnto++;
+		}
+		else
+		{
+			Assert(cmpval > 0);
+			Assert(cnti < inner_bi->ndatums);
+
+			/* A datum missing from the outer side. */
+			merged_datum = idatums;
+
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				int			i_index = inner_bi->indexes[cnti];
+
+				Assert(i_index >= 0);
+				if (!process_inner_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 i_index,
+											 outer_default,
+											 jointype,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next datum on the inner side. */
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	/* Merge null partitions if any. */
+	if (outer_has_null || inner_has_null)
+	{
+		if (!merge_null_partitions(outer_bi, inner_bi,
+								   &outer_map, &inner_map,
+								   outer_has_default, inner_has_default,
+								   outer_has_null, inner_has_null,
+								   jointype, &next_index, &default_index,
+								   &null_index))
+			return NULL;
+	}
+	else
+		Assert(null_index == -1);
+
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+	{
+		if (!merge_default_partitions(&outer_map, &inner_map,
+									  outer_has_default, outer_default,
+									  inner_has_default, inner_default,
+									  jointype, &next_index, &default_index))
+			return NULL;
+	}
+	else
+		Assert(default_index == -1);
+
+	/* Fix the merged_indexes list if necessary. */
+	if (outer_map.did_remapping || inner_map.did_remapping)
+	{
+		Assert(jointype == JOIN_FULL);
+		fix_merged_indexes(&outer_map, &inner_map,
+						   next_index, merged_indexes);
+	}
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 &outer_map, &inner_map,
+								 next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) <= next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes, NIL,
+													  null_index, default_index);
+	}
+
+	/* Free up all extra memory before returning from this function. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+	list_free(merged_datums);
+	list_free(merged_indexes);
+
+	return merged_bounds;
+}
+
+/*
+ * init_partition_map
+ *
+ * Initialize a PartitionMap struct for given relation.
+ */
+static void
+init_partition_map(RelOptInfo *rel, PartitionMap *map)
+{
+	int			nparts = rel->nparts;
+	int			i;
+
+	map->nparts = nparts;
+	map->merged_indexes = (int *) palloc(sizeof(int) * nparts);
+	map->merged = (bool *) palloc(sizeof(bool) * nparts);
+	map->did_remapping = false;
+	map->old_indexes = (int *) palloc(sizeof(int) * nparts);
+	for (i = 0; i < nparts; i++)
+	{
+		map->merged_indexes[i] = map->old_indexes[i] = -1;
+		map->merged[i] = false;
+	}
+}
+
+/*
+ * free_partition_map
+ */
+static void
+free_partition_map(PartitionMap *map)
+{
+	pfree(map->merged_indexes);
+	pfree(map->merged);
+	pfree(map->old_indexes);
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given partitions.
+ */
+static int
+map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_index, int inner_index, int *next_index)
+{
+	int 		outer_merged_index;
+	bool 		outer_merged;
+	int 		inner_merged_index;
+	bool 		inner_merged;
+
+	Assert(outer_index >= 0 && outer_index < outer_map->nparts);
+	outer_merged_index = outer_map->merged_indexes[outer_index];
+	outer_merged = outer_map->merged[outer_index];
+	Assert(inner_index >= 0 && inner_index < inner_map->nparts);
+	inner_merged_index = inner_map->merged_indexes[inner_index];
+	inner_merged = inner_map->merged[inner_index];
+
+	/*
+	 * Handle cases where both partitions are mapped to merged partitions.
+	 */
+	if (outer_merged_index >= 0 && inner_merged_index >= 0)
+	{
+		/*
+		 * If the mereged partitions are the same, no need to do anything;
+		 * return the index of the merged partition.  Otherwise, if both
+		 * partitions are merged with dummy partitions, re-merge them; map
+		 * them to the merged partition with the smaller of the two merged
+		 * indexes and return the smaller index.  Otherwise they can't be
+		 * merged, so return -1.
+		 */
+		if (outer_merged_index == inner_merged_index)
+		{
+			Assert(outer_merged);
+			Assert(inner_merged);
+			return outer_merged_index;
+		}
+		if (!outer_merged && !inner_merged)
+		{
+			/*
+			 * Note that we will fix the larger index that have been added to
+			 * the merged_indexes list so far in fix_merged_indexes().
+			 */
+			if (outer_merged_index < inner_merged_index)
+			{
+				outer_map->merged[outer_index] = true;
+				inner_map->merged_indexes[inner_index] = outer_merged_index;
+				inner_map->merged[inner_index] = true;
+				inner_map->did_remapping = true;
+				inner_map->old_indexes[inner_index] = inner_merged_index;
+				return outer_merged_index;
+			}
+			else
+			{
+				inner_map->merged[inner_index] = true;
+				outer_map->merged_indexes[outer_index] = inner_merged_index;
+				outer_map->merged[outer_index] = true;
+				outer_map->did_remapping = true;
+				outer_map->old_indexes[outer_index] = outer_merged_index;
+				return inner_merged_index;
+			}
+		}
+		return -1;
+	}
+
+	/* At least one partition isn't mapped to a merged partition. */
+	Assert(outer_merged_index == -1 || inner_merged_index == -1);
+
+	/*
+	 * If neither of partitions isn't mapped, assign them a new merged
+	 * partition and return the index of the merged partition.  Otherwise, if
+	 * one of partitions is merged with a dummy relation (and the other isn't
+	 * merged), re-merge it with the other, with the same index, and return
+	 * the index.  Otherwise they can't be merged, so return -1.
+	 */
+	if (outer_merged_index == -1 && inner_merged_index == -1)
+	{
+		int 		merged_index = *next_index;
+
+		Assert(!outer_merged);
+		Assert(!inner_merged);
+		outer_map->merged_indexes[outer_index] = merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged_indexes[inner_index] = merged_index;
+		inner_map->merged[inner_index] = true;
+		*next_index = *next_index + 1;
+		return merged_index;
+	}
+	if (outer_merged_index >= 0 && !outer_map->merged[outer_index])
+	{
+		Assert(inner_merged_index == -1);
+		Assert(!inner_merged);
+		inner_map->merged_indexes[inner_index] = outer_merged_index;
+		inner_map->merged[inner_index] = true;
+		outer_map->merged[outer_index] = true;
+		return outer_merged_index;
+	}
+	if (inner_merged_index >= 0 && !inner_map->merged[inner_index])
+	{
+		Assert(outer_merged_index == -1);
+		Assert(!outer_merged);
+		outer_map->merged_indexes[outer_index] = inner_merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged[inner_index] = true;
+		return inner_merged_index;
+	}
+	return -1;
+}
+
+/*
+ * merge_partition_with_dummy
+ *
+ * *next_index is incremented.
+ */
+static int
+merge_partition_with_dummy(PartitionMap *map, int index, int *next_index)
+{
+	int 		merged_index = *next_index;
+
+	Assert(index >= 0 && index < map->nparts);
+	Assert(map->merged_indexes[index] == -1);
+	Assert(!map->merged[index]);
+	map->merged_indexes[index] = merged_index;
+	/* Leave the merged flag alone! */
+	*next_index = *next_index + 1;
+	return merged_index;
+}
+
+/*
+ * process_outer_partition
+ *
+ * Determine the merged partition associated with the given outer partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given outer partition.
+ */
+static bool
+process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index,
+						int *merged_index)
+{
+	Assert(outer_index >= 0);
+
+	/*
+	 * If the inner side has the default partition, the outer partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an outer join, in which case the outer partition has to be
+	 * scanned all the way anyway; if the outer partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the outer partition with a dummy partition.
+	 */
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0);
+
+		/*
+		 * If the outer side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (outer_has_default)
+			return false;
+
+		*merged_index = map_and_merge_partitions(outer_map, inner_map,
+												 outer_index, inner_default,
+												 next_index);
+		if (*merged_index == -1)
+			return false;
+
+		/*
+		 * If this is a FULL join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (*default_index == -1)
+				*default_index = *merged_index;
+			else
+				Assert(*merged_index == *default_index);
+			/* Don't add this index to the list of merged indexes. */
+			*merged_index = -1;
+		}
+	}
+	else
+	{
+		Assert(IS_OUTER_JOIN(jointype));
+		Assert(jointype != JOIN_RIGHT);
+
+		if (outer_map->merged_indexes[outer_index] >= 0)
+			*merged_index = outer_map->merged_indexes[outer_index];
+		else
+			*merged_index = merge_partition_with_dummy(outer_map, outer_index,
+													   next_index);
+	}
+	return true;
+}
+
+/*
+ * process_inner_partition
+ *
+ * Determine the merged partition associated with the given inner partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given inner partition.
+ */
+static bool
+process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index,
+						int *merged_index)
+{
+	Assert(inner_index >= 0);
+
+	/*
+	 * If the outer side has the default partition, the inner partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an FULL join, in which case the inner partition has to be
+	 * scanned all the way anyway; if the inner partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the inner partition with a dummy partition.
+	 */
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0);
+
+		/*
+		 * If the inner side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (inner_has_default)
+			return false;
+
+		*merged_index = map_and_merge_partitions(outer_map, inner_map,
+												 outer_default, inner_index,
+												 next_index);
+		if (*merged_index == -1)
+			return false;
+
+		/*
+		 * If this is an outer join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			if (*default_index == -1)
+				*default_index = *merged_index;
+			else
+				Assert(*merged_index == *default_index);
+			/* Don't add this index to the list of merged indexes. */
+			*merged_index = -1;
+		}
+	}
+	else
+	{
+		Assert(jointype == JOIN_FULL);
+
+		if (inner_map->merged_indexes[inner_index] >= 0)
+			*merged_index = inner_map->merged_indexes[inner_index];
+		else
+			*merged_index = merge_partition_with_dummy(inner_map, inner_index,
+													   next_index);
+	}
+	return true;
+}
+
+/*
+ * fix_merged_indexes
+ */
+static void
+fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes)
+{
+	int		   *new_indexes;
+	int			merged_index;
+	int			i;
+	ListCell   *lc;
+
+	new_indexes = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		new_indexes[i] = -1;
+
+	/* Build the mapping of old merged indexes to new merged indexes. */
+	if (outer_map->did_remapping)
+	{
+		for (i = 0; i < outer_map->nparts; i++)
+		{
+			merged_index = outer_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = outer_map->merged_indexes[i];
+		}
+	}
+	if (inner_map->did_remapping)
+	{
+		for (i = 0; i < inner_map->nparts; i++)
+		{
+			merged_index = inner_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = inner_map->merged_indexes[i];
+		}
+	}
+
+	/* Fix the merged_indexes list using the mapping. */
+	foreach(lc, merged_indexes)
+	{
+		merged_index = lfirst_int(lc);
+		Assert(merged_index >= 0);
+		if (new_indexes[merged_index] >= 0)
+			lfirst_int(lc) = new_indexes[merged_index];
+	}
+
+	pfree(new_indexes);
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * This function produces the list pairs of partitions which produce merged
+ * partitions in the order of merged partition indexes.
+ *
+ * nmerged is the number of merged partitions.
+ *
+ * If successful, the list pairs are returned as two separate lists,
+ * outer_part_list and inner_part_list, one for each side.  Otherwise, those
+ * lists will be set to NIL.
+ */
+static void
+generate_matching_part_pairs(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 PartitionMap *outer_map, PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_part_list, List **inner_part_list)
+{
+	int			outer_nparts = outer_map->nparts;
+	int			inner_nparts = inner_map->nparts;
+	int		   *outer_part_array;
+	int		   *inner_part_array;
+	int			max_nparts;
+	int 		i;
+
+	Assert(outer_nparts == outer_rel->nparts);
+	Assert(inner_nparts == inner_rel->nparts);
+
+	outer_part_array = (int *) palloc(sizeof(int) * nmerged);
+	inner_part_array = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		outer_part_array[i] = inner_part_array[i] = -1;
+
+	/* Set pairs of matching partitions. */
+	max_nparts = Max(outer_nparts, inner_nparts);
+	for (i = 0; i < max_nparts; i++)
+	{
+		if (i < outer_nparts)
+		{
+			int 		merged_index = outer_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				outer_part_array[merged_index] = i;
+			}
+		}
+		if (i < inner_nparts)
+		{
+			int 		merged_index = inner_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				inner_part_array[merged_index] = i;
+			}
+		}
+	}
+
+	*outer_part_list = NIL;
+	*inner_part_list = NIL;
+
+	/* Build the list pairs. */
+	for (i = 0; i < nmerged; i++)
+	{
+		int			outer_part = outer_part_array[i];
+		int			inner_part = inner_part_array[i];
+
+		/*
+		 * If both partitions are dummy, it means the merged partition was
+		 * removed by partition re-merging in map_and_merge_partitions();
+		 * ignore the merged partition
+		 */
+		if (outer_part == -1 && inner_part == -1)
+			continue;
+
+		*outer_part_list = lappend(*outer_part_list, outer_part >= 0 ?
+								   outer_rel->part_rels[outer_part] : NULL);
+		*inner_part_list = lappend(*inner_part_list, inner_part >= 0 ?
+								   inner_rel->part_rels[inner_part] : NULL);
+	}
+
+	pfree(outer_part_array);
+	pfree(inner_part_array);
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * Merge default partitions from both sides, if any, and assign the default
+ * partition for the join result, if necessary.
+ *
+ * If both the relations have default partitions, try mapping those to each
+ * other. If the mapping succeeds corresponding merged partition will act as
+ * the default partition of the join result.
+ *
+ * If inner side of the join has default but not the outer side, rows in it
+ * won't appear in the join result. So don't create a default partition. If
+ * outer side of the join has default but not the inner side, rows in it will
+ * appear in the join result, so create a default merged partition.
+ */
+static bool
+merge_default_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 bool outer_has_default, int outer_default,
+						 bool inner_has_default, int inner_default,
+						 JoinType jointype, int *next_index,
+						 int *default_index)
+{
+	Assert(outer_has_default || inner_has_default);
+
+	if (outer_has_default && !inner_has_default)
+	{
+		if (IS_OUTER_JOIN(jointype))
+		{
+			int			merged_index;
+
+			Assert(jointype != JOIN_RIGHT);
+			Assert(outer_default >= 0 && outer_default < outer_map->nparts);
+			merged_index = outer_map->merged_indexes[outer_default];
+			if (merged_index == -1)
+			{
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(outer_map,
+															outer_default,
+															next_index);
+			}
+			else
+				Assert(*default_index == merged_index);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else if (!outer_has_default && inner_has_default)
+	{
+		if (jointype == JOIN_FULL)
+		{
+			int			merged_index;
+
+			Assert(inner_default >= 0 && inner_default < inner_map->nparts);
+			merged_index = inner_map->merged_indexes[inner_default];
+			if (merged_index == -1)
+			{
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(inner_map,
+															inner_default,
+															next_index);
+			}
+			else
+				Assert(*default_index == merged_index);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		*default_index = map_and_merge_partitions(outer_map,
+												  inner_map,
+												  outer_default,
+												  inner_default,
+												  next_index);
+		if (*default_index == -1)
+			return false;
+	}
+
+	return true;
+}
+
+/*
+ * merge_null_partitions
+ *
+ * Merge NULL partitions, i.e. a partition that can hold NULL values for a list
+ * partitioned table, if any. Find the index of merged partition to which the
+ * NULL values would belong in the join result. If one joining relation has a
+ * NULL partition but not the other, try matching it with the default partition
+ * from the other relation since the default partition may have rows with NULL
+ * partition key. We can eliminate a NULL partition when it appears only on the
+ * inner side of the join and the outer side doesn't have a default partition.
+ *
+ * When the equality operator used for join is strict, two NULL values will not
+ * be considered as equal, and thus a NULL partition can be eliminated for an
+ * inner join. But we don't check the strictness operator here.
+ */
+static bool
+merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_map, PartitionMap *inner_map,
+					  bool outer_has_default, bool inner_has_default,
+					  bool outer_has_null, bool inner_has_null,
+					  JoinType jointype, int *next_index,
+					  int *default_index, int *null_index)
+{
+	Assert(outer_has_null || inner_has_null);
+	Assert(*null_index == -1);
+
+	if (outer_has_null && !inner_has_null)
+	{
+		int			merged_index = -1;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join to which the NULL partition matches will
+		 * contain the NULL values and thus become the NULL partition of the
+		 * the join.
+		 */
+		if (inner_has_default || IS_OUTER_JOIN(jointype))
+		{
+			if (!process_outer_partition(outer_map,
+										 inner_map,
+										 outer_has_default,
+										 inner_has_default,
+										 outer_bi->null_index,
+										 inner_bi->default_index,
+										 jointype,
+										 next_index,
+										 default_index,
+										 &merged_index))
+				return false;
+		}
+		*null_index = merged_index;
+	}
+	else if (!outer_has_null && inner_has_null)
+	{
+		int			merged_index = -1;
+
+		/*
+		 * If the NULL partition was missing from the outer side of the join,
+		 * the partition of the join to which the NULL partition matches will
+		 * contain the NULL values and thus become the NULL partition of the
+		 * the join.
+		 */
+		if (outer_has_default || jointype == JOIN_FULL)
+		{
+			if (!process_inner_partition(outer_map,
+										 inner_map,
+										 outer_has_default,
+										 inner_has_default,
+										 inner_bi->null_index,
+										 outer_bi->default_index,
+										 jointype,
+										 next_index,
+										 default_index,
+										 &merged_index))
+				return false;
+		}
+		*null_index = merged_index;
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		*null_index = map_and_merge_partitions(outer_map,
+											   inner_map,
+											   outer_bi->null_index,
+											   inner_bi->null_index,
+											   next_index);
+		if (*null_index == -1)
+			return false;
+	}
+
+	return true;
+}
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index 23a06d718e..e06eb9aaae 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -577,8 +577,10 @@ typedef struct PartitionSchemeData *PartitionScheme;
  *		part_scheme - Partitioning scheme of the relation
  *		nparts - Number of partitions
  *		boundinfo - Partition bounds
+ *		merged - true if partition bounds are merged ones
  *		partition_qual - Partition constraint if not the root
  *		part_rels - RelOptInfos for each partition
+ *		all_partrels - Relids set of all partition relids
  *		partexprs, nullable_partexprs - Partition key expressions
  *		partitioned_child_rels - RT indexes of unpruned partitions of
  *								 this relation that are partitioned tables
@@ -718,9 +720,12 @@ typedef struct RelOptInfo
 	PartitionScheme part_scheme;	/* Partitioning scheme. */
 	int			nparts;			/* number of partitions */
 	struct PartitionBoundInfoData *boundinfo;	/* Partition bounds */
+	bool		merged;			/* true if partition bounds were created by
+								 * partition_bounds_merge() */
 	List	   *partition_qual; /* partition constraint */
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of partitions,
 									 * stored in the same order of bounds */
+	Relids		all_partrels;	/* Relids set of all partition relids */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 	List	   *partitioned_child_rels; /* List of RT indexes. */
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index 0d0fd42b18..9292aa11e5 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -16,6 +16,7 @@
 #include "nodes/pg_list.h"
 #include "partitioning/partdefs.h"
 #include "utils/relcache.h"
+struct RelOptInfo;				/* avoid including pathnodes.h here */
 
 
 /*
@@ -108,5 +109,11 @@ extern int	partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 										  int nvalues, Datum *values, bool *is_equal);
 extern int	partition_hash_bsearch(PartitionBoundInfo boundinfo,
 								   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   struct RelOptInfo *outer_rel, struct RelOptInfo *inner_rel,
+					   JoinType jointype,
+					   List **outer_parts, List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 975bf6765c..1675abb1e7 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -8,59 +8,86 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-(4 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+(7 rows)
 
 -- left outer join, with whole-row reference; partitionwise join does not apply
 EXPLAIN (COSTS OFF)
@@ -72,35 +99,50 @@ SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER
    ->  Hash Right Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(22 rows)
 
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-      t1      |      t2      
---------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
- (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
- (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
- (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
- (550,0,0550) | 
-(12 rows)
+       t1       |       t2       
+----------------+----------------
+ (-250,0,-0250) | 
+ (-200,0,-0200) | 
+ (-150,0,-0150) | (0,-150,-0150)
+ (-100,0,-0100) | 
+ (-50,0,-0050)  | 
+ (0,0,0000)     | (0,0,0000)
+ (50,0,0050)    | 
+ (100,0,0100)   | 
+ (150,0,0150)   | (0,150,0150)
+ (200,0,0200)   | 
+ (250,0,0250)   | 
+ (300,0,0300)   | (0,300,0300)
+ (350,0,0350)   | 
+ (400,0,0400)   | 
+ (450,0,0450)   | (0,450,0450)
+ (500,0,0500)   | 
+ (550,0,0550)   | 
+ (600,0,0600)   | (0,600,0600)
+ (650,0,0650)   | 
+ (700,0,0700)   | 
+ (750,0,0750)   | (0,750,0750)
+(21 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
@@ -112,35 +154,53 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
    ->  Append
          ->  Hash Right Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Right Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t2_2.b)
-(20 rows)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+      |       | -225 | -0225
+      |       |  -75 | -0075
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(14 rows)
 
 -- full outer join, with placeholder vars
 EXPLAIN (COSTS OFF)
@@ -148,8 +208,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                             QUERY PLAN                            
 ------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b
+   Sort Key: prt1_p0.a, prt2_p0.b
    ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_p0.a = prt2_p0.b)
+               Filter: (((50) = prt1_p0.a) OR ((75) = prt2_p0.b))
+               ->  Seq Scan on prt1_p0
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0
+                           Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p1.a = prt2_p1.b)
                Filter: (((50) = prt1_p1.a) OR ((75) = prt2_p1.b))
@@ -174,7 +242,15 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
                ->  Hash
                      ->  Seq Scan on prt2_p3
                            Filter: (a = 0)
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = prt2_p4.b)
+               Filter: (((50) = prt1_p4.a) OR ((75) = prt2_p4.b))
+               ->  Seq Scan on prt1_p4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4
+                           Filter: (a = 0)
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
@@ -211,35 +287,44 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Hash Right Join
-         Hash Cond: (prt2_p2.b = prt1_p1.a)
+         Hash Cond: (prt2_p2.b = prt1_p0.a)
          ->  Append
                ->  Seq Scan on prt2_p2
                      Filter: (b > 250)
                ->  Seq Scan on prt2_p3
                      Filter: (b > 250)
+               ->  Seq Scan on prt2_p4
+                     Filter: (b > 250)
          ->  Hash
                ->  Append
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p1
                            Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p2
                            Filter: ((a < 450) AND (b = 0))
-(15 rows)
+(19 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-(9 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+(14 rows)
 
 -- Currently we can't do partitioned join if nullable-side partitions are pruned
 EXPLAIN (COSTS OFF)
@@ -247,11 +332,13 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                       QUERY PLAN                      
 ------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Hash Full Join
-         Hash Cond: (prt1_p1.a = prt2_p2.b)
-         Filter: ((prt1_p1.b = 0) OR (prt2_p2.a = 0))
+         Hash Cond: (prt1_p0.a = prt2_p2.b)
+         Filter: ((prt1_p0.b = 0) OR (prt2_p2.a = 0))
          ->  Append
+               ->  Seq Scan on prt1_p0
+                     Filter: (a < 450)
                ->  Seq Scan on prt1_p1
                      Filter: (a < 450)
                ->  Seq Scan on prt1_p2
@@ -262,64 +349,147 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                            Filter: (b > 250)
                      ->  Seq Scan on prt2_p3
                            Filter: (b > 250)
-(16 rows)
+                     ->  Seq Scan on prt2_p4
+                           Filter: (b > 250)
+(20 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
-(12 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+      |       | 375 | 0375
+      |       | 450 | 0450
+      |       | 525 | 0525
+      |       | 600 | 0600
+      |       | 675 | 0675
+      |       | 750 | 0750
+(20 rows)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Semi Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Semi Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                            Filter: (a = 0)
-         ->  Nested Loop Semi Join
-               Join Filter: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
-               ->  Materialize
-                     ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
-(24 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt1_p3 t2_3
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+                           Filter: (b = 0)
+(37 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
 
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
@@ -330,27 +500,82 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
    ->  Append
          ->  Hash Anti Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Hash Anti Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Hash Anti Join
                Hash Cond: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
-(17 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(27 rows)
 
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
-  sum  |         avg          | sum  |         avg         
--------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+  sum  |         avg          | sum  |        avg         
+-------+----------------------+------+--------------------
+ 95550 | 273.0000000000000000 | 2200 | 6.2857142857142857
 (1 row)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Append
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p0 t1
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+               Index Cond: (a = t1.b)
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.b = t2_1.a)
+         ->  Seq Scan on prt2_p1 t1_1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p1 t2_1
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p2 t1_2
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+               Index Cond: (a = t1_2.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p3 t1_3
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+               Index Cond: (a = t1_3.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p4 t1_4
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+               Index Cond: (a = t1_4.b)
+(27 rows)
+
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+  b   |   c   
+------+-------
+ -225 | -0225
+  -75 | -0075
+   75 | 0075
+  225 | 0225
+  375 | 0375
+  525 | 0525
+  675 | 0675
+(7 rows)
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -362,49 +587,74 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
+                     ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
                            Index Cond: (a = t1.a)
-                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t3
                            Index Cond: (b = t2.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_1
+                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
                            Index Cond: (a = t1_1.a)
-                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3_1
                            Index Cond: (b = t2_1.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_2
+                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
                            Index Cond: (a = t1_2.a)
-                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_2
                            Index Cond: (b = t2_2.a)
-(27 rows)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                           Index Cond: (a = t1_3.a)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_3
+                           Index Cond: (b = t2_3.a)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                           Index Cond: (a = t1_4.a)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t3_4
+                           Index Cond: (b = t2_4.a)
+(43 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   | t2a | t3a | least 
------+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
- 100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
- 250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
- 400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
- 550 | 0 | 0550 |     |     |      
-(12 rows)
+  a   | b |   c   | t2a  | t3a | least 
+------+---+-------+------+-----+-------
+ -250 | 0 | -0250 |      |     |      
+ -200 | 0 | -0200 |      |     |      
+ -150 | 0 | -0150 | -150 |   0 |  -150
+ -100 | 0 | -0100 |      |     |      
+  -50 | 0 | -0050 |      |     |      
+    0 | 0 | 0000  |    0 |   0 |     0
+   50 | 0 | 0050  |      |     |      
+  100 | 0 | 0100  |      |     |      
+  150 | 0 | 0150  |  150 |   0 |   150
+  200 | 0 | 0200  |      |     |      
+  250 | 0 | 0250  |      |     |      
+  300 | 0 | 0300  |  300 |   0 |   300
+  350 | 0 | 0350  |      |     |      
+  400 | 0 | 0400  |      |     |      
+  450 | 0 | 0450  |  450 |   0 |   450
+  500 | 0 | 0500  |      |     |      
+  550 | 0 | 0550  |      |     |      
+  600 | 0 | 0600  |  600 |   0 |   600
+  650 | 0 | 0650  |      |     |      
+  700 | 0 | 0700  |      |     |      
+  750 | 0 | 0750  |  750 |   0 |   750
+(21 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
@@ -418,46 +668,67 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
          Hash Cond: ((t1.c)::text = (t2.c)::text)
          Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Seq Scan on prt1_p1 t1
-               ->  Seq Scan on prt1_p2 t1_1
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
          ->  Hash
                ->  Append
                      ->  Hash Join
                            Hash Cond: (t2.a = t3.b)
-                           ->  Seq Scan on prt1_p1 t2
+                           ->  Seq Scan on prt1_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t3
+                                 ->  Seq Scan on prt2_p0 t3
                      ->  Hash Join
                            Hash Cond: (t2_1.a = t3_1.b)
-                           ->  Seq Scan on prt1_p2 t2_1
+                           ->  Seq Scan on prt1_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t3_1
+                                 ->  Seq Scan on prt2_p1 t3_1
                      ->  Hash Join
                            Hash Cond: (t2_2.a = t3_2.b)
-                           ->  Seq Scan on prt1_p3 t2_2
+                           ->  Seq Scan on prt1_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p3 t3_2
-(26 rows)
+                                 ->  Seq Scan on prt2_p2 t3_2
+                     ->  Hash Join
+                           Hash Cond: (t2_3.a = t3_3.b)
+                           ->  Seq Scan on prt1_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t3_3
+                     ->  Hash Join
+                           Hash Cond: (t2_4.a = t3_4.b)
+                           ->  Seq Scan on prt1_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t3_4
+(38 rows)
 
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, t2.b t2b, t2.c t2c, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
-  a  | t2a | t2c  
------+-----+------
-   0 |   0 | 0000
-  50 |     | 
- 100 |     | 
- 150 | 150 | 0150
- 200 |     | 
- 250 |     | 
- 300 | 300 | 0300
- 350 |     | 
- 400 |     | 
- 450 | 450 | 0450
- 500 |     | 
- 550 |     | 
-(12 rows)
+  a   | t2a  |  t2c  
+------+------+-------
+ -250 |      | 
+ -200 |      | 
+ -150 | -150 | -0150
+ -100 |      | 
+  -50 |      | 
+    0 |    0 | 0000
+   50 |      | 
+  100 |      | 
+  150 |  150 | 0150
+  200 |      | 
+  250 |      | 
+  300 |  300 | 0300
+  350 |      | 
+  400 |      | 
+  450 |  450 | 0450
+  500 |      | 
+  550 |      | 
+  600 |  600 | 0600
+  650 |      | 
+  700 |      | 
+  750 |  750 | 0750
+(21 rows)
 
 -- bug with inadequate sort key representation
 SET enable_partitionwise_aggregate TO true;
@@ -469,49 +740,75 @@ SELECT a, b FROM prt1 FULL JOIN prt2 p2(b,a,c) USING(a,b)
                                                     QUERY PLAN                                                     
 -------------------------------------------------------------------------------------------------------------------
  Group
-   Group Key: (COALESCE(prt1_p1.a, p2.a)), (COALESCE(prt1_p1.b, p2.b))
+   Group Key: (COALESCE(prt1_p0.a, p2.a)), (COALESCE(prt1_p0.b, p2.b))
    ->  Merge Append
-         Sort Key: (COALESCE(prt1_p1.a, p2.a)), (COALESCE(prt1_p1.b, p2.b))
+         Sort Key: (COALESCE(prt1_p0.a, p2.a)), (COALESCE(prt1_p0.b, p2.b))
+         ->  Group
+               Group Key: (COALESCE(prt1_p0.a, p2.a)), (COALESCE(prt1_p0.b, p2.b))
+               ->  Sort
+                     Sort Key: (COALESCE(prt1_p0.a, p2.a)), (COALESCE(prt1_p0.b, p2.b))
+                     ->  Merge Full Join
+                           Merge Cond: ((prt1_p0.a = p2.a) AND (prt1_p0.b = p2.b))
+                           Filter: ((COALESCE(prt1_p0.a, p2.a) >= 490) AND (COALESCE(prt1_p0.a, p2.a) <= 510))
+                           ->  Sort
+                                 Sort Key: prt1_p0.a, prt1_p0.b
+                                 ->  Seq Scan on prt1_p0
+                           ->  Sort
+                                 Sort Key: p2.a, p2.b
+                                 ->  Seq Scan on prt2_p0 p2
          ->  Group
-               Group Key: (COALESCE(prt1_p1.a, p2.a)), (COALESCE(prt1_p1.b, p2.b))
+               Group Key: (COALESCE(prt1_p1.a, p2_1.a)), (COALESCE(prt1_p1.b, p2_1.b))
                ->  Sort
-                     Sort Key: (COALESCE(prt1_p1.a, p2.a)), (COALESCE(prt1_p1.b, p2.b))
+                     Sort Key: (COALESCE(prt1_p1.a, p2_1.a)), (COALESCE(prt1_p1.b, p2_1.b))
                      ->  Merge Full Join
-                           Merge Cond: ((prt1_p1.a = p2.a) AND (prt1_p1.b = p2.b))
-                           Filter: ((COALESCE(prt1_p1.a, p2.a) >= 490) AND (COALESCE(prt1_p1.a, p2.a) <= 510))
+                           Merge Cond: ((prt1_p1.a = p2_1.a) AND (prt1_p1.b = p2_1.b))
+                           Filter: ((COALESCE(prt1_p1.a, p2_1.a) >= 490) AND (COALESCE(prt1_p1.a, p2_1.a) <= 510))
                            ->  Sort
                                  Sort Key: prt1_p1.a, prt1_p1.b
                                  ->  Seq Scan on prt1_p1
                            ->  Sort
-                                 Sort Key: p2.a, p2.b
-                                 ->  Seq Scan on prt2_p1 p2
+                                 Sort Key: p2_1.a, p2_1.b
+                                 ->  Seq Scan on prt2_p1 p2_1
          ->  Group
-               Group Key: (COALESCE(prt1_p2.a, p2_1.a)), (COALESCE(prt1_p2.b, p2_1.b))
+               Group Key: (COALESCE(prt1_p2.a, p2_2.a)), (COALESCE(prt1_p2.b, p2_2.b))
                ->  Sort
-                     Sort Key: (COALESCE(prt1_p2.a, p2_1.a)), (COALESCE(prt1_p2.b, p2_1.b))
+                     Sort Key: (COALESCE(prt1_p2.a, p2_2.a)), (COALESCE(prt1_p2.b, p2_2.b))
                      ->  Merge Full Join
-                           Merge Cond: ((prt1_p2.a = p2_1.a) AND (prt1_p2.b = p2_1.b))
-                           Filter: ((COALESCE(prt1_p2.a, p2_1.a) >= 490) AND (COALESCE(prt1_p2.a, p2_1.a) <= 510))
+                           Merge Cond: ((prt1_p2.a = p2_2.a) AND (prt1_p2.b = p2_2.b))
+                           Filter: ((COALESCE(prt1_p2.a, p2_2.a) >= 490) AND (COALESCE(prt1_p2.a, p2_2.a) <= 510))
                            ->  Sort
                                  Sort Key: prt1_p2.a, prt1_p2.b
                                  ->  Seq Scan on prt1_p2
                            ->  Sort
-                                 Sort Key: p2_1.a, p2_1.b
-                                 ->  Seq Scan on prt2_p2 p2_1
+                                 Sort Key: p2_2.a, p2_2.b
+                                 ->  Seq Scan on prt2_p2 p2_2
          ->  Group
-               Group Key: (COALESCE(prt1_p3.a, p2_2.a)), (COALESCE(prt1_p3.b, p2_2.b))
+               Group Key: (COALESCE(prt1_p3.a, p2_3.a)), (COALESCE(prt1_p3.b, p2_3.b))
                ->  Sort
-                     Sort Key: (COALESCE(prt1_p3.a, p2_2.a)), (COALESCE(prt1_p3.b, p2_2.b))
+                     Sort Key: (COALESCE(prt1_p3.a, p2_3.a)), (COALESCE(prt1_p3.b, p2_3.b))
                      ->  Merge Full Join
-                           Merge Cond: ((prt1_p3.a = p2_2.a) AND (prt1_p3.b = p2_2.b))
-                           Filter: ((COALESCE(prt1_p3.a, p2_2.a) >= 490) AND (COALESCE(prt1_p3.a, p2_2.a) <= 510))
+                           Merge Cond: ((prt1_p3.a = p2_3.a) AND (prt1_p3.b = p2_3.b))
+                           Filter: ((COALESCE(prt1_p3.a, p2_3.a) >= 490) AND (COALESCE(prt1_p3.a, p2_3.a) <= 510))
                            ->  Sort
                                  Sort Key: prt1_p3.a, prt1_p3.b
                                  ->  Seq Scan on prt1_p3
                            ->  Sort
-                                 Sort Key: p2_2.a, p2_2.b
-                                 ->  Seq Scan on prt2_p3 p2_2
-(43 rows)
+                                 Sort Key: p2_3.a, p2_3.b
+                                 ->  Seq Scan on prt2_p3 p2_3
+         ->  Group
+               Group Key: (COALESCE(prt1_p4.a, p2_4.a)), (COALESCE(prt1_p4.b, p2_4.b))
+               ->  Sort
+                     Sort Key: (COALESCE(prt1_p4.a, p2_4.a)), (COALESCE(prt1_p4.b, p2_4.b))
+                     ->  Merge Full Join
+                           Merge Cond: ((prt1_p4.a = p2_4.a) AND (prt1_p4.b = p2_4.b))
+                           Filter: ((COALESCE(prt1_p4.a, p2_4.a) >= 490) AND (COALESCE(prt1_p4.a, p2_4.a) <= 510))
+                           ->  Sort
+                                 Sort Key: prt1_p4.a, prt1_p4.b
+                                 ->  Seq Scan on prt1_p4
+                           ->  Sort
+                                 Sort Key: p2_4.a, p2_4.b
+                                 ->  Seq Scan on prt2_p4 p2_4
+(69 rows)
 
 SELECT a, b FROM prt1 FULL JOIN prt2 p2(b,a,c) USING(a,b)
   WHERE a BETWEEN 490 AND 510
@@ -540,19 +837,29 @@ RESET enable_hashjoin;
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
@@ -563,32 +870,49 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
    ->  Append
          ->  Hash Join
                Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
-               ->  Seq Scan on prt2_e_p1 t2
+               ->  Seq Scan on prt2_e_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 t1
+                     ->  Seq Scan on prt1_e_p0 t1
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
-               ->  Seq Scan on prt2_e_p2 t2_1
+               Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+               ->  Seq Scan on prt1_e_p1 t1_1
+                     Filter: (c = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 t1_1
-                           Filter: (c = 0)
+                     ->  Seq Scan on prt2_e_p1 t2_1
          ->  Hash Join
                Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
-               ->  Seq Scan on prt2_e_p3 t2_2
+               ->  Seq Scan on prt2_e_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 t1_2
+                     ->  Seq Scan on prt1_e_p2 t1_2
                            Filter: (c = 0)
-(21 rows)
+         ->  Hash Join
+               Hash Cond: (((t2_3.b + t2_3.a) / 2) = ((t1_3.a + t1_3.b) / 2))
+               ->  Seq Scan on prt2_e_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p3 t1_3
+                           Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (((t2_4.b + t2_4.a) / 2) = ((t1_4.a + t1_4.b) / 2))
+               ->  Seq Scan on prt2_e_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4 t1_4
+                           Filter: (c = 0)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
- 150 | 0 | 150 | 0
- 300 | 0 | 300 | 0
- 450 | 0 | 450 | 0
-(4 rows)
+  a   | c |  b   | c 
+------+---+------+---
+ -250 | 0 | -250 | 0
+ -100 | 0 | -100 | 0
+    0 | 0 |    0 | 0
+    0 | 0 |    0 | 0
+  150 | 0 |  150 | 0
+  300 | 0 |  300 | 0
+  450 | 0 |  450 | 0
+  600 | 0 |  600 | 0
+  750 | 0 |  750 | 0
+(9 rows)
 
 --
 -- N-way join
@@ -601,154 +925,232 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = ((t3.a + t3.b) / 2))
+               Join Filter: (t1.a = t2.b)
                ->  Hash Join
-                     Hash Cond: (t2.b = t1.a)
-                     ->  Seq Scan on prt2_p1 t2
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on prt1_e_p0 t3
                      ->  Hash
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                     Index Cond: (((a + b) / 2) = t2.b)
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
+                     Index Cond: (b = ((t3.a + t3.b) / 2))
          ->  Nested Loop
                Join Filter: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_1.b = t1_1.a)
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
+               ->  Index Scan using iprt1_e_p4_ab2 on prt1_e_p1 t3_1
                      Index Cond: (((a + b) / 2) = t2_1.b)
          ->  Nested Loop
                Join Filter: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_2.b = t1_2.a)
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
+               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_2
                      Index Cond: (((a + b) / 2) = t2_2.b)
-(33 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = ((t3_3.a + t3_3.b) / 2))
+               ->  Nested Loop
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t2_3.b)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t2_4.b)
+               ->  Hash Join
+                     Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                     ->  Seq Scan on prt1_e_p4 t3_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_p4 t1_4
+                                 Filter: (b = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = ((t3_4.a + t3_4.b) / 2))
+(52 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
-(4 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(8 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                          QUERY PLAN                          
---------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Hash Right Join
                Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
-               ->  Seq Scan on prt1_e_p1 t3
+               ->  Seq Scan on prt1_e_p0 t3
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2.b = t1.a)
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_1.a)
-               ->  Seq Scan on prt1_e_p2 t3_1
+               ->  Seq Scan on prt1_e_p1 t3_1
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_1.b = t1_1.a)
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_2.a)
-               ->  Seq Scan on prt1_e_p3 t3_2
+               ->  Seq Scan on prt1_e_p2 t3_2
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_2.b = t1_2.a)
-                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                                        Filter: (b = 0)
-(33 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+               ->  Seq Scan on prt1_e_p4 t3_4
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_4.b = t1_4.a)
+                           ->  Seq Scan on prt2_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_p4 t1_4
+                                       Filter: (b = 0)
+(51 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                            QUERY PLAN                             
--------------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p1 t3
+                           ->  Seq Scan on prt1_e_p0 t3
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p1_b on prt2_p1 t2
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
                      Index Cond: (b = t1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p2 t3_1
+                           ->  Seq Scan on prt1_e_p1 t3_1
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
+               ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
                      Index Cond: (b = t1_1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p3 t3_2
+                           ->  Seq Scan on prt1_e_p2 t3_2
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
                      Index Cond: (b = t1_2.a)
-(30 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_e_p3 t3_3
+                           Filter: (c = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                           Index Cond: (a = ((t3_3.a + t3_3.b) / 2))
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Nested Loop Left Join
+               ->  Hash Right Join
+                     Hash Cond: (t1_4.a = ((t3_4.a + t3_4.b) / 2))
+                     ->  Seq Scan on prt1_p4 t1_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p4 t3_4
+                                 Filter: (c = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = t1_4.a)
+(47 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -757,21 +1159,34 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                                                    QUERY PLAN                                                   
 ----------------------------------------------------------------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   Sort Key: prt1_p0.a, prt2_p0.b, ((prt1_e_p0.a + prt1_e_p0.b))
    ->  Append
          ->  Hash Full Join
-               Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
-               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               Hash Cond: (prt1_p0.a = ((prt1_e_p0.a + prt1_e_p0.b) / 2))
+               Filter: ((prt1_p0.a = (50)) OR (prt2_p0.b = (75)) OR (((prt1_e_p0.a + prt1_e_p0.b) / 2) = (50)))
                ->  Hash Full Join
-                     Hash Cond: (prt1_p1.a = prt2_p1.b)
-                     ->  Seq Scan on prt1_p1
+                     Hash Cond: (prt1_p0.a = prt2_p0.b)
+                     ->  Seq Scan on prt1_p0
                            Filter: (b = 0)
                      ->  Hash
-                           ->  Seq Scan on prt2_p1
+                           ->  Seq Scan on prt2_p0
                                  Filter: (a = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1
+                     ->  Seq Scan on prt1_e_p0
                            Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = prt1_p1.a)
+               Filter: ((prt1_p1.a = (50)) OR (prt2_p1.b = (75)) OR (((prt1_e_p1.a + prt1_e_p1.b) / 2) = (50)))
+               ->  Seq Scan on prt1_e_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on prt1_p1
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1
+                                       Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
                Filter: ((prt1_p2.a = (50)) OR (prt2_p2.b = (75)) OR (((prt1_e_p2.a + prt1_e_p2.b) / 2) = (50)))
@@ -798,7 +1213,20 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                ->  Hash
                      ->  Seq Scan on prt1_e_p3
                            Filter: (c = 0)
-(42 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_p4.a = ((prt1_e_p4.a + prt1_e_p4.b) / 2))
+               Filter: ((prt1_p4.a = (50)) OR (prt2_p4.b = (75)) OR (((prt1_e_p4.a + prt1_e_p4.b) / 2) = (50)))
+               ->  Hash Full Join
+                     Hash Cond: (prt1_p4.a = prt2_p4.b)
+                     ->  Seq Scan on prt1_p4
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p4
+                                 Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p4
+                           Filter: (c = 0)
+(68 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
@@ -816,172 +1244,260 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = t1_3.b)
+               Join Filter: (t1.a = t1_5.b)
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Join
-                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
-                           ->  Seq Scan on prt1_e_p1 t2
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t1_3
+                                 ->  Seq Scan on prt2_p0 t1_5
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
                      Index Cond: (a = ((t2.a + t2.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_1.a = t1_4.b)
+               Join Filter: (t1_1.a = t1_6.b)
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Join
-                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
-                           ->  Seq Scan on prt1_e_p2 t2_1
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_6.b)
+                           ->  Seq Scan on prt1_e_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t1_4
+                                 ->  Seq Scan on prt2_p1 t1_6
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
                      Index Cond: (a = ((t2_1.a + t2_1.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_2.a = t1_5.b)
+               Join Filter: (t1_2.a = t1_7.b)
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
+                           ->  Seq Scan on prt2_p2 t1_7
                                  Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
+                           ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t2_2
+                                 Index Cond: (((a + b) / 2) = t1_7.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t1_8.b)
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Nested Loop
+                           ->  Seq Scan on prt2_p3 t1_8
+                                 Filter: (a = 0)
+                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_3
+                                 Index Cond: (((a + b) / 2) = t1_8.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = ((t2_3.a + t2_3.b) / 2))
+                     Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t1_9.b)
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Join
+                           Hash Cond: (((t2_4.a + t2_4.b) / 2) = t1_9.b)
+                           ->  Seq Scan on prt1_e_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t1_9
+                                       Filter: (a = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = ((t2_4.a + t2_4.b) / 2))
+                     Filter: (b = 0)
+(66 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                               QUERY PLAN                                
--------------------------------------------------------------------------
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
-                           ->  Seq Scan on prt2_p1 t1_3
+                           Hash Cond: (t1_5.b = ((t1_10.a + t1_10.b) / 2))
+                           ->  Seq Scan on prt2_p0 t1_5
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t1_6
+                                 ->  Seq Scan on prt1_e_p0 t1_10
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
-                     Index Cond: (a = t1_3.b)
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t1_5.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
-                           ->  Seq Scan on prt2_p2 t1_4
+                           Hash Cond: (t1_6.b = ((t1_11.a + t1_11.b) / 2))
+                           ->  Seq Scan on prt2_p1 t1_6
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t1_7
+                                 ->  Seq Scan on prt1_e_p1 t1_11
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
-                     Index Cond: (a = t1_4.b)
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
+                     Index Cond: (a = t1_6.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
-                           ->  Seq Scan on prt2_p3 t1_5
+                           Hash Cond: (t1_7.b = ((t1_12.a + t1_12.b) / 2))
+                           ->  Seq Scan on prt2_p2 t1_7
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p3 t1_8
+                                 ->  Seq Scan on prt1_e_p2 t1_12
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t1_5.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t1_7.b)
                      Filter: (b = 0)
-(39 rows)
-
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_8.b = ((t1_13.a + t1_13.b) / 2))
+                           ->  Seq Scan on prt2_p3 t1_8
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p3 t1_13
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t1_8.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_9.b = ((t1_14.a + t1_14.b) / 2))
+                           ->  Seq Scan on prt2_p4 t1_9
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t1_14
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = t1_9.b)
+                     Filter: (b = 0)
+(63 rows)
+
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 -- test merge joins
 SET enable_hashjoin TO off;
 SET enable_nestloop TO off;
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                           QUERY PLAN                           
-----------------------------------------------------------------
+                            QUERY PLAN                            
+------------------------------------------------------------------
  Merge Append
    Sort Key: t1.a
    ->  Merge Semi Join
-         Merge Cond: (t1.a = t1_3.b)
+         Merge Cond: (t1.a = t1_5.b)
          ->  Sort
                Sort Key: t1.a
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+               Merge Cond: (t1_5.b = (((t1_10.a + t1_10.b) / 2)))
                ->  Sort
-                     Sort Key: t1_3.b
-                     ->  Seq Scan on prt2_p1 t1_3
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on prt2_p0 t1_5
                ->  Sort
-                     Sort Key: (((t1_6.a + t1_6.b) / 2))
-                     ->  Seq Scan on prt1_e_p1 t1_6
+                     Sort Key: (((t1_10.a + t1_10.b) / 2))
+                     ->  Seq Scan on prt1_e_p0 t1_10
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_1.a = t1_4.b)
+         Merge Cond: (t1_1.a = t1_6.b)
          ->  Sort
                Sort Key: t1_1.a
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+               Merge Cond: (t1_6.b = (((t1_11.a + t1_11.b) / 2)))
                ->  Sort
-                     Sort Key: t1_4.b
-                     ->  Seq Scan on prt2_p2 t1_4
+                     Sort Key: t1_6.b
+                     ->  Seq Scan on prt2_p1 t1_6
                ->  Sort
-                     Sort Key: (((t1_7.a + t1_7.b) / 2))
-                     ->  Seq Scan on prt1_e_p2 t1_7
+                     Sort Key: (((t1_11.a + t1_11.b) / 2))
+                     ->  Seq Scan on prt1_e_p1 t1_11
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_2.a = t1_5.b)
+         Merge Cond: (t1_2.a = t1_7.b)
          ->  Sort
                Sort Key: t1_2.a
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+               Merge Cond: (t1_7.b = (((t1_12.a + t1_12.b) / 2)))
                ->  Sort
-                     Sort Key: t1_5.b
-                     ->  Seq Scan on prt2_p3 t1_5
+                     Sort Key: t1_7.b
+                     ->  Seq Scan on prt2_p2 t1_7
                ->  Sort
-                     Sort Key: (((t1_8.a + t1_8.b) / 2))
-                     ->  Seq Scan on prt1_e_p3 t1_8
+                     Sort Key: (((t1_12.a + t1_12.b) / 2))
+                     ->  Seq Scan on prt1_e_p2 t1_12
                            Filter: (c = 0)
-(47 rows)
+   ->  Merge Semi Join
+         Merge Cond: (t1_3.a = t1_8.b)
+         ->  Sort
+               Sort Key: t1_3.a
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_8.b = (((t1_13.a + t1_13.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_8.b
+                     ->  Seq Scan on prt2_p3 t1_8
+               ->  Sort
+                     Sort Key: (((t1_13.a + t1_13.b) / 2))
+                     ->  Seq Scan on prt1_e_p3 t1_13
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_4.a = t1_9.b)
+         ->  Sort
+               Sort Key: t1_4.a
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_9.b = (((t1_14.a + t1_14.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_9.b
+                     ->  Seq Scan on prt2_p4 t1_9
+               ->  Sort
+                     Sort Key: (((t1_14.a + t1_14.b) / 2))
+                     ->  Seq Scan on prt1_e_p4 t1_14
+                           Filter: (c = 0)
+(77 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
@@ -998,14 +1514,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
                            ->  Sort
                                  Sort Key: (((t3.a + t3.b) / 2))
-                                 ->  Seq Scan on prt1_e_p1 t3
+                                 ->  Seq Scan on prt1_e_p0 t3
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1.a
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                ->  Sort
                      Sort Key: t2.b
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Merge Left Join
                Merge Cond: (t1_1.a = t2_1.b)
                ->  Sort
@@ -1014,14 +1530,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
                            ->  Sort
                                  Sort Key: (((t3_1.a + t3_1.b) / 2))
-                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                 ->  Seq Scan on prt1_e_p1 t3_1
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_1.a
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                ->  Sort
                      Sort Key: t2_1.b
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Merge Left Join
                Merge Cond: (t1_2.a = t2_2.b)
                ->  Sort
@@ -1030,32 +1546,74 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
                            ->  Sort
                                  Sort Key: (((t3_2.a + t3_2.b) / 2))
-                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                 ->  Seq Scan on prt1_e_p2 t3_2
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_2.a
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                ->  Sort
                      Sort Key: t2_2.b
-                     ->  Seq Scan on prt2_p3 t2_2
-(51 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Merge Left Join
+               Merge Cond: (t1_3.a = t2_3.b)
+               ->  Sort
+                     Sort Key: t1_3.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_3.a + t3_3.b) / 2)) = t1_3.a)
+                           ->  Sort
+                                 Sort Key: (((t3_3.a + t3_3.b) / 2))
+                                 ->  Seq Scan on prt1_e_p3 t3_3
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_3.a
+                                 ->  Seq Scan on prt1_p3 t1_3
+               ->  Sort
+                     Sort Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Merge Left Join
+               Merge Cond: (t1_4.a = t2_4.b)
+               ->  Sort
+                     Sort Key: t1_4.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_4.a + t3_4.b) / 2)) = t1_4.a)
+                           ->  Sort
+                                 Sort Key: (((t3_4.a + t3_4.b) / 2))
+                                 ->  Seq Scan on prt1_e_p4 t3_4
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_4.a
+                                 ->  Seq Scan on prt1_p4 t1_4
+               ->  Sort
+                     Sort Key: t2_4.b
+                     ->  Seq Scan on prt2_p4 t2_4
+(83 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- MergeAppend on nullable column
 -- This should generate a partitionwise join, but currently fails to
@@ -1064,12 +1622,14 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                         QUERY PLAN                         
 -----------------------------------------------------------
  Sort
-   Sort Key: prt1_p1.a, prt2_p2.b
+   Sort Key: prt1_p0.a, prt2_p2.b
    ->  Merge Left Join
-         Merge Cond: (prt1_p1.a = prt2_p2.b)
+         Merge Cond: (prt1_p0.a = prt2_p2.b)
          ->  Sort
-               Sort Key: prt1_p1.a
+               Sort Key: prt1_p0.a
                ->  Append
+                     ->  Seq Scan on prt1_p0
+                           Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p1
                            Filter: ((a < 450) AND (b = 0))
                      ->  Seq Scan on prt1_p2
@@ -1081,21 +1641,28 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                            Filter: (b > 250)
                      ->  Seq Scan on prt2_p3
                            Filter: (b > 250)
-(18 rows)
+                     ->  Seq Scan on prt2_p4
+                           Filter: (b > 250)
+(22 rows)
 
 SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  b  
------+-----
-   0 |    
-  50 |    
- 100 |    
- 150 |    
- 200 |    
- 250 |    
- 300 | 300
- 350 |    
- 400 |    
-(9 rows)
+  a   |  b  
+------+-----
+ -250 |    
+ -200 |    
+ -150 |    
+ -100 |    
+  -50 |    
+    0 |    
+   50 |    
+  100 |    
+  150 |    
+  200 |    
+  250 |    
+  300 | 300
+  350 |    
+  400 |    
+(14 rows)
 
 -- merge join when expression with whole-row reference needs to be sorted;
 -- partitionwise join does not apply
@@ -1109,175 +1676,2403 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
          Sort Key: t1.a, ((((t1.*)::prt1))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
    ->  Sort
          Sort Key: t2.b, ((((t2.*)::prt2))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt2_p1 t2
-                     ->  Seq Scan on prt2_p2 t2_1
-                     ->  Seq Scan on prt2_p3 t2_2
-(16 rows)
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+(20 rows)
 
 SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.b ORDER BY t1.a;
- a  | b  
-----+----
-  0 |  0
-  6 |  6
- 12 | 12
- 18 | 18
- 24 | 24
-(5 rows)
+  a  |  b  
+-----+-----
+ -24 | -24
+ -18 | -18
+ -12 | -12
+  -6 |  -6
+   0 |   0
+   6 |   6
+  12 |  12
+  18 |  18
+  24 |  24
+(9 rows)
 
 RESET enable_hashjoin;
 RESET enable_nestloop;
---
--- partitioned by multiple columns
---
-CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
-CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
-ANALYZE prt1_m;
-CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
-CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
-CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
-CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
-INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
-ANALYZE prt2_m;
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches prt2_p3 and
+-- prt2_p4 partition from prt2.
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-                                                             QUERY PLAN                                                             
-------------------------------------------------------------------------------------------------------------------------------------
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
  Sort
-   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_4
+                           Filter: (b = 0)
+(22 rows)
+
+-- partitionwise join should be possible when we drop prt2_p4 from prt2.
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
    ->  Append
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
-               ->  Seq Scan on prt1_m_p1
-                     Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt2_m_p1
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
-               ->  Seq Scan on prt1_m_p2
-                     Filter: (c = 0)
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt2_m_p2
-                           Filter: (c = 0)
-         ->  Hash Full Join
-               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
-               ->  Seq Scan on prt1_m_p3
-                     Filter: (c = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt2_m_p3
-                           Filter: (c = 0)
-(24 rows)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+(26 rows)
 
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
-  50 | 0 |     |  
- 100 | 0 |     |  
- 150 | 0 | 150 | 0
- 200 | 0 |     |  
- 250 | 0 |     |  
- 300 | 0 | 300 | 0
- 350 | 0 |     |  
- 400 | 0 |     |  
- 450 | 0 | 450 | 0
- 500 | 0 |     |  
- 550 | 0 |     |  
-     |   |  75 | 0
-     |   | 225 | 0
-     |   | 375 | 0
-     |   | 525 | 0
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ANALYZE prt2;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -150 | -0150 | 0 | -0150
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+(7 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 |   | 
+ -200 | -0200 |   | 
+ -150 | -0150 | 0 | -0150
+ -100 | -0100 |   | 
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+(21 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -200 | 0 | -0200
+ -100 | 0 | -0100
+  -50 | 0 | -0050
+   50 | 0 | 0050
+  100 | 0 | 0100
+  200 | 0 | 0200
+  250 | 0 | 0250
+  350 | 0 | 0350
+  400 | 0 | 0400
+  500 | 0 | 0500
+  550 | 0 | 0550
+  650 | 0 | 0650
+  700 | 0 | 0700
+(14 rows)
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+                             QUERY PLAN                              
+---------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t3.c
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p0_a on prt1_p0 t3
+                           Index Cond: (a = t2.b)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t2.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_1.a = t2_1.b)
+                           ->  Seq Scan on prt1_p1 t3_1
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1 t2_1
+                                       Filter: (a = 0)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p2_a on prt1_p2 t3_2
+                           Index Cond: (a = t2_2.b)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t3_3
+                           Index Cond: (a = t2_3.b)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_4.a = t2_4.b)
+                           ->  Seq Scan on prt1_p4 t3_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t2_4
+                                       Filter: (a = 0)
+(47 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+  a   | a |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.b
+   ->  Hash Right Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p5 t2_5
+                           Filter: (a = 0)
+(24 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: (t1.a = t2.b)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.a, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
+                     Filter: (a = 0)
+(22 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c int) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt1_m SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+CREATE TABLE prt2_m (a int, b int, c int) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES FROM (0, 0) TO (250, 250);
+CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES FROM (250, 250) TO (500, 500);
+CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES FROM (500, 500) TO (600, 600);
+INSERT INTO prt2_m SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p1.a = ((prt2_m_p1.b + prt2_m_p1.a) / 2)) AND (((prt1_m_p1.a + prt1_m_p1.b) / 2) = prt2_m_p1.b))
+               ->  Seq Scan on prt1_m_p1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p1
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p2.a = ((prt2_m_p2.b + prt2_m_p2.a) / 2)) AND (((prt1_m_p2.a + prt1_m_p2.b) / 2) = prt2_m_p2.b))
+               ->  Seq Scan on prt1_m_p2
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p2
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: ((prt1_m_p3.a = ((prt2_m_p3.b + prt2_m_p3.a) / 2)) AND (((prt1_m_p3.a + prt1_m_p3.b) / 2) = prt2_m_p3.b))
+               ->  Seq Scan on prt1_m_p3
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_m_p3
+                           Filter: (c = 0)
+(24 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.c = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+  a  | c |  b  | c 
+-----+---+-----+---
+   0 | 0 |   0 | 0
+  50 | 0 |     |  
+ 100 | 0 |     |  
+ 150 | 0 | 150 | 0
+ 200 | 0 |     |  
+ 250 | 0 |     |  
+ 300 | 0 | 300 | 0
+ 350 | 0 |     |  
+ 400 | 0 |     |  
+ 450 | 0 | 450 | 0
+ 500 | 0 |     |  
+ 550 | 0 |     |  
+     |   |  75 | 0
+     |   | 225 | 0
+     |   | 375 | 0
+     |   | 525 | 0
+(16 rows)
+
+--
+-- tests for list partitioned tables.
+--
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
+--
+-- list partitioned by expression
+--
+CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
+ANALYZE plt1_e;
+-- test partition matching with N-way join
+EXPLAIN (COSTS OFF)
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+                                           QUERY PLAN                                           
+------------------------------------------------------------------------------------------------
+ GroupAggregate
+   Group Key: t1.c, t2.c, t3.c
+   ->  Sort
+         Sort Key: t1.c, t3.c
+         ->  Append
+               ->  Hash Join
+                     Hash Cond: ((t1.c)::text = ltrim(t3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2.b = t1.b) AND ((t2.c)::text = (t1.c)::text))
+                           ->  Seq Scan on plt2_p4 t2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p4 t1
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p4 t3
+               ->  Hash Join
+                     Hash Cond: ((t1_1.c)::text = ltrim(t3_1.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t1_1.b = t2_1.b) AND ((t1_1.c)::text = (t2_1.c)::text))
+                           ->  Seq Scan on plt1_p1 t1_1
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p1 t2_1
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p1 t3_1
+               ->  Hash Join
+                     Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t1_2.b = t2_2.b) AND ((t1_2.c)::text = (t2_2.c)::text))
+                           ->  Seq Scan on plt1_p2 t1_2
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p2 t2_2
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p2 t3_2
+               ->  Hash Join
+                     Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2_3.b = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text))
+                           ->  Seq Scan on plt2_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p3 t3_3
+(41 rows)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p5 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_3
+                                 ->  Seq Scan on plt2_p3 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
 (16 rows)
 
---
--- tests for list partitioned tables.
---
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
---
--- list partitioned by expression
---
-CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1_e;
--- test partition matching with N-way join
+-- semi join
 EXPLAIN (COSTS OFF)
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                   QUERY PLAN                                   
---------------------------------------------------------------------------------
- GroupAggregate
-   Group Key: t1.c, t2.c, t3.c
-   ->  Sort
-         Sort Key: t1.c, t3.c
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
          ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.c = ltrim(t3.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1.b = t2.b) AND (t1.c = t2.c))
-                           ->  Seq Scan on plt1_p1 t1
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.c = ltrim(t3_1.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
-                           ->  Seq Scan on plt1_p2 t1_1
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.c = ltrim(t3_2.c, 'A'::text))
-                     ->  Hash Join
-                           Hash Cond: ((t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
-                           ->  Seq Scan on plt1_p3 t1_2
-                           ->  Hash
-                                 ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_e_p3 t3_2
-(32 rows)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Seq Scan on plt2_p2 t2_2
+                                 ->  Seq Scan on plt2_p3 t2_3
+(22 rows)
 
-SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  75.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 123.0000000000000000 | 123.0000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 225.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 273.0000000000000000 | 273.0000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 375.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 423.0000000000000000 | 423.0000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 525.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 573.0000000000000000 | 573.0000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1302,22 +4097,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
 --------------------------------------------------
  Hash Left Join
    Hash Cond: (t2.b = a)
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t3.a = t2.b)
-               ->  Seq Scan on prt1_p1 t3
-               ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
-         ->  Hash Join
-               Hash Cond: (t3_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t3_1
-               ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
-         ->  Hash Join
-               Hash Cond: (t3_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t3_2
-               ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
+   ->  Hash Join
+         Hash Cond: (t3.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t3
+               ->  Seq Scan on prt1_p1 t3_1
+               ->  Seq Scan on prt1_p2 t3_2
+               ->  Seq Scan on prt1_p3 t3_3
+               ->  Seq Scan on prt1_p4 t3_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
    ->  Hash
          ->  Result
                One-Time Filter: false
@@ -1332,16 +4127,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 --
 -- tests for hash partitioned tables.
@@ -1417,41 +4218,9 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  273.0000000000000000 | 273.0000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
 (6 rows)
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
- Sort
-   Sort Key: t1.a
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
-               ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
-               ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
-                           Filter: (b = 0)
-(21 rows)
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
@@ -1466,26 +4235,24 @@ SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c
    Sort Key: t1.c
    ->  HashAggregate
          Group Key: t1.c, t2.c
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t2.c = t1.c)
-                     ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p1 t1
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on plt1_p4 t1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_1.c = t1_1.c)
-                     ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_p2 t1_1
+                           ->  Seq Scan on plt1_p1 t1_1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_2.c = t1_2.c)
-                     ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Seq Scan on plt1_p2 t1_2
                                  Filter: ((a % 25) = 0)
-(23 rows)
+                           ->  Seq Scan on plt1_p3 t1_3
+                                 Filter: ((a % 25) = 0)
+(21 rows)
 
 --
 -- multiple levels of partitioning
@@ -1881,64 +4648,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
  Hash Join
    Hash Cond: (t1.a = t2.a)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt4_n_p1 t2
                ->  Seq Scan on prt4_n_p2 t2_1
                ->  Seq Scan on prt4_n_p3 t2_2
-(11 rows)
+(13 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a and t1.a = t3.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                        QUERY PLAN                        
+----------------------------------------------------------
  Hash Join
-   Hash Cond: (t2.a = t1.a)
+   Hash Cond: (t3.b = t1.a)
    ->  Append
-         ->  Seq Scan on prt4_n_p1 t2
-         ->  Seq Scan on prt4_n_p2 t2_1
-         ->  Seq Scan on prt4_n_p3 t2_2
+         ->  Seq Scan on prt2_p0 t3
+         ->  Seq Scan on prt2_p1 t3_1
+         ->  Seq Scan on prt2_p2 t3_2
+         ->  Seq Scan on prt2_p3 t3_3
+         ->  Seq Scan on prt2_p4 t3_4
+         ->  Seq Scan on prt2_p5 t3_5
    ->  Hash
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.a = t3_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_2.b)
-                     ->  Seq Scan on prt1_p3 t1_2
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3 t3_2
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt4_n_p1 t2
+                           ->  Seq Scan on prt4_n_p2 t2_1
+                           ->  Seq Scan on prt4_n_p3 t2_2
 (23 rows)
 
 -- partitionwise join can not be applied if there are no equi-join conditions
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (b > t1.a)
-(12 rows)
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+(16 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partitionwise join
@@ -2024,16 +4797,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
-                     ->  Seq Scan on plt1_p1 t3
-                     ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3
+                     ->  Seq Scan on plt1_p1 t3_1
+                     ->  Seq Scan on plt1_p2 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partitionwise join can not be applied for a join between list and range
 -- partitioned tables
@@ -2044,14 +4818,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
  Hash Full Join
    Hash Cond: ((t2.c)::text = (t1.c)::text)
    ->  Append
-         ->  Seq Scan on prt1_p1 t2
-         ->  Seq Scan on prt1_p2 t2_1
-         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p0 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p2 t2_2
+         ->  Seq Scan on prt1_p3 t2_3
+         ->  Seq Scan on prt1_p4 t2_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt1_n_p1 t1
                ->  Seq Scan on prt1_n_p2 t1_1
-(10 rows)
+(12 rows)
 
 -- partitionwise join can not be applied if only one of joining tables has
 -- default partition
@@ -2067,16 +4843,279 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
    ->  Hash Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007');
+INSERT INTO plt1 SELECT i, i, to_char(i % 8, 'FM0000') FROM generate_series(0, 39) i WHERE i % 8 IN (0, 2, 3, 4, 6, 7);
+ANALYZE plt1;
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0005', '0007');
+INSERT INTO plt2 SELECT i, i, to_char(i % 8, 'FM0000') FROM generate_series(0, 39) i WHERE i % 8 IN (1, 2, 3, 4, 5, 7);
+ANALYZE plt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 8 != 2 AND COALESCE(t1.a, 0) % 8 != 3 AND COALESCE(t1.a, 0) % 8 != 4 AND COALESCE(t1.a, 0) % 8 != 7 ORDER BY t1.c, t2.c, t1.a, t2.a;
+                                                                               QUERY PLAN                                                                                
+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t2.c, t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.c = t2.c)
+               Filter: (((COALESCE(t1.a, 0) % 8) <> 2) AND ((COALESCE(t1.a, 0) % 8) <> 3) AND ((COALESCE(t1.a, 0) % 8) <> 4) AND ((COALESCE(t1.a, 0) % 8) <> 7))
+               ->  Seq Scan on plt1_p1 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t2_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 8) <> 2) AND ((COALESCE(t1_1.a, 0) % 8) <> 3) AND ((COALESCE(t1_1.a, 0) % 8) <> 4) AND ((COALESCE(t1_1.a, 0) % 8) <> 7))
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_1
+         ->  Hash Full Join
+               Hash Cond: (t1_2.c = t2_2.c)
+               Filter: (((COALESCE(t1_2.a, 0) % 8) <> 2) AND ((COALESCE(t1_2.a, 0) % 8) <> 3) AND ((COALESCE(t1_2.a, 0) % 8) <> 4) AND ((COALESCE(t1_2.a, 0) % 8) <> 7))
+               ->  Seq Scan on plt1_p3 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_2
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 8 != 2 AND COALESCE(t1.a, 0) % 8 != 3 AND COALESCE(t1.a, 0) % 8 != 4 AND COALESCE(t1.a, 0) % 8 != 7 ORDER BY t1.c, t2.c, t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+  0 | 0000 |    | 
+  8 | 0000 |    | 
+ 16 | 0000 |    | 
+ 24 | 0000 |    | 
+ 32 | 0000 |    | 
+  6 | 0006 |    | 
+ 14 | 0006 |    | 
+ 22 | 0006 |    | 
+ 30 | 0006 |    | 
+ 38 | 0006 |    | 
+    |      |  1 | 0001
+    |      |  9 | 0001
+    |      | 17 | 0001
+    |      | 25 | 0001
+    |      | 33 | 0001
+    |      |  5 | 0005
+    |      | 13 | 0005
+    |      | 21 | 0005
+    |      | 29 | 0005
+    |      | 37 | 0005
+(20 rows)
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN (NULL, '0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
+ANALYZE plt1;
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 DEFAULT;
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+ANALYZE plt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t2_1.c, t1.a, t2_1.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.c = t2_1.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 2) AND ((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt1_p1 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t2.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 2) AND ((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+  0 | 0000 |    | 
+  5 | 0000 |    | 
+ 10 | 0000 |    | 
+ 15 | 0000 |    | 
+ 20 | 0000 |    | 
+    |      |  1 | 0001
+    |      |  6 | 0001
+    |      | 11 | 0001
+    |      | 16 | 0001
+    |      | 21 | 0001
+(10 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2 t2 FULL JOIN plt1 t1 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t2_1.c, t1.a, t2_1.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t2_1.c = t1.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 2) AND ((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1
+         ->  Hash Full Join
+               Hash Cond: (t2.c = t1_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 2) AND ((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt2_p2 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_1
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2 t2 FULL JOIN plt1 t1 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+  0 | 0000 |    | 
+  5 | 0000 |    | 
+ 10 | 0000 |    | 
+ 15 | 0000 |    | 
+ 20 | 0000 |    | 
+    |      |  1 | 0001
+    |      |  6 | 0001
+    |      | 11 | 0001
+    |      | 16 | 0001
+    |      | 21 | 0001
+(10 rows)
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1;
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2;
+CREATE TABLE plt3 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt3_p1 PARTITION OF plt3 FOR VALUES IN ('0001');
+CREATE TABLE plt3_p2 PARTITION OF plt3 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2.a, t3.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.c = t3.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1.c = t2.c)
+                     ->  Seq Scan on plt1_p1 t1
+                     ->  Hash
+                           ->  Seq Scan on plt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on plt3_p1 t3
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t3_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_1.c = t2_1.c)
+                     ->  Seq Scan on plt1_p2 t1_1
+                     ->  Hash
+                           ->  Seq Scan on plt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt3_p2 t3_1
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+ a  |  c   | a  |  c   | a  |  c   
+----+------+----+------+----+------
+  0 | 0000 |    |      |    | 
+  5 | 0000 |    |      |    | 
+ 10 | 0000 |    |      |    | 
+ 15 | 0000 |    |      |    | 
+ 20 | 0000 |    |      |    | 
+  1 | 0001 |    |      |  1 | 0001
+  1 | 0001 |    |      |  6 | 0001
+  1 | 0001 |    |      | 11 | 0001
+  1 | 0001 |    |      | 16 | 0001
+  1 | 0001 |    |      | 21 | 0001
+  6 | 0001 |    |      |  1 | 0001
+  6 | 0001 |    |      |  6 | 0001
+  6 | 0001 |    |      | 11 | 0001
+  6 | 0001 |    |      | 16 | 0001
+  6 | 0001 |    |      | 21 | 0001
+ 11 | 0001 |    |      |  1 | 0001
+ 11 | 0001 |    |      |  6 | 0001
+ 11 | 0001 |    |      | 11 | 0001
+ 11 | 0001 |    |      | 16 | 0001
+ 11 | 0001 |    |      | 21 | 0001
+ 16 | 0001 |    |      |  1 | 0001
+ 16 | 0001 |    |      |  6 | 0001
+ 16 | 0001 |    |      | 11 | 0001
+ 16 | 0001 |    |      | 16 | 0001
+ 16 | 0001 |    |      | 21 | 0001
+ 21 | 0001 |    |      |  1 | 0001
+ 21 | 0001 |    |      |  6 | 0001
+ 21 | 0001 |    |      | 11 | 0001
+ 21 | 0001 |    |      | 16 | 0001
+ 21 | 0001 |    |      | 21 | 0001
+  2 | 0002 |  2 | 0002 |    | 
+  2 | 0002 |  7 | 0002 |    | 
+  2 | 0002 | 12 | 0002 |    | 
+  2 | 0002 | 17 | 0002 |    | 
+  2 | 0002 | 22 | 0002 |    | 
+  7 | 0002 |  2 | 0002 |    | 
+  7 | 0002 |  7 | 0002 |    | 
+  7 | 0002 | 12 | 0002 |    | 
+  7 | 0002 | 17 | 0002 |    | 
+  7 | 0002 | 22 | 0002 |    | 
+ 12 | 0002 |  2 | 0002 |    | 
+ 12 | 0002 |  7 | 0002 |    | 
+ 12 | 0002 | 12 | 0002 |    | 
+ 12 | 0002 | 17 | 0002 |    | 
+ 12 | 0002 | 22 | 0002 |    | 
+ 17 | 0002 |  2 | 0002 |    | 
+ 17 | 0002 |  7 | 0002 |    | 
+ 17 | 0002 | 12 | 0002 |    | 
+ 17 | 0002 | 17 | 0002 |    | 
+ 17 | 0002 | 22 | 0002 |    | 
+ 22 | 0002 |  2 | 0002 |    | 
+ 22 | 0002 |  7 | 0002 |    | 
+ 22 | 0002 | 12 | 0002 |    | 
+ 22 | 0002 | 17 | 0002 |    | 
+ 22 | 0002 | 22 | 0002 |    | 
+(55 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 92994b479b..d8434f6b1a 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -10,25 +10,39 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -69,11 +83,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -110,20 +132,30 @@ RESET enable_hashjoin;
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 
 EXPLAIN (COSTS OFF)
@@ -187,6 +219,114 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches prt2_p3 and
+-- prt2_p4 partition from prt2.
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- partitionwise join should be possible when we drop prt2_p4 from prt2.
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ANALYZE prt2;
+
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -211,28 +351,79 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
 -- test partition matching with N-way join
@@ -240,6 +431,175 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
@@ -285,27 +645,18 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
 ALTER TABLE plt2 DETACH PARTITION plt2_p3;
 ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
 ANALYZE plt2;
-
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.a % 25 = 0 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
 --
 -- multiple levels of partitioning
 --
@@ -450,3 +801,72 @@ ANALYZE prt2;
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007');
+INSERT INTO plt1 SELECT i, i, to_char(i % 8, 'FM0000') FROM generate_series(0, 39) i WHERE i % 8 IN (0, 2, 3, 4, 6, 7);
+ANALYZE plt1;
+
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0005', '0007');
+INSERT INTO plt2 SELECT i, i, to_char(i % 8, 'FM0000') FROM generate_series(0, 39) i WHERE i % 8 IN (1, 2, 3, 4, 5, 7);
+ANALYZE plt2;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 8 != 2 AND COALESCE(t1.a, 0) % 8 != 3 AND COALESCE(t1.a, 0) % 8 != 4 AND COALESCE(t1.a, 0) % 8 != 7 ORDER BY t1.c, t2.c, t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 8 != 2 AND COALESCE(t1.a, 0) % 8 != 3 AND COALESCE(t1.a, 0) % 8 != 4 AND COALESCE(t1.a, 0) % 8 != 7 ORDER BY t1.c, t2.c, t1.a, t2.a;
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN (NULL, '0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
+ANALYZE plt1;
+
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 DEFAULT;
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+ANALYZE plt2;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2 t2 FULL JOIN plt1 t1 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2 t2 FULL JOIN plt1 t1 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1;
+
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2;
+
+CREATE TABLE plt3 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt3_p1 PARTITION OF plt3 FOR VALUES IN ('0001');
+CREATE TABLE plt3_p2 PARTITION OF plt3 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
-- 
2.19.2

v28-0002-Modify-merge_null_partitions.patchapplication/octet-stream; name=v28-0002-Modify-merge_null_partitions.patchDownload
From fb9884419dd05f0e774cb193ed4e8950783ebbeb Mon Sep 17 00:00:00 2001
From: Etsuro Fujita <efujita@postgresql.org>
Date: Fri, 22 Nov 2019 21:56:40 +0900
Subject: [PATCH 2/2] Modify merge_null_partitions()

---
 src/backend/partitioning/partbounds.c        | 490 ++++++++++---------
 src/test/regress/expected/partition_join.out | 292 ++++++-----
 src/test/regress/sql/partition_join.sql      |   5 +-
 3 files changed, 427 insertions(+), 360 deletions(-)

diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index a362998e57..494632fc7c 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -141,7 +141,7 @@ static int map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner
 						 int outer_part, int inner_part, int *next_index);
 static int merge_partition_with_dummy(PartitionMap *map, int index,
 						   int *next_index);
-static bool process_outer_partition(PartitionMap *outer_map,
+static int process_outer_partition(PartitionMap *outer_map,
 						PartitionMap *inner_map,
 						bool outer_has_default,
 						bool inner_has_default,
@@ -149,9 +149,8 @@ static bool process_outer_partition(PartitionMap *outer_map,
 						int inner_default,
 						JoinType jointype,
 						int *next_index,
-						int *default_index,
-						int *merged_index);
-static bool process_inner_partition(PartitionMap *outer_map,
+						int *default_index);
+static int process_inner_partition(PartitionMap *outer_map,
 						PartitionMap *inner_map,
 						bool outer_has_default,
 						bool inner_has_default,
@@ -159,8 +158,7 @@ static bool process_inner_partition(PartitionMap *outer_map,
 						int outer_default,
 						JoinType jointype,
 						int *next_index,
-						int *default_index,
-						int *merged_index);
+						int *default_index);
 static void fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
 				   int nmerged, List *merged_indexes);
 static void generate_matching_part_pairs(RelOptInfo *outer_rel,
@@ -200,21 +198,24 @@ static void add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
 						List **merged_datums,
 						List **merged_kinds,
 						List **merged_indexes);
-static bool merge_default_partitions(PartitionMap *outer_map,
+static void merge_default_partitions(PartitionMap *outer_map,
 						 PartitionMap *inner_map,
 						 bool outer_has_default,
-						 int outer_default,
 						 bool inner_has_default,
+						 int outer_default,
 						 int inner_default,
 						 JoinType jointype,
 						 int *next_index,
 						 int *default_index);
-static bool merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
-					  PartitionMap *outer_map, PartitionMap *inner_map,
-					  bool outer_has_default, bool inner_has_default,
-					  bool outer_has_null, bool inner_has_null,
-					  JoinType jointype, int *next_index,
-					  int *default_index, int *null_index);
+static void merge_null_partitions(PartitionMap *outer_map,
+					  PartitionMap *inner_map,
+					  bool outer_has_null,
+					  bool inner_has_null,
+					  int outer_null,
+					  int inner_null,
+					  JoinType jointype,
+					  int *next_index,
+					  int *null_index);
 
 /*
  * get_qual_from_partbound
@@ -3554,11 +3555,11 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 			if (ub_cmpval > 0 && inner_part >= 0 &&
 				compare_range_bounds(partnatts, partsupfuncs, partcollations,
 									 &save_outer_ub, &inner_lb) > 0)
-				return NULL;
+				goto cleanup;
 			if (ub_cmpval < 0 && outer_part >= 0 &&
 				compare_range_bounds(partnatts, partsupfuncs, partcollations,
 									 &outer_lb, &save_inner_ub) < 0)
-				return NULL;
+				goto cleanup;
 
 			/*
 			 * A row from a non-overlapping portion (if any) of a partition
@@ -3568,7 +3569,7 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 			 */
 			if ((outer_has_default && (lb_cmpval > 0 || ub_cmpval < 0)) ||
 				(inner_has_default && (lb_cmpval < 0 || ub_cmpval > 0)))
-				return NULL;
+				goto cleanup;
 		}
 		else if (ub_cmpval < 0)
 		{
@@ -3579,17 +3580,17 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 
 			if (inner_has_default || IS_OUTER_JOIN(jointype))
 			{
-				if (!process_outer_partition(&outer_map,
-											 &inner_map,
-											 outer_has_default,
-											 inner_has_default,
-											 outer_part,
-											 inner_default,
-											 jointype,
-											 &next_index,
-											 &default_index,
-											 &merged_index))
-					return NULL;
+				merged_index = process_outer_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   outer_part,
+													   inner_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
 			}
 
 			/* Move to the next partition on the outer side. */
@@ -3606,17 +3607,17 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 
 			if (outer_has_default || jointype == JOIN_FULL)
 			{
-				if (!process_inner_partition(&outer_map,
-											 &inner_map,
-											 outer_has_default,
-											 inner_has_default,
-											 inner_part,
-											 outer_default,
-											 jointype,
-											 &next_index,
-											 &default_index,
-											 &merged_index))
-					return NULL;
+				merged_index = process_inner_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   inner_part,
+													   outer_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
 			}
 
 			/* Move to the next partition on the inner side. */
@@ -3624,7 +3625,7 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 											 &inner_lb, &inner_ub);
 		}
 
-		if (merged_index >= 0)
+		if (merged_index >= 0 && merged_index != default_index)
 		{
 			/* Add the range bounds of the merged partition. */
 			add_merged_range_bounds(partnatts, partsupfuncs, partcollations,
@@ -3636,18 +3637,10 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 
 	/* Merge default partitions if any. */
 	if (outer_has_default || inner_has_default)
-	{
-		if (!merge_default_partitions(&outer_map,
-									  &inner_map,
-									  outer_has_default,
-									  outer_default,
-									  inner_has_default,
-									  inner_default,
-									  jointype,
-									  &next_index,
-									  &default_index))
-			return NULL;
-	}
+		merge_default_partitions(&outer_map, &inner_map,
+								 outer_has_default, inner_has_default,
+								 outer_default, inner_default,
+								 jointype, &next_index, &default_index);
 	else
 		Assert(default_index == -1);
 
@@ -3675,7 +3668,8 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 													  -1, default_index);
 	}
 
-	/* Free any memory we used in this function. */
+cleanup:
+	/* Free local memory before returning. */
 	free_partition_map(&outer_map);
 	free_partition_map(&inner_map);
 	list_free(merged_datums);
@@ -3789,8 +3783,8 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 													o_index, i_index,
 													&next_index);
 
-			if (merged_index < 0)
-				return NULL;
+			if (merged_index == -1)
+				goto cleanup;
 
 			/* Move to the next pair of bounds. */
 			cnto++;
@@ -3808,17 +3802,17 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 				int			o_index = outer_bi->indexes[cnto];
 
 				Assert(o_index >= 0);
-				if (!process_outer_partition(&outer_map,
-											 &inner_map,
-											 outer_has_default,
-											 inner_has_default,
-											 o_index,
-											 inner_default,
-											 jointype,
-											 &next_index,
-											 &default_index,
-											 &merged_index))
-					return NULL;
+				merged_index = process_outer_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   o_index,
+													   inner_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
 			}
 
 			/* Move to the next datum on the outer side. */
@@ -3837,17 +3831,17 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 				int			i_index = inner_bi->indexes[cnti];
 
 				Assert(i_index >= 0);
-				if (!process_inner_partition(&outer_map,
-											 &inner_map,
-											 outer_has_default,
-											 inner_has_default,
-											 i_index,
-											 outer_default,
-											 jointype,
-											 &next_index,
-											 &default_index,
-											 &merged_index))
-					return NULL;
+				merged_index = process_inner_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   i_index,
+													   outer_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
 			}
 
 			/* Move to the next datum on the inner side. */
@@ -3858,7 +3852,7 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 		 * Add the list value with appropriate index in the list of datums, if
 		 * we have associated a partition with this list value.
 		 */
-		if (merged_index >= 0)
+		if (merged_index >= 0 && merged_index != default_index)
 		{
 			merged_indexes = lappend_int(merged_indexes, merged_index);
 			merged_datums = lappend(merged_datums, merged_datum);
@@ -3867,27 +3861,19 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 
 	/* Merge null partitions if any. */
 	if (outer_has_null || inner_has_null)
-	{
-		if (!merge_null_partitions(outer_bi, inner_bi,
-								   &outer_map, &inner_map,
-								   outer_has_default, inner_has_default,
-								   outer_has_null, inner_has_null,
-								   jointype, &next_index, &default_index,
-								   &null_index))
-			return NULL;
-	}
+		merge_null_partitions(&outer_map, &inner_map,
+							  outer_has_null, inner_has_null,
+							  outer_bi->null_index, inner_bi->null_index,
+							  jointype, &next_index, &null_index);
 	else
 		Assert(null_index == -1);
 
 	/* Merge default partitions if any. */
 	if (outer_has_default || inner_has_default)
-	{
-		if (!merge_default_partitions(&outer_map, &inner_map,
-									  outer_has_default, outer_default,
-									  inner_has_default, inner_default,
-									  jointype, &next_index, &default_index))
-			return NULL;
-	}
+		merge_default_partitions(&outer_map, &inner_map,
+								 outer_has_default, inner_has_default,
+								 outer_default, inner_default,
+								 jointype, &next_index, &default_index);
 	else
 		Assert(default_index == -1);
 
@@ -3916,7 +3902,8 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 													  null_index, default_index);
 	}
 
-	/* Free up all extra memory before returning from this function. */
+cleanup:
+	/* Free local memory before returning. */
 	free_partition_map(&outer_map);
 	free_partition_map(&inner_map);
 	list_free(merged_datums);
@@ -4099,7 +4086,7 @@ merge_partition_with_dummy(PartitionMap *map, int index, int *next_index)
  * *next_index is incremented when creating a new merged partition associated
  * with the given outer partition.
  */
-static bool
+static int
 process_outer_partition(PartitionMap *outer_map,
 						PartitionMap *inner_map,
 						bool outer_has_default,
@@ -4108,9 +4095,10 @@ process_outer_partition(PartitionMap *outer_map,
 						int inner_default,
 						JoinType jointype,
 						int *next_index,
-						int *default_index,
-						int *merged_index)
+						int *default_index)
 {
+	int 		merged_index = -1;
+
 	Assert(outer_index >= 0);
 
 	/*
@@ -4131,13 +4119,13 @@ process_outer_partition(PartitionMap *outer_map,
 		 * up on it.
 		 */
 		if (outer_has_default)
-			return false;
+			return -1;
 
-		*merged_index = map_and_merge_partitions(outer_map, inner_map,
-												 outer_index, inner_default,
-												 next_index);
-		if (*merged_index == -1)
-			return false;
+		merged_index = map_and_merge_partitions(outer_map, inner_map,
+												outer_index, inner_default,
+												next_index);
+		if (merged_index == -1)
+			return -1;
 
 		/*
 		 * If this is a FULL join, the merged partition would act as the
@@ -4147,11 +4135,9 @@ process_outer_partition(PartitionMap *outer_map,
 		if (jointype == JOIN_FULL)
 		{
 			if (*default_index == -1)
-				*default_index = *merged_index;
+				*default_index = merged_index;
 			else
-				Assert(*merged_index == *default_index);
-			/* Don't add this index to the list of merged indexes. */
-			*merged_index = -1;
+				Assert(*default_index == merged_index);
 		}
 	}
 	else
@@ -4159,13 +4145,12 @@ process_outer_partition(PartitionMap *outer_map,
 		Assert(IS_OUTER_JOIN(jointype));
 		Assert(jointype != JOIN_RIGHT);
 
-		if (outer_map->merged_indexes[outer_index] >= 0)
-			*merged_index = outer_map->merged_indexes[outer_index];
-		else
-			*merged_index = merge_partition_with_dummy(outer_map, outer_index,
-													   next_index);
+		merged_index = outer_map->merged_indexes[outer_index];
+		if (merged_index == -1)
+			merged_index = merge_partition_with_dummy(outer_map, outer_index,
+													  next_index);
 	}
-	return true;
+	return merged_index;
 }
 
 /*
@@ -4176,7 +4161,7 @@ process_outer_partition(PartitionMap *outer_map,
  * *next_index is incremented when creating a new merged partition associated
  * with the given inner partition.
  */
-static bool
+static int
 process_inner_partition(PartitionMap *outer_map,
 						PartitionMap *inner_map,
 						bool outer_has_default,
@@ -4185,9 +4170,10 @@ process_inner_partition(PartitionMap *outer_map,
 						int outer_default,
 						JoinType jointype,
 						int *next_index,
-						int *default_index,
-						int *merged_index)
+						int *default_index)
 {
+	int 		merged_index = -1;
+
 	Assert(inner_index >= 0);
 
 	/*
@@ -4208,13 +4194,13 @@ process_inner_partition(PartitionMap *outer_map,
 		 * up on it.
 		 */
 		if (inner_has_default)
-			return false;
+			return -1;
 
-		*merged_index = map_and_merge_partitions(outer_map, inner_map,
-												 outer_default, inner_index,
-												 next_index);
-		if (*merged_index == -1)
-			return false;
+		merged_index = map_and_merge_partitions(outer_map, inner_map,
+												outer_default, inner_index,
+												next_index);
+		if (merged_index == -1)
+			return -1;
 
 		/*
 		 * If this is an outer join, the merged partition would act as the
@@ -4225,24 +4211,21 @@ process_inner_partition(PartitionMap *outer_map,
 		{
 			Assert(jointype != JOIN_RIGHT);
 			if (*default_index == -1)
-				*default_index = *merged_index;
+				*default_index = merged_index;
 			else
-				Assert(*merged_index == *default_index);
-			/* Don't add this index to the list of merged indexes. */
-			*merged_index = -1;
+				Assert(*default_index == merged_index);
 		}
 	}
 	else
 	{
 		Assert(jointype == JOIN_FULL);
 
-		if (inner_map->merged_indexes[inner_index] >= 0)
-			*merged_index = inner_map->merged_indexes[inner_index];
-		else
-			*merged_index = merge_partition_with_dummy(inner_map, inner_index,
-													   next_index);
+		merged_index = inner_map->merged_indexes[inner_index];
+		if (merged_index == -1)
+			merged_index = merge_partition_with_dummy(inner_map, inner_index,
+													  next_index);
 	}
-	return true;
+	return merged_index;
 }
 
 /*
@@ -4430,175 +4413,200 @@ build_merged_partition_bounds(char strategy, List *merged_datums,
 }
 
 /*
- * Merge default partitions from both sides, if any, and assign the default
- * partition for the join result, if necessary.
- *
- * If both the relations have default partitions, try mapping those to each
- * other. If the mapping succeeds corresponding merged partition will act as
- * the default partition of the join result.
+ * merge_default_partitions
+ *		Merge the default partitions from both sides, if any
  *
- * If inner side of the join has default but not the outer side, rows in it
- * won't appear in the join result. So don't create a default partition. If
- * outer side of the join has default but not the inner side, rows in it will
- * appear in the join result, so create a default merged partition.
+ * *default_index is set to the index of the default partition of the joinrel,
+ * if appropriate.
  */
-static bool
-merge_default_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
-						 bool outer_has_default, int outer_default,
-						 bool inner_has_default, int inner_default,
-						 JoinType jointype, int *next_index,
+static void
+merge_default_partitions(PartitionMap *outer_map,
+						 PartitionMap *inner_map,
+						 bool outer_has_default,
+						 bool inner_has_default,
+						 int outer_default,
+						 int inner_default,
+						 JoinType jointype,
+						 int *next_index,
 						 int *default_index)
 {
+	int 		outer_merged_index = -1;
+	int 		inner_merged_index = -1;
+
 	Assert(outer_has_default || inner_has_default);
 
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0 && outer_default < outer_map->nparts);
+		outer_merged_index = outer_map->merged_indexes[outer_default];
+	}
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0 && inner_default < inner_map->nparts);
+		inner_merged_index = inner_map->merged_indexes[inner_default];
+	}
+
 	if (outer_has_default && !inner_has_default)
 	{
+		/*
+		 * If this is an outer join, assign a new merged partition if not
+		 * already done.  The resulting partition should be the default
+		 * partition of the joinrel.
+		 */
 		if (IS_OUTER_JOIN(jointype))
 		{
-			int			merged_index;
-
 			Assert(jointype != JOIN_RIGHT);
-			Assert(outer_default >= 0 && outer_default < outer_map->nparts);
-			merged_index = outer_map->merged_indexes[outer_default];
-			if (merged_index == -1)
-			{
-				Assert(*default_index == -1);
+			if (outer_merged_index == -1)
 				*default_index = merge_partition_with_dummy(outer_map,
 															outer_default,
 															next_index);
-			}
 			else
-				Assert(*default_index == merged_index);
+				Assert(*default_index == outer_merged_index);
 		}
 		else
-			Assert(*default_index < 0);
+			Assert(*default_index == -1);
 	}
 	else if (!outer_has_default && inner_has_default)
 	{
+		/*
+		 * If this is a FULL join, assign a new merged partition if not
+		 * already done.  The resulting partition should be the default
+		 * partition of the joinrel.
+		 */
 		if (jointype == JOIN_FULL)
 		{
-			int			merged_index;
-
-			Assert(inner_default >= 0 && inner_default < inner_map->nparts);
-			merged_index = inner_map->merged_indexes[inner_default];
-			if (merged_index == -1)
-			{
-				Assert(*default_index == -1);
+			if (inner_merged_index == -1)
 				*default_index = merge_partition_with_dummy(inner_map,
 															inner_default,
 															next_index);
-			}
 			else
-				Assert(*default_index == merged_index);
+				Assert(*default_index == inner_merged_index);
 		}
 		else
-			Assert(*default_index < 0);
+			Assert(*default_index == -1);
 	}
 	else
 	{
 		Assert(outer_has_default && inner_has_default);
+		Assert(outer_merged_index == -1);
+		Assert(inner_merged_index == -1);
+		Assert(*default_index == -1);
 
+		/*
+		 * Merge the default partitions.  Both partitions aren't merged yet,
+		 * so the partitions should be merged successfully.  The resulting
+		 * partition should be the default partition of the joinrel.
+		 */
 		*default_index = map_and_merge_partitions(outer_map,
 												  inner_map,
 												  outer_default,
 												  inner_default,
 												  next_index);
-		if (*default_index == -1)
-			return false;
+		Assert(*default_index >= 0);
 	}
-
-	return true;
 }
 
 /*
  * merge_null_partitions
+ *		Merge the NULL partitions from both sides, if any
  *
- * Merge NULL partitions, i.e. a partition that can hold NULL values for a list
- * partitioned table, if any. Find the index of merged partition to which the
- * NULL values would belong in the join result. If one joining relation has a
- * NULL partition but not the other, try matching it with the default partition
- * from the other relation since the default partition may have rows with NULL
- * partition key. We can eliminate a NULL partition when it appears only on the
- * inner side of the join and the outer side doesn't have a default partition.
- *
- * When the equality operator used for join is strict, two NULL values will not
- * be considered as equal, and thus a NULL partition can be eliminated for an
- * inner join. But we don't check the strictness operator here.
+ * *null_index is set to the index of the NULL partition of the joinrel, if
+ * appropriate.
  */
-static bool
-merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
-					  PartitionMap *outer_map, PartitionMap *inner_map,
-					  bool outer_has_default, bool inner_has_default,
-					  bool outer_has_null, bool inner_has_null,
-					  JoinType jointype, int *next_index,
-					  int *default_index, int *null_index)
+static void
+merge_null_partitions(PartitionMap *outer_map,
+					  PartitionMap *inner_map,
+					  bool outer_has_null,
+					  bool inner_has_null,
+					  int outer_null,
+					  int inner_null,
+					  JoinType jointype,
+					  int *next_index,
+					  int *null_index)
 {
+	bool 		merge_outer_null = false;
+	bool 		merge_inner_null = false;
+
 	Assert(outer_has_null || inner_has_null);
 	Assert(*null_index == -1);
 
-	if (outer_has_null && !inner_has_null)
+	/*
+	 * If this is an INNER or SEMI join, two NULL values won't be considered
+	 * to be equal, in which case we don't need to do anything.
+	 */
+	if (jointype == JOIN_INNER || jointype == JOIN_SEMI)
+		return;
+
+	Assert(IS_OUTER_JOIN(jointype));
+	Assert(jointype != JOIN_RIGHT);
+
+	if (outer_has_null)
 	{
-		int			merged_index = -1;
+		/*
+		 * Since this is an outer join, if the NULL partition of the outer
+		 * side isn't merged yet, it means that the partition contains only
+		 * NULL values as the key values, in which case we need to create a
+		 * merged partition below.
+		 */
+		Assert(outer_null >= 0 && outer_null < outer_map->nparts);
+		if (outer_map->merged_indexes[outer_null] == -1)
+			merge_outer_null = true;
+	}
+	if (inner_has_null)
+	{
+		/*
+		 * If this is a FULL join, and the NULL partition of the inner side
+		 * isn't merged yet, it means that the partition contains only NULL
+		 * values as the key values, in which case we need to create a merged
+		 * partition below.
+		 */
+		Assert(inner_null >= 0 && inner_null < inner_map->nparts);
+		if (jointype == JOIN_FULL &&
+			inner_map->merged_indexes[inner_null] == -1)
+			merge_inner_null = true;
+	}
+
+	if (!merge_outer_null && !merge_inner_null)
+		return;
+
+	if (merge_outer_null && !merge_inner_null)
+	{
+		Assert(outer_has_null);
 
 		/*
-		 * If the NULL partition was missing from the inner side of the join,
-		 * the partition of the join to which the NULL partition matches will
-		 * contain the NULL values and thus become the NULL partition of the
-		 * the join.
+		 * Assign a new merged partition.  The resulting partition should be
+		 * the NULL partition of the joinrel.
 		 */
-		if (inner_has_default || IS_OUTER_JOIN(jointype))
-		{
-			if (!process_outer_partition(outer_map,
-										 inner_map,
-										 outer_has_default,
-										 inner_has_default,
-										 outer_bi->null_index,
-										 inner_bi->default_index,
-										 jointype,
-										 next_index,
-										 default_index,
-										 &merged_index))
-				return false;
-		}
-		*null_index = merged_index;
+		*null_index = merge_partition_with_dummy(outer_map, outer_null,
+												 next_index);
 	}
-	else if (!outer_has_null && inner_has_null)
+	else if (!merge_outer_null && merge_inner_null)
 	{
-		int			merged_index = -1;
+		Assert(inner_has_null);
+		Assert(jointype == JOIN_FULL);
 
 		/*
-		 * If the NULL partition was missing from the outer side of the join,
-		 * the partition of the join to which the NULL partition matches will
-		 * contain the NULL values and thus become the NULL partition of the
-		 * the join.
+		 * Assign a new merged partition.  The resulting partition should be
+		 * the NULL partition of the joinrel.
 		 */
-		if (outer_has_default || jointype == JOIN_FULL)
-		{
-			if (!process_inner_partition(outer_map,
-										 inner_map,
-										 outer_has_default,
-										 inner_has_default,
-										 inner_bi->null_index,
-										 outer_bi->default_index,
-										 jointype,
-										 next_index,
-										 default_index,
-										 &merged_index))
-				return false;
-		}
-		*null_index = merged_index;
+		*null_index = merge_partition_with_dummy(inner_map, inner_null,
+												 next_index);
 	}
 	else
 	{
-		/* Both the relations have NULL partitions, try merging them. */
-		*null_index = map_and_merge_partitions(outer_map,
-											   inner_map,
-											   outer_bi->null_index,
-											   inner_bi->null_index,
+		Assert(merge_outer_null && merge_inner_null);
+		Assert(outer_has_null);
+		Assert(inner_has_null);
+		Assert(jointype == JOIN_FULL);
+
+		/*
+		 * Merge the NULL partitions.  Both partitions aren't merged yet, so
+		 * the partitions should be merged successfully.  The resulting
+		 * partition should be the NULL partition of the joinrel.
+		 */
+		*null_index = map_and_merge_partitions(outer_map, inner_map,
+											   outer_null, inner_null,
 											   next_index);
-		if (*null_index == -1)
-			return false;
+		Assert(*null_index >= 0);
 	}
-
-	return true;
 }
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 1675abb1e7..25249adf9c 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -3866,10 +3866,7 @@ select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHE
  470 | 0 | 
 (1 row)
 
--- partition have a NULL on both side with different partition bounds w.r.t other side
--- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
--- Partition-wise join can not handle the case when one partition from one side
--- matches with multiple partitions on the other side
+-- test the null-partition handling in the new partition-matching algorithm
 DROP TABLE plt1_p3;
 CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
 INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
@@ -3877,94 +3874,138 @@ ANALYZE plt1;
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                         QUERY PLAN                         
+------------------------------------------------------------
  Sort
    Sort Key: t1.a
-   ->  Hash Join
-         Hash Cond: ((t2.c)::text = (t1.c)::text)
-         Join Filter: ((t1.b + t2.b) = 0)
-         ->  Append
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
                ->  Seq Scan on plt2_p4 t2
-               ->  Seq Scan on plt2_p1 t2_1
-               ->  Seq Scan on plt2_p2 t2_2
-               ->  Seq Scan on plt2_p3 t2_3
-         ->  Hash
-               ->  Append
+               ->  Hash
                      ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
                      ->  Seq Scan on plt1_p1 t1_1
-                     ->  Seq Scan on plt1_p2 t1_2
-                     ->  Seq Scan on plt1_p3 t1_3
-(16 rows)
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
 
 -- left join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                         QUERY PLAN                         
+------------------------------------------------------------
  Sort
    Sort Key: t1.a
-   ->  Hash Right Join
-         Hash Cond: ((t2.c)::text = (t1.c)::text)
-         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
-         ->  Append
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
                ->  Seq Scan on plt2_p4 t2
-               ->  Seq Scan on plt2_p1 t2_1
-               ->  Seq Scan on plt2_p2 t2_2
-               ->  Seq Scan on plt2_p3 t2_3
-         ->  Hash
-               ->  Append
+               ->  Hash
                      ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
                      ->  Seq Scan on plt1_p1 t1_1
-                     ->  Seq Scan on plt1_p2 t1_2
-                     ->  Seq Scan on plt1_p3 t1_3
-(16 rows)
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
 
 -- right join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                         QUERY PLAN                         
+------------------------------------------------------------
  Sort
    Sort Key: t2.a
-   ->  Hash Left Join
-         Hash Cond: ((t2.c)::text = (t1.c)::text)
-         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
-         ->  Append
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
                ->  Seq Scan on plt2_p4 t2
-               ->  Seq Scan on plt2_p1 t2_1
-               ->  Seq Scan on plt2_p2 t2_2
-               ->  Seq Scan on plt2_p3 t2_3
-         ->  Hash
-               ->  Append
+               ->  Hash
                      ->  Seq Scan on plt1_p4 t1
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
                      ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
                      ->  Seq Scan on plt1_p2 t1_2
-                     ->  Seq Scan on plt1_p3 t1_3
-(16 rows)
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
 
 -- full join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
-                          QUERY PLAN                           
----------------------------------------------------------------
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.a
-   ->  Hash Full Join
-         Hash Cond: ((t2.c)::text = (t1.c)::text)
-         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
-         ->  Append
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
                ->  Seq Scan on plt2_p4 t2
-               ->  Seq Scan on plt2_p1 t2_1
-               ->  Seq Scan on plt2_p2 t2_2
-               ->  Seq Scan on plt2_p3 t2_3
-         ->  Hash
-               ->  Append
+               ->  Hash
                      ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
                      ->  Seq Scan on plt1_p1 t1_1
-                     ->  Seq Scan on plt1_p2 t1_2
-                     ->  Seq Scan on plt1_p3 t1_3
-(16 rows)
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
 
 -- semi join
 EXPLAIN (COSTS OFF)
@@ -3973,27 +4014,34 @@ select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t
 --------------------------------------------------------------
  Sort
    Sort Key: t1.a, t1.c
-   ->  Hash Join
-         Hash Cond: ((t1.c)::text = (t2.c)::text)
-         ->  Append
-               ->  Seq Scan on plt1_p4 t1
-                     Filter: (b = 0)
-               ->  Seq Scan on plt1_p1 t1_1
-                     Filter: (b = 0)
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
                ->  Seq Scan on plt1_p2 t1_2
                      Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
                ->  Seq Scan on plt1_p3 t1_3
                      Filter: (b = 0)
-         ->  Hash
-               ->  HashAggregate
-                     Group Key: (t2.c)::text
-                     ->  Result
-                           ->  Append
-                                 ->  Seq Scan on plt2_p4 t2
-                                 ->  Seq Scan on plt2_p1 t2_1
-                                 ->  Seq Scan on plt2_p2 t2_2
-                                 ->  Seq Scan on plt2_p3 t2_3
-(22 rows)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
 
 EXPLAIN (COSTS OFF)
 select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
@@ -4001,78 +4049,92 @@ select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t
 --------------------------------------------------------------
  Sort
    Sort Key: t1.a, t1.c
-   ->  Hash Join
-         Hash Cond: ((t1.c)::text = (t2.c)::text)
-         ->  Append
-               ->  Seq Scan on plt2_p4 t1
-                     Filter: (b = 0)
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
                ->  Seq Scan on plt2_p1 t1_1
                      Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
                ->  Seq Scan on plt2_p2 t1_2
                      Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
                ->  Seq Scan on plt2_p3 t1_3
                      Filter: (b = 0)
-         ->  Hash
-               ->  HashAggregate
-                     Group Key: (t2.c)::text
-                     ->  Result
-                           ->  Append
-                                 ->  Seq Scan on plt1_p4 t2
-                                 ->  Seq Scan on plt1_p1 t2_1
-                                 ->  Seq Scan on plt1_p2 t2_2
-                                 ->  Seq Scan on plt1_p3 t2_3
-(22 rows)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
 
 -- anti join
 EXPLAIN (COSTS OFF)
 select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                          
+--------------------------------------------------------------
  Sort
    Sort Key: t1.a, t1.c
-   ->  Hash Anti Join
-         Hash Cond: ((t1.c)::text = (t2.c)::text)
-         ->  Append
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
                ->  Seq Scan on plt1_p4 t1
                      Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
                ->  Seq Scan on plt1_p1 t1_1
                      Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
                ->  Seq Scan on plt1_p2 t1_2
                      Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
                ->  Seq Scan on plt1_p3 t1_3
                      Filter: (b = 0)
-         ->  Hash
-               ->  Append
-                     ->  Seq Scan on plt2_p4 t2
-                     ->  Seq Scan on plt2_p1 t2_1
-                     ->  Seq Scan on plt2_p2 t2_2
-                     ->  Seq Scan on plt2_p3 t2_3
-(19 rows)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
 
 EXPLAIN (COSTS OFF)
 select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                          
+--------------------------------------------------------------
  Sort
    Sort Key: t1.a, t1.c
-   ->  Hash Anti Join
-         Hash Cond: ((t1.c)::text = (t2.c)::text)
-         ->  Append
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
                ->  Seq Scan on plt2_p4 t1
                      Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
                ->  Seq Scan on plt2_p1 t1_1
                      Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
                ->  Seq Scan on plt2_p2 t1_2
                      Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
                ->  Seq Scan on plt2_p3 t1_3
                      Filter: (b = 0)
-         ->  Hash
-               ->  Append
-                     ->  Seq Scan on plt1_p4 t2
-                     ->  Seq Scan on plt1_p1 t2_1
-                     ->  Seq Scan on plt1_p2 t2_2
-                     ->  Seq Scan on plt1_p3 t2_3
-(19 rows)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index d8434f6b1a..a552874742 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -561,10 +561,7 @@ EXPLAIN (COSTS OFF)
 select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
--- partition have a NULL on both side with different partition bounds w.r.t other side
--- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
--- Partition-wise join can not handle the case when one partition from one side
--- matches with multiple partitions on the other side
+-- test the null-partition handling in the new partition-matching algorithm
 DROP TABLE plt1_p3;
 CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
 INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
-- 
2.19.2

#108Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Etsuro Fujita (#107)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Fri, Nov 22, 2019 at 10:08 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

Done. I modified compare_range_partitions() as well to match its the
variable names with others. Attached is a new version of the patch.

I reviewed the rest of the partbounds.c changes. Here are my review comments:

* I don't think this analysis is correct:

+/*
+ * merge_null_partitions
+ *
+ * Merge NULL partitions, i.e. a partition that can hold NULL values for a lis\
t
+ * partitioned table, if any. Find the index of merged partition to which the
+ * NULL values would belong in the join result. If one joining relation has a
+ * NULL partition but not the other, try matching it with the default partitio\
n
+ * from the other relation since the default partition may have rows with NULL
+ * partition key. We can eliminate a NULL partition when it appears only on th\
e
+ * inner side of the join and the outer side doesn't have a default partition.
+ *
+ * When the equality operator used for join is strict, two NULL values will no\
t
+ * be considered as equal, and thus a NULL partition can be eliminated for an
+ * inner join. But we don't check the strictness operator here.
+ */

First of all, I think we can assume here that the equality operator is
*strict*, because 1) have_partkey_equi_join(), which is executed
before calling merge_null_partitions(), requires that the
corresponding clause is mergejoinable, and 2) currently, we assume
that mergejoinable operators are strict (see MJEvalOuterValues() and
MJEvalInnerValues()). So I don't think it's needed to try merging a
NULL partition on one side with the default partition on the other
side as above. (merge_null_partitions() tries merging NULL partitions
as well, but in some cases I don't think we need to do that, either.)
So I rewrote merge_null_partitions() completely. Another change is
that I eliminated the NULL partition of the joinrel in more cases.
Attached is a patch (v28-0002-modify-merge_null_partitions.patch) for
that created on top of the main patch. I might be missing something,
though.

Other changes in that patch:

* I fixed memory leaks in partition_list_bounds_merge() and
partition_range_bounds_merge().

* I noticed this in merge_default_partitions():

+       Assert(outer_has_default && inner_has_default);
+
+       *default_index = map_and_merge_partitions(outer_map,
+                                                 inner_map,
+                                                 outer_default,
+                                                 inner_default,
+                                                 next_index);
+       if (*default_index == -1)
+           return false;

I think the merging should be done successfully, because of 1) this in
process_outer_partition():

+   if (inner_has_default)
+   {
+       Assert(inner_default >= 0);
+
+       /*
+        * If the outer side has the default partition as well, we need to
+        * merge the default partitions (see merge_default_partitions()); give
+        * up on it.
+        */
+       if (outer_has_default)
+           return false;

and 2) this in process_inner_partition():

+   if (outer_has_default)
+   {
+       Assert(outer_default >= 0);
+
+       /*
+        * If the inner side has the default partition as well, we need to
+        * merge the default partitions (see merge_default_partitions()); give
+        * up on it.
+        */
+       if (inner_has_default)
+           return false;

So I removed the above if test (ie, *default_index == -1). I also
modified that function a bit further, including comments.

* I simplified process_outer_partition() and process_inner_partition()
a bit, changing the APIs to match that of map_and_merge_partitions().
Also, I think this in these functions is a bit ugly;

+           /* Don't add this index to the list of merged indexes. */
+           *merged_index = -1;

so I removed it, and modified the condition on whether or not we add
merged bounds to the lists in partition_list_bounds_merge() and
partition_range_bounds_merge(), instead.

Moved to the next CF.

Best regards,
Etsuro Fujita

#109amul sul
sulamul@gmail.com
In reply to: Etsuro Fujita (#108)
2 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Attached is the rebase version atop the latest master head(2d0fdfaccec).

Regards,
Amul

Attachments:

v29-0001-Improve-partition-matching-for-partitionwise-join.patchapplication/octet-stream; name=v29-0001-Improve-partition-matching-for-partitionwise-join.patchDownload
From d581e5a4a1d73d40092c820b9e65e1bfe84b52e9 Mon Sep 17 00:00:00 2001
From: Amul Sul <amul.sul@enterprisedb.com>
Date: Sun, 8 Dec 2019 23:56:11 -0500
Subject: [PATCH 1/2] 
 v29-0001-Improve-partition-matching-for-partitionwise-join

---
 src/backend/nodes/outfuncs.c                 |    2 +
 src/backend/optimizer/path/joinrels.c        |  246 +-
 src/backend/optimizer/util/inherit.c         |    2 +
 src/backend/optimizer/util/relnode.c         |   45 +-
 src/backend/partitioning/partbounds.c        | 1607 +++++++
 src/include/nodes/pathnodes.h                |    5 +
 src/include/partitioning/partbounds.h        |    7 +
 src/test/regress/expected/partition_join.out | 4251 +++++++++++++++---
 src/test/regress/sql/partition_join.sql      |  482 +-
 9 files changed, 5958 insertions(+), 689 deletions(-)

diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index a80eccc2c10..4c8bb43065e 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -2281,6 +2281,8 @@ _outRelOptInfo(StringInfo str, const RelOptInfo *node)
 	WRITE_BOOL_FIELD(has_eclass_joins);
 	WRITE_BOOL_FIELD(consider_partitionwise_join);
 	WRITE_BITMAPSET_FIELD(top_parent_relids);
+	WRITE_BOOL_FIELD(merged);
+	WRITE_BITMAPSET_FIELD(all_partrels);
 	WRITE_NODE_FIELD(partitioned_child_rels);
 }
 
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 6a480ab7644..ed7bc23c7be 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -48,6 +48,9 @@ static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 												Relids left_relids, Relids right_relids);
 static int	match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel,
 										 bool strict_op);
+static void get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2);
 
 
 /*
@@ -1357,25 +1360,30 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	bool		merged = false;
+	List	   *parts1 = NIL;
+	List	   *parts2 = NIL;
+	ListCell   *lcr1 = NULL;
+	ListCell   *lcr2 = NULL;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1390,34 +1398,107 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		   joinrel->part_scheme == rel2->part_scheme);
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * If we don't have the partition bounds for the join rel yet, try to
+	 * create it along with pairs of partitions to be joined; else generate
+	 * those using the partitioning info for the join rel we already have.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	if (joinrel->nparts == -1)
+	{
+		PartitionScheme part_scheme = joinrel->part_scheme;
+		PartitionBoundInfo boundinfo = NULL;
+		int			nparts = 0;
+
+		Assert(joinrel->boundinfo == NULL);
+		Assert(joinrel->part_rels == NULL);
+
+		/*
+		 * See if the partition bounds for inputs are exactly the same, in
+		 * which case we don't need to work hard: partitions with the same
+		 * partition indexes will form join pairs, and the join rel will have
+		 * the same partition bounds as inputs; otherwise try to merge the
+		 * partition bounds along with generating join pairs.
+		 *
+		 * Even if one or both inputs have merged partition bounds, it'd be
+		 * possible for the partition bounds to be exactly the same, but it
+		 * seems unlikely to be worth the cycles to check; do this check only
+		 * if both inputs have non-merged partition bounds.
+		 */
+		if (!rel1->merged &&
+			!rel2->merged &&
+			rel1->nparts == rel2->nparts &&
+			partition_bounds_equal(part_scheme->partnatts,
+								   part_scheme->parttyplen,
+								   part_scheme->parttypbyval,
+								   rel1->boundinfo, rel2->boundinfo))
+		{
+			boundinfo = rel1->boundinfo;
+			nparts = rel1->nparts;
+		}
+		else
+		{
+			boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											   part_scheme->parttyplen,
+											   part_scheme->parttypbyval,
+											   part_scheme->partsupfunc,
+											   part_scheme->partcollation,
+											   rel1, rel2,
+											   parent_sjinfo->jointype,
+											   &parts1, &parts2);
+			if (boundinfo == NULL)
+			{
+				joinrel->nparts = 0;
+				return;
+			}
+			nparts = list_length(parts1);
+			merged = true;
+		}
+
+		Assert(nparts > 0);
+		joinrel->boundinfo = boundinfo;
+		joinrel->merged = merged;
+		joinrel->nparts = nparts;
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+	}
+	else
+	{
+		Assert(joinrel->nparts > 0);
+		Assert(joinrel->boundinfo);
+		Assert(joinrel->part_rels);
+
+		/*
+		 * If the partition bounds for the join rel are not merged ones,
+		 * inputs are guaranteed to have the same partition bounds, so
+		 * partitions with the same partition indexes will form join pairs;
+		 * else let get_matching_part_pairs() do the work.
+		 */
+		if (joinrel->merged)
+		{
+			get_matching_part_pairs(root, joinrel, rel1, rel2,
+									&parts1, &parts2);
+			Assert(list_length(parts1) == joinrel->nparts);
+			Assert(list_length(parts2) == joinrel->nparts);
+			merged = true;
+		}
+	}
 
-	nparts = joinrel->nparts;
+	if (merged)
+	{
+		lcr1 = list_head(parts1);
+		lcr2 = list_head(parts2);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
-		bool		rel1_empty = (child_rel1 == NULL ||
-								  IS_DUMMY_REL(child_rel1));
-		bool		rel2_empty = (child_rel2 == NULL ||
-								  IS_DUMMY_REL(child_rel2));
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		bool		rel1_empty;
+		bool		rel2_empty;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1425,6 +1506,22 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		if (merged)
+		{
+			child_rel1 = lfirst_node(RelOptInfo, lcr1);
+			child_rel2 = lfirst_node(RelOptInfo, lcr2);
+			lcr1 = lnext(parts1, lcr1);
+			lcr2 = lnext(parts2, lcr2);
+		}
+		else
+		{
+			child_rel1 = rel1->part_rels[cnt_parts];
+			child_rel2 = rel2->part_rels[cnt_parts];
+		}
+
+		rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1));
+		rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2));
+
 		/*
 		 * Check for cases where we can prove that this segment of the join
 		 * returns no rows, due to one or both inputs being empty (including
@@ -1522,6 +1619,8 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 												 child_sjinfo,
 												 child_sjinfo->jointype);
 			joinrel->part_rels[cnt_parts] = child_joinrel;
+			joinrel->all_partrels = bms_add_members(joinrel->all_partrels,
+													child_joinrel->relids);
 		}
 
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
@@ -1529,6 +1628,7 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		populate_joinrel_with_paths(root, child_rel1, child_rel2,
 									child_joinrel, child_sjinfo,
 									child_restrictlist);
+
 	}
 }
 
@@ -1738,3 +1838,99 @@ match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op)
 
 	return -1;
 }
+
+/*
+ * get_matching_part_pairs
+ *		Generate join pairs of partitions for the two inputs
+ */
+static void
+get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2)
+{
+	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
+	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
+	int 		cnt_parts;
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
+	{
+		RelOptInfo *child_joinrel = joinrel->part_rels[cnt_parts];
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		Relids		child_relids1;
+		Relids		child_relids2;
+
+		/*
+		 * If this segment of the join is empty, it means that this segment
+		 * was ignored when previously creating child-join paths for it in
+		 * try_partitionwise_join() as it would not contribute to the join
+		 * result, due to one or both inputs being empty; add NULL to each of
+		 * the given lists so that this segment will be ignored again in that
+		 * function.
+		 */
+		if (!child_joinrel)
+		{
+			*parts1 = lappend(*parts1, NULL);
+			*parts2 = lappend(*parts2, NULL);
+			continue;
+		}
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel1 side.
+		 */
+		child_relids1 = bms_intersect(child_joinrel->relids,
+									  rel1->all_partrels);
+		Assert(bms_num_members(child_relids1) == bms_num_members(rel1->relids));
+
+		/*
+		 * Get a child rel for rel1 with the relids.  Note that we should have
+		 * the child rel even if rel1 is a join rel, because in that case the
+		 * partitions specified in the relids would have matching/overlapping
+		 * boundaries, so those partitions should be considered as ones to be
+		 * joined even when planning partitionwise joins of rel1, meaning that
+		 * the child rel would have been built by the time we get here.
+		 */
+		if (rel1_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids1);
+
+			child_rel1 = find_base_rel(root, varno);
+		}
+		else
+			child_rel1 = find_join_rel(root, child_relids1);
+		Assert(child_rel1);
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel2 side.
+		 */
+		child_relids2 = bms_intersect(child_joinrel->relids,
+									  rel2->all_partrels);
+		Assert(bms_num_members(child_relids2) == bms_num_members(rel2->relids));
+
+		/*
+		 * Get a child rel for rel2 with the relids.  See above comments.
+		 */
+		if (rel2_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids2);
+
+			child_rel2 = find_base_rel(root, varno);
+		}
+		else
+			child_rel2 = find_join_rel(root, child_relids2);
+		Assert(child_rel2);
+
+		/*
+		 * The join of rel1 and rel2 is legal, so is the join of the child
+		 * rels obtained above; add them to the given lists as a join pair
+		 * producing this join segment.
+		 */
+		*parts1 = lappend(*parts1, child_rel1);
+		*parts2 = lappend(*parts2, child_rel2);
+	}
+}
diff --git a/src/backend/optimizer/util/inherit.c b/src/backend/optimizer/util/inherit.c
index a38df47e47e..1bd8f9ea6f4 100644
--- a/src/backend/optimizer/util/inherit.c
+++ b/src/backend/optimizer/util/inherit.c
@@ -376,6 +376,8 @@ expand_partitioned_rtentry(PlannerInfo *root, RelOptInfo *relinfo,
 		/* Create the otherrel RelOptInfo too. */
 		childrelinfo = build_simple_rel(root, childRTindex, relinfo);
 		relinfo->part_rels[i] = childrelinfo;
+		relinfo->all_partrels = bms_add_members(relinfo->all_partrels,
+												childrelinfo->relids);
 
 		/* If this child is itself partitioned, recurse */
 		if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index c9eb447d070..126d2af006e 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -240,10 +240,12 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->has_eclass_joins = false;
 	rel->consider_partitionwise_join = false;	/* might get changed later */
 	rel->part_scheme = NULL;
-	rel->nparts = 0;
+	rel->nparts = -1;
 	rel->boundinfo = NULL;
+	rel->merged = false;
 	rel->partition_qual = NIL;
 	rel->part_rels = NULL;
+	rel->all_partrels = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->partitioned_child_rels = NIL;
@@ -653,10 +655,12 @@ build_join_rel(PlannerInfo *root,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -829,10 +833,12 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -1639,7 +1645,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1652,24 +1658,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1681,17 +1669,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index cfb44e23e29..a362998e574 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -25,6 +25,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/pathnodes.h"
 #include "parser/parse_coerce.h"
 #include "partitioning/partbounds.h"
 #include "partitioning/partdesc.h"
@@ -69,6 +70,23 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+typedef struct PartitionMap
+{
+	int			nparts;			/* number of partitions */
+	int		   *merged_indexes;	/* indexes of merged partitions */
+	bool	   *merged;			/* flags to indicate whether partitions are
+								 * merged with non-dummy partitions */
+	bool		did_remapping;	/* did we remap partitions? */
+	int		   *old_indexes;	/* old indexes of merged partitions if
+								 * did_remapping */
+} PartitionMap;
+
+#define compare_range_bounds(partnatts, partsupfunc, partcollations, \
+							 bound1, bound2) \
+	(partition_rbound_cmp(partnatts, partsupfunc, partcollations, \
+						  (bound1)->datums, (bound1)->kind, (bound1)->lower, \
+						  bound2))
+
 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);
@@ -108,6 +126,95 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 									 Expr **keyCol,
 									 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static PartitionBoundInfo partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts);
+static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							JoinType jointype,
+							List **outer_parts, List **inner_parts);
+static void init_partition_map(RelOptInfo *rel, PartitionMap *map);
+static void free_partition_map(PartitionMap *map);
+static int map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_part, int inner_part, int *next_index);
+static int merge_partition_with_dummy(PartitionMap *map, int index,
+						   int *next_index);
+static bool process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index,
+						int *merged_index);
+static bool process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index,
+						int *merged_index);
+static void fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes);
+static void generate_matching_part_pairs(RelOptInfo *outer_rel,
+							 RelOptInfo *inner_rel,
+							 PartitionMap *outer_map,
+							 PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_part_list,
+							 List **inner_part_list);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int get_range_partition(PartitionBoundInfo bi, int *lb_pos,
+					PartitionRangeBound *lb, PartitionRangeBound *ub);
+static bool compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+						 Oid *partcollations,
+						 PartitionRangeBound *outer_lb,
+						 PartitionRangeBound *outer_ub,
+						 PartitionRangeBound *inner_lb,
+						 PartitionRangeBound *inner_ub,
+						 int *lb_cmpval, int *ub_cmpval);
+static void get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations, JoinType jointype,
+						PartitionRangeBound *outer_lb,
+						PartitionRangeBound *outer_ub,
+						PartitionRangeBound *inner_lb,
+						PartitionRangeBound *inner_ub,
+						int	lb_cmpval, int ub_cmpval,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub);
+static void add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub,
+						int merged_index,
+						List **merged_datums,
+						List **merged_kinds,
+						List **merged_indexes);
+static bool merge_default_partitions(PartitionMap *outer_map,
+						 PartitionMap *inner_map,
+						 bool outer_has_default,
+						 int outer_default,
+						 bool inner_has_default,
+						 int inner_default,
+						 JoinType jointype,
+						 int *next_index,
+						 int *default_index);
+static bool merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_map, PartitionMap *inner_map,
+					  bool outer_has_default, bool inner_has_default,
+					  bool outer_has_null, bool inner_has_null,
+					  JoinType jointype, int *next_index,
+					  int *default_index, int *null_index);
 
 /*
  * get_qual_from_partbound
@@ -2995,3 +3102,1503 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * This function builds and returns the partition bounds for a join relation
+ * between input relations, creating two lists of partitions, which are
+ * returned to *outer_parts and *inner_parts respectively.  The lists contain
+ * the same number of partitions, and the partitions at the same positions in
+ * the lists indicate join pairs used for partitioned join.
+ *
+ * This function returns NULL, setting *outer_parts and *inner_parts to NIL,
+ * if a partition on one side matches multiple partitions on the other side,
+ * in which case we currently don't support partitioned join.
+ */
+PartitionBoundInfo
+partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
+{
+	PartitionBoundInfo 	merged_bounds;
+	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
+						inner_binfo = inner_rel->boundinfo;
+	char				strategy = outer_binfo->strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = NULL;
+
+			break;
+
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partsupfunc,
+														partcollation,
+														outer_rel,
+														inner_rel,
+														jointype,
+														outer_parts,
+														inner_parts);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(partnatts,
+														 partsupfunc,
+														 partcollation,
+														 outer_rel,
+														 inner_rel,
+														 jointype,
+														 outer_parts,
+														 inner_parts);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	return merged_bounds;
+}
+
+/*
+ * get_range_partition
+ *		Returns the index of the range partition with the given lower bound
+ *
+ * *lb and *ub are set to the lower and upper bounds of the range partition
+ * respectively, and *lb_index is advanced to the next lower bound, if any.
+ */
+static int
+get_range_partition(PartitionBoundInfo bi, int *lb_index,
+					PartitionRangeBound *lb, PartitionRangeBound *ub)
+{
+	/* Return the index as -1 if we've exhausted all the lower bounds. */
+	if (*lb_index >= bi->ndatums)
+		return -1;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(*lb_index + 1 < bi->ndatums);
+
+	lb->index = bi->indexes[*lb_index];
+	lb->kind = bi->kind[*lb_index];
+	lb->datums = bi->datums[*lb_index];
+	lb->lower = true;
+	ub->index = bi->indexes[*lb_index + 1];
+	ub->kind = bi->kind[*lb_index + 1];
+	ub->datums = bi->datums[*lb_index + 1];
+	ub->lower = false;
+
+	/* The partition index of an upper bound should be valid. */
+	Assert(ub->index >= 0);
+
+	/*
+	 * Advance the lower bound; if there are no bounds left beyond the upper
+	 * bound, we have reached the last lower bound.
+	 */
+	if (*lb_index + 2 >= bi->ndatums)
+		*lb_index = bi->ndatums;
+	else
+	{
+		/*
+		 * If the index assigned to the bound next to the upper bound isn't
+		 * valid, that is the lower bound of the next range partition; else,
+		 * the upper bound of the current range partition is also the lower
+		 * bound of the next range partition.
+		 */
+		if (bi->indexes[*lb_index + 2] < 0)
+			*lb_index = *lb_index + 2;
+		else
+			*lb_index = *lb_index + 1;
+	}
+
+	return ub->index;
+}
+
+/*
+ * compare_range_partitions
+ *		Compares the bounds of two range partitions, and returns true if the
+ *		ranges of the partitions overlap, false otherwise
+ *
+ * *lb_cmpval is set to -1, 0 or 1 if the outer partition's lower bound is
+ * lower than, equal to or higher than the inner partition's lower bound
+ * respectively.  Likewise, *ub_cmpval is set to -1, 0 or 1 if the outer
+ * partition's upper bound is lower than, equal to or higher than the inner
+ * partition's upper bound respectively.
+ */
+static bool
+compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+						 Oid *partcollations,
+						 PartitionRangeBound *outer_lb,
+						 PartitionRangeBound *outer_ub,
+						 PartitionRangeBound *inner_lb,
+						 PartitionRangeBound *inner_ub,
+						 int *lb_cmpval, int *ub_cmpval)
+{
+	/*
+	 * Check to see if the upper bound of the outer partition is lower than
+	 * the lower bound of the inner partition; in which case the partitions
+	 * aren't overlapping.
+	 */
+	if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+							 outer_ub, inner_lb) < 0)
+	{
+		*lb_cmpval = -1;
+		*ub_cmpval = -1;
+		return false;
+	}
+
+	/*
+	 * Check to see if the lower bound of the outer partition is higher than
+	 * the upper bound of the inner partition; in which case the partitions
+	 * aren't overlapping.
+	 */
+	if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+							 outer_lb, inner_ub) > 0)
+	{
+		*lb_cmpval = 1;
+		*ub_cmpval = 1;
+		return false;
+	}
+
+	/* All other cases indicate overlapping partitions. */
+	*lb_cmpval = compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									  outer_lb, inner_lb);
+	*ub_cmpval = compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									  outer_ub, inner_ub);
+	return true;
+}
+
+/*
+ * get_merged_range_bounds
+ *		Given the bounds of range partitions to be join, determine the range
+ *		bounds of the merged partition produced from the range partitions
+ *
+ * *merged_lb and *merged_ub are set to the lower and upper bounds of the
+ * merged partition.
+ */
+static void
+get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations, JoinType jointype,
+						PartitionRangeBound *outer_lb,
+						PartitionRangeBound *outer_ub,
+						PartitionRangeBound *inner_lb,
+						PartitionRangeBound *inner_ub,
+						int	lb_cmpval, int ub_cmpval,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub)
+{
+	Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								outer_lb, inner_lb) == lb_cmpval);
+	Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								outer_ub, inner_ub) == ub_cmpval);
+
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			*merged_lb = (lb_cmpval > 0) ? *outer_lb : *inner_lb;
+			*merged_ub = (ub_cmpval < 0) ? *outer_ub : *inner_ub;
+			break;
+
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = *outer_ub;
+			*merged_lb = *outer_lb;
+			break;
+
+		case JOIN_FULL:
+			*merged_lb = (lb_cmpval < 0) ? *outer_lb : *inner_lb;
+			*merged_ub = (ub_cmpval > 0) ? *outer_ub : *inner_ub;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+}
+
+/*
+ * add_merged_range_bounds
+ *		Add the range bounds of a merged partition to the lists of range
+ *		bounds
+ */
+static void
+add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub,
+						int merged_index,
+						List **merged_datums,
+						List **merged_kinds,
+						List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		/* First merged partition */
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		Assert(*merged_kinds && *merged_indexes);
+
+		/* Get the last upper bound. */
+		prev_ub.index = llast_int(*merged_indexes);
+		prev_ub.datums = (Datum *) llast(*merged_datums);
+		prev_ub.kind = (PartitionRangeDatumKind *) llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		/*
+		 * We pass to partition_rbound_cmp() lower1 as false to prevent it
+		 * from considering the last upper bound to be smaller than the lower
+		 * bound of the merged partition when the values of the two range
+		 * bounds compare equal.
+		 */
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  merged_lb->datums, merged_lb->kind,
+									  false, &prev_ub);
+		Assert(cmpval >= 0);
+	}
+
+	/*
+	 * If the lower bound is higher than the last upper bound, add the lower
+	 * bound with the index as -1 indicating that that is a lower bound; else,
+	 * the last upper bound will be reused as the lower bound of the merged
+	 * partition, so skip this.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, merged_lb->datums);
+		*merged_kinds = lappend(*merged_kinds, merged_lb->kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	/* Add the upper bound and index of the merged partition. */
+	*merged_datums = lappend(*merged_datums, merged_ub->datums);
+	*merged_kinds = lappend(*merged_kinds, merged_ub->kind);
+	*merged_indexes = lappend_int(*merged_indexes, merged_index);
+}
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			outer_part;
+	int			inner_part;
+	int			outer_lb_index;
+	int			inner_lb_index;
+	PartitionRangeBound outer_lb;
+	PartitionRangeBound outer_ub;
+	PartitionRangeBound inner_lb;
+	PartitionRangeBound inner_ub;
+	int			next_index = 0;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_kinds = NIL;
+	List	   *merged_indexes = NIL;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = inner_lb_index = 0;
+	outer_part = get_range_partition(outer_bi, &outer_lb_index,
+									 &outer_lb, &outer_ub);
+	inner_part = get_range_partition(inner_bi, &inner_lb_index,
+									 &inner_lb, &inner_ub);
+	while (outer_part >= 0 || inner_part >= 0)
+	{
+		PartitionRangeBound merged_lb;
+		PartitionRangeBound merged_ub;
+		int			merged_index = -1;
+		bool		overlap;
+		int			ub_cmpval;
+		int			lb_cmpval;
+
+		if (outer_part >= 0)
+			Assert(outer_map.merged_indexes[outer_part] == -1 &&
+				   outer_map.merged[outer_part] == false);
+		if (inner_part >= 0)
+			Assert(inner_map.merged_indexes[inner_part] == -1 &&
+				   inner_map.merged[inner_part] == false);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (outer_part == -1)
+		{
+			overlap = false;
+			lb_cmpval = 1;
+			ub_cmpval = 1;
+		}
+		else if (inner_part == -1)
+		{
+			overlap = false;
+			lb_cmpval = -1;
+			ub_cmpval = -1;
+		}
+		else
+			overlap = compare_range_partitions(partnatts, partsupfuncs,
+											   partcollations,
+											   &outer_lb, &outer_ub,
+											   &inner_lb, &inner_ub,
+											   &lb_cmpval, &ub_cmpval);
+
+		if (overlap)
+		{
+			/* The ranges of partitions overlap; form a join pair of them. */
+
+			PartitionRangeBound save_outer_ub;
+			PartitionRangeBound save_inner_ub;
+
+			/* Get the bounds of the merged partition. */
+			get_merged_range_bounds(partnatts, partsupfuncs,
+									partcollations, jointype,
+									&outer_lb, &outer_ub,
+									&inner_lb, &inner_ub,
+									lb_cmpval, ub_cmpval,
+									&merged_lb, &merged_ub);
+
+			/*
+			 * Get the index of the merged partition.  Both partitions aren't
+			 * merged yet, so the partitions should be merged successfully.
+			 */
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
+													outer_part, inner_part,
+													&next_index);
+			Assert(merged_index >= 0);
+
+			/* Save the upper bounds of both partitions for use below. */
+			save_outer_ub = outer_ub;
+			save_inner_ub = inner_ub;
+
+			/* Move to the next pair of partitions. */
+			outer_part = get_range_partition(outer_bi, &outer_lb_index,
+											 &outer_lb, &outer_ub);
+			inner_part = get_range_partition(inner_bi, &inner_lb_index,
+											 &inner_lb, &inner_ub);
+
+			/*
+			 * If the range of a partition on one side overlaps the range of
+			 * the next partition on the other side, that will cause the
+			 * partition on one side to match at least two partitions on the
+			 * other side, which is the case that we currently don't support
+			 * partitioned join for; give up.
+			 */
+			if (ub_cmpval > 0 && inner_part >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &save_outer_ub, &inner_lb) > 0)
+				return NULL;
+			if (ub_cmpval < 0 && outer_part >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &outer_lb, &save_inner_ub) < 0)
+				return NULL;
+
+			/*
+			 * A row from a non-overlapping portion (if any) of a partition
+			 * on one side might find its join partner in the default
+			 * partition (if any) on the other side, causing the same
+			 * situation as above; if so, give up.
+			 */
+			if ((outer_has_default && (lb_cmpval > 0 || ub_cmpval < 0)) ||
+				(inner_has_default && (lb_cmpval < 0 || ub_cmpval > 0)))
+				return NULL;
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			merged_lb = outer_lb;
+			merged_ub = outer_ub;
+
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				if (!process_outer_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 outer_part,
+											 inner_default,
+											 jointype,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the outer side. */
+			outer_part = get_range_partition(outer_bi, &outer_lb_index,
+											 &outer_lb, &outer_ub);
+		}
+		else
+		{
+			/* Upper bound of outer range higher than that of the inner. */
+			Assert(ub_cmpval > 0);
+
+			merged_lb = inner_lb;
+			merged_ub = inner_ub;
+
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				if (!process_inner_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 inner_part,
+											 outer_default,
+											 jointype,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next partition on the inner side. */
+			inner_part = get_range_partition(inner_bi, &inner_lb_index,
+											 &inner_lb, &inner_ub);
+		}
+
+		if (merged_index >= 0)
+		{
+			/* Add the range bounds of the merged partition. */
+			add_merged_range_bounds(partnatts, partsupfuncs, partcollations,
+									&merged_lb, &merged_ub, merged_index,
+									&merged_datums, &merged_kinds,
+									&merged_indexes);
+		}
+	}
+
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+	{
+		if (!merge_default_partitions(&outer_map,
+									  &inner_map,
+									  outer_has_default,
+									  outer_default,
+									  inner_has_default,
+									  inner_default,
+									  jointype,
+									  &next_index,
+									  &default_index))
+			return NULL;
+	}
+	else
+		Assert(default_index == -1);
+
+	/*
+	 * Unlike the case for list partitioning, we wouldn't have re-merged
+	 * partitions, so did_remapping should be left alone.
+	 */
+	Assert(!outer_map.did_remapping && !inner_map.did_remapping);
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 &outer_map, &inner_map,
+								 next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes,
+													  merged_kinds,
+													  -1, default_index);
+	}
+
+	/* Free any memory we used in this function. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							JoinType jointype,
+							List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			next_index = 0;
+	int			null_index = -1;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			cnto;
+	int			cnti;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *merged_datum = NULL;
+		int			merged_index = -1;
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			cmpval;
+
+		/* Get the list datums of the next pair of partitions. */
+		odatums = cnto < outer_bi->ndatums ? outer_bi->datums[cnto] : NULL;
+		idatums = cnti < inner_bi->ndatums ? inner_bi->datums[cnti] : NULL;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+			cmpval = 1;
+		else if (cnti >= inner_bi->ndatums)
+			cmpval = -1;
+		else
+		{
+			Assert(odatums != NULL && idatums != NULL);
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0],
+													 idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			int			o_index = outer_bi->indexes[cnto];
+			int			i_index = inner_bi->indexes[cnti];
+
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			Assert(o_index >= 0 && i_index >= 0);
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
+													o_index, i_index,
+													&next_index);
+
+			if (merged_index < 0)
+				return NULL;
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			Assert(cnto < outer_bi->ndatums);
+
+			/* A datum missing from the inner side. */
+			merged_datum = odatums;
+
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				int			o_index = outer_bi->indexes[cnto];
+
+				Assert(o_index >= 0);
+				if (!process_outer_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 o_index,
+											 inner_default,
+											 jointype,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next datum on the outer side. */
+			cnto++;
+		}
+		else
+		{
+			Assert(cmpval > 0);
+			Assert(cnti < inner_bi->ndatums);
+
+			/* A datum missing from the outer side. */
+			merged_datum = idatums;
+
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				int			i_index = inner_bi->indexes[cnti];
+
+				Assert(i_index >= 0);
+				if (!process_inner_partition(&outer_map,
+											 &inner_map,
+											 outer_has_default,
+											 inner_has_default,
+											 i_index,
+											 outer_default,
+											 jointype,
+											 &next_index,
+											 &default_index,
+											 &merged_index))
+					return NULL;
+			}
+
+			/* Move to the next datum on the inner side. */
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	/* Merge null partitions if any. */
+	if (outer_has_null || inner_has_null)
+	{
+		if (!merge_null_partitions(outer_bi, inner_bi,
+								   &outer_map, &inner_map,
+								   outer_has_default, inner_has_default,
+								   outer_has_null, inner_has_null,
+								   jointype, &next_index, &default_index,
+								   &null_index))
+			return NULL;
+	}
+	else
+		Assert(null_index == -1);
+
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+	{
+		if (!merge_default_partitions(&outer_map, &inner_map,
+									  outer_has_default, outer_default,
+									  inner_has_default, inner_default,
+									  jointype, &next_index, &default_index))
+			return NULL;
+	}
+	else
+		Assert(default_index == -1);
+
+	/* Fix the merged_indexes list if necessary. */
+	if (outer_map.did_remapping || inner_map.did_remapping)
+	{
+		Assert(jointype == JOIN_FULL);
+		fix_merged_indexes(&outer_map, &inner_map,
+						   next_index, merged_indexes);
+	}
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 &outer_map, &inner_map,
+								 next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) <= next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes, NIL,
+													  null_index, default_index);
+	}
+
+	/* Free up all extra memory before returning from this function. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+	list_free(merged_datums);
+	list_free(merged_indexes);
+
+	return merged_bounds;
+}
+
+/*
+ * init_partition_map
+ *
+ * Initialize a PartitionMap struct for given relation.
+ */
+static void
+init_partition_map(RelOptInfo *rel, PartitionMap *map)
+{
+	int			nparts = rel->nparts;
+	int			i;
+
+	map->nparts = nparts;
+	map->merged_indexes = (int *) palloc(sizeof(int) * nparts);
+	map->merged = (bool *) palloc(sizeof(bool) * nparts);
+	map->did_remapping = false;
+	map->old_indexes = (int *) palloc(sizeof(int) * nparts);
+	for (i = 0; i < nparts; i++)
+	{
+		map->merged_indexes[i] = map->old_indexes[i] = -1;
+		map->merged[i] = false;
+	}
+}
+
+/*
+ * free_partition_map
+ */
+static void
+free_partition_map(PartitionMap *map)
+{
+	pfree(map->merged_indexes);
+	pfree(map->merged);
+	pfree(map->old_indexes);
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given partitions.
+ */
+static int
+map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_index, int inner_index, int *next_index)
+{
+	int 		outer_merged_index;
+	bool 		outer_merged;
+	int 		inner_merged_index;
+	bool 		inner_merged;
+
+	Assert(outer_index >= 0 && outer_index < outer_map->nparts);
+	outer_merged_index = outer_map->merged_indexes[outer_index];
+	outer_merged = outer_map->merged[outer_index];
+	Assert(inner_index >= 0 && inner_index < inner_map->nparts);
+	inner_merged_index = inner_map->merged_indexes[inner_index];
+	inner_merged = inner_map->merged[inner_index];
+
+	/*
+	 * Handle cases where both partitions are mapped to merged partitions.
+	 */
+	if (outer_merged_index >= 0 && inner_merged_index >= 0)
+	{
+		/*
+		 * If the mereged partitions are the same, no need to do anything;
+		 * return the index of the merged partition.  Otherwise, if both
+		 * partitions are merged with dummy partitions, re-merge them; map
+		 * them to the merged partition with the smaller of the two merged
+		 * indexes and return the smaller index.  Otherwise they can't be
+		 * merged, so return -1.
+		 */
+		if (outer_merged_index == inner_merged_index)
+		{
+			Assert(outer_merged);
+			Assert(inner_merged);
+			return outer_merged_index;
+		}
+		if (!outer_merged && !inner_merged)
+		{
+			/*
+			 * Note that we will fix the larger index that have been added to
+			 * the merged_indexes list so far in fix_merged_indexes().
+			 */
+			if (outer_merged_index < inner_merged_index)
+			{
+				outer_map->merged[outer_index] = true;
+				inner_map->merged_indexes[inner_index] = outer_merged_index;
+				inner_map->merged[inner_index] = true;
+				inner_map->did_remapping = true;
+				inner_map->old_indexes[inner_index] = inner_merged_index;
+				return outer_merged_index;
+			}
+			else
+			{
+				inner_map->merged[inner_index] = true;
+				outer_map->merged_indexes[outer_index] = inner_merged_index;
+				outer_map->merged[outer_index] = true;
+				outer_map->did_remapping = true;
+				outer_map->old_indexes[outer_index] = outer_merged_index;
+				return inner_merged_index;
+			}
+		}
+		return -1;
+	}
+
+	/* At least one partition isn't mapped to a merged partition. */
+	Assert(outer_merged_index == -1 || inner_merged_index == -1);
+
+	/*
+	 * If neither of partitions isn't mapped, assign them a new merged
+	 * partition and return the index of the merged partition.  Otherwise, if
+	 * one of partitions is merged with a dummy relation (and the other isn't
+	 * merged), re-merge it with the other, with the same index, and return
+	 * the index.  Otherwise they can't be merged, so return -1.
+	 */
+	if (outer_merged_index == -1 && inner_merged_index == -1)
+	{
+		int 		merged_index = *next_index;
+
+		Assert(!outer_merged);
+		Assert(!inner_merged);
+		outer_map->merged_indexes[outer_index] = merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged_indexes[inner_index] = merged_index;
+		inner_map->merged[inner_index] = true;
+		*next_index = *next_index + 1;
+		return merged_index;
+	}
+	if (outer_merged_index >= 0 && !outer_map->merged[outer_index])
+	{
+		Assert(inner_merged_index == -1);
+		Assert(!inner_merged);
+		inner_map->merged_indexes[inner_index] = outer_merged_index;
+		inner_map->merged[inner_index] = true;
+		outer_map->merged[outer_index] = true;
+		return outer_merged_index;
+	}
+	if (inner_merged_index >= 0 && !inner_map->merged[inner_index])
+	{
+		Assert(outer_merged_index == -1);
+		Assert(!outer_merged);
+		outer_map->merged_indexes[outer_index] = inner_merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged[inner_index] = true;
+		return inner_merged_index;
+	}
+	return -1;
+}
+
+/*
+ * merge_partition_with_dummy
+ *
+ * *next_index is incremented.
+ */
+static int
+merge_partition_with_dummy(PartitionMap *map, int index, int *next_index)
+{
+	int 		merged_index = *next_index;
+
+	Assert(index >= 0 && index < map->nparts);
+	Assert(map->merged_indexes[index] == -1);
+	Assert(!map->merged[index]);
+	map->merged_indexes[index] = merged_index;
+	/* Leave the merged flag alone! */
+	*next_index = *next_index + 1;
+	return merged_index;
+}
+
+/*
+ * process_outer_partition
+ *
+ * Determine the merged partition associated with the given outer partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given outer partition.
+ */
+static bool
+process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index,
+						int *merged_index)
+{
+	Assert(outer_index >= 0);
+
+	/*
+	 * If the inner side has the default partition, the outer partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an outer join, in which case the outer partition has to be
+	 * scanned all the way anyway; if the outer partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the outer partition with a dummy partition.
+	 */
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0);
+
+		/*
+		 * If the outer side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (outer_has_default)
+			return false;
+
+		*merged_index = map_and_merge_partitions(outer_map, inner_map,
+												 outer_index, inner_default,
+												 next_index);
+		if (*merged_index == -1)
+			return false;
+
+		/*
+		 * If this is a FULL join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (*default_index == -1)
+				*default_index = *merged_index;
+			else
+				Assert(*merged_index == *default_index);
+			/* Don't add this index to the list of merged indexes. */
+			*merged_index = -1;
+		}
+	}
+	else
+	{
+		Assert(IS_OUTER_JOIN(jointype));
+		Assert(jointype != JOIN_RIGHT);
+
+		if (outer_map->merged_indexes[outer_index] >= 0)
+			*merged_index = outer_map->merged_indexes[outer_index];
+		else
+			*merged_index = merge_partition_with_dummy(outer_map, outer_index,
+													   next_index);
+	}
+	return true;
+}
+
+/*
+ * process_inner_partition
+ *
+ * Determine the merged partition associated with the given inner partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given inner partition.
+ */
+static bool
+process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index,
+						int *merged_index)
+{
+	Assert(inner_index >= 0);
+
+	/*
+	 * If the outer side has the default partition, the inner partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an FULL join, in which case the inner partition has to be
+	 * scanned all the way anyway; if the inner partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the inner partition with a dummy partition.
+	 */
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0);
+
+		/*
+		 * If the inner side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (inner_has_default)
+			return false;
+
+		*merged_index = map_and_merge_partitions(outer_map, inner_map,
+												 outer_default, inner_index,
+												 next_index);
+		if (*merged_index == -1)
+			return false;
+
+		/*
+		 * If this is an outer join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			if (*default_index == -1)
+				*default_index = *merged_index;
+			else
+				Assert(*merged_index == *default_index);
+			/* Don't add this index to the list of merged indexes. */
+			*merged_index = -1;
+		}
+	}
+	else
+	{
+		Assert(jointype == JOIN_FULL);
+
+		if (inner_map->merged_indexes[inner_index] >= 0)
+			*merged_index = inner_map->merged_indexes[inner_index];
+		else
+			*merged_index = merge_partition_with_dummy(inner_map, inner_index,
+													   next_index);
+	}
+	return true;
+}
+
+/*
+ * fix_merged_indexes
+ */
+static void
+fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes)
+{
+	int		   *new_indexes;
+	int			merged_index;
+	int			i;
+	ListCell   *lc;
+
+	new_indexes = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		new_indexes[i] = -1;
+
+	/* Build the mapping of old merged indexes to new merged indexes. */
+	if (outer_map->did_remapping)
+	{
+		for (i = 0; i < outer_map->nparts; i++)
+		{
+			merged_index = outer_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = outer_map->merged_indexes[i];
+		}
+	}
+	if (inner_map->did_remapping)
+	{
+		for (i = 0; i < inner_map->nparts; i++)
+		{
+			merged_index = inner_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = inner_map->merged_indexes[i];
+		}
+	}
+
+	/* Fix the merged_indexes list using the mapping. */
+	foreach(lc, merged_indexes)
+	{
+		merged_index = lfirst_int(lc);
+		Assert(merged_index >= 0);
+		if (new_indexes[merged_index] >= 0)
+			lfirst_int(lc) = new_indexes[merged_index];
+	}
+
+	pfree(new_indexes);
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * This function produces the list pairs of partitions which produce merged
+ * partitions in the order of merged partition indexes.
+ *
+ * nmerged is the number of merged partitions.
+ *
+ * If successful, the list pairs are returned as two separate lists,
+ * outer_part_list and inner_part_list, one for each side.  Otherwise, those
+ * lists will be set to NIL.
+ */
+static void
+generate_matching_part_pairs(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 PartitionMap *outer_map, PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_part_list, List **inner_part_list)
+{
+	int			outer_nparts = outer_map->nparts;
+	int			inner_nparts = inner_map->nparts;
+	int		   *outer_part_array;
+	int		   *inner_part_array;
+	int			max_nparts;
+	int 		i;
+
+	Assert(outer_nparts == outer_rel->nparts);
+	Assert(inner_nparts == inner_rel->nparts);
+
+	outer_part_array = (int *) palloc(sizeof(int) * nmerged);
+	inner_part_array = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		outer_part_array[i] = inner_part_array[i] = -1;
+
+	/* Set pairs of matching partitions. */
+	max_nparts = Max(outer_nparts, inner_nparts);
+	for (i = 0; i < max_nparts; i++)
+	{
+		if (i < outer_nparts)
+		{
+			int 		merged_index = outer_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				outer_part_array[merged_index] = i;
+			}
+		}
+		if (i < inner_nparts)
+		{
+			int 		merged_index = inner_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				inner_part_array[merged_index] = i;
+			}
+		}
+	}
+
+	*outer_part_list = NIL;
+	*inner_part_list = NIL;
+
+	/* Build the list pairs. */
+	for (i = 0; i < nmerged; i++)
+	{
+		int			outer_part = outer_part_array[i];
+		int			inner_part = inner_part_array[i];
+
+		/*
+		 * If both partitions are dummy, it means the merged partition was
+		 * removed by partition re-merging in map_and_merge_partitions();
+		 * ignore the merged partition
+		 */
+		if (outer_part == -1 && inner_part == -1)
+			continue;
+
+		*outer_part_list = lappend(*outer_part_list, outer_part >= 0 ?
+								   outer_rel->part_rels[outer_part] : NULL);
+		*inner_part_list = lappend(*inner_part_list, inner_part >= 0 ?
+								   inner_rel->part_rels[inner_part] : NULL);
+	}
+
+	pfree(outer_part_array);
+	pfree(inner_part_array);
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * Merge default partitions from both sides, if any, and assign the default
+ * partition for the join result, if necessary.
+ *
+ * If both the relations have default partitions, try mapping those to each
+ * other. If the mapping succeeds corresponding merged partition will act as
+ * the default partition of the join result.
+ *
+ * If inner side of the join has default but not the outer side, rows in it
+ * won't appear in the join result. So don't create a default partition. If
+ * outer side of the join has default but not the inner side, rows in it will
+ * appear in the join result, so create a default merged partition.
+ */
+static bool
+merge_default_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 bool outer_has_default, int outer_default,
+						 bool inner_has_default, int inner_default,
+						 JoinType jointype, int *next_index,
+						 int *default_index)
+{
+	Assert(outer_has_default || inner_has_default);
+
+	if (outer_has_default && !inner_has_default)
+	{
+		if (IS_OUTER_JOIN(jointype))
+		{
+			int			merged_index;
+
+			Assert(jointype != JOIN_RIGHT);
+			Assert(outer_default >= 0 && outer_default < outer_map->nparts);
+			merged_index = outer_map->merged_indexes[outer_default];
+			if (merged_index == -1)
+			{
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(outer_map,
+															outer_default,
+															next_index);
+			}
+			else
+				Assert(*default_index == merged_index);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else if (!outer_has_default && inner_has_default)
+	{
+		if (jointype == JOIN_FULL)
+		{
+			int			merged_index;
+
+			Assert(inner_default >= 0 && inner_default < inner_map->nparts);
+			merged_index = inner_map->merged_indexes[inner_default];
+			if (merged_index == -1)
+			{
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(inner_map,
+															inner_default,
+															next_index);
+			}
+			else
+				Assert(*default_index == merged_index);
+		}
+		else
+			Assert(*default_index < 0);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		*default_index = map_and_merge_partitions(outer_map,
+												  inner_map,
+												  outer_default,
+												  inner_default,
+												  next_index);
+		if (*default_index == -1)
+			return false;
+	}
+
+	return true;
+}
+
+/*
+ * merge_null_partitions
+ *
+ * Merge NULL partitions, i.e. a partition that can hold NULL values for a list
+ * partitioned table, if any. Find the index of merged partition to which the
+ * NULL values would belong in the join result. If one joining relation has a
+ * NULL partition but not the other, try matching it with the default partition
+ * from the other relation since the default partition may have rows with NULL
+ * partition key. We can eliminate a NULL partition when it appears only on the
+ * inner side of the join and the outer side doesn't have a default partition.
+ *
+ * When the equality operator used for join is strict, two NULL values will not
+ * be considered as equal, and thus a NULL partition can be eliminated for an
+ * inner join. But we don't check the strictness operator here.
+ */
+static bool
+merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
+					  PartitionMap *outer_map, PartitionMap *inner_map,
+					  bool outer_has_default, bool inner_has_default,
+					  bool outer_has_null, bool inner_has_null,
+					  JoinType jointype, int *next_index,
+					  int *default_index, int *null_index)
+{
+	Assert(outer_has_null || inner_has_null);
+	Assert(*null_index == -1);
+
+	if (outer_has_null && !inner_has_null)
+	{
+		int			merged_index = -1;
+
+		/*
+		 * If the NULL partition was missing from the inner side of the join,
+		 * the partition of the join to which the NULL partition matches will
+		 * contain the NULL values and thus become the NULL partition of the
+		 * the join.
+		 */
+		if (inner_has_default || IS_OUTER_JOIN(jointype))
+		{
+			if (!process_outer_partition(outer_map,
+										 inner_map,
+										 outer_has_default,
+										 inner_has_default,
+										 outer_bi->null_index,
+										 inner_bi->default_index,
+										 jointype,
+										 next_index,
+										 default_index,
+										 &merged_index))
+				return false;
+		}
+		*null_index = merged_index;
+	}
+	else if (!outer_has_null && inner_has_null)
+	{
+		int			merged_index = -1;
+
+		/*
+		 * If the NULL partition was missing from the outer side of the join,
+		 * the partition of the join to which the NULL partition matches will
+		 * contain the NULL values and thus become the NULL partition of the
+		 * the join.
+		 */
+		if (outer_has_default || jointype == JOIN_FULL)
+		{
+			if (!process_inner_partition(outer_map,
+										 inner_map,
+										 outer_has_default,
+										 inner_has_default,
+										 inner_bi->null_index,
+										 outer_bi->default_index,
+										 jointype,
+										 next_index,
+										 default_index,
+										 &merged_index))
+				return false;
+		}
+		*null_index = merged_index;
+	}
+	else
+	{
+		/* Both the relations have NULL partitions, try merging them. */
+		*null_index = map_and_merge_partitions(outer_map,
+											   inner_map,
+											   outer_bi->null_index,
+											   inner_bi->null_index,
+											   next_index);
+		if (*null_index == -1)
+			return false;
+	}
+
+	return true;
+}
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index 6dab810d68f..abb0a0f80de 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -577,8 +577,10 @@ typedef struct PartitionSchemeData *PartitionScheme;
  *		part_scheme - Partitioning scheme of the relation
  *		nparts - Number of partitions
  *		boundinfo - Partition bounds
+ *		merged - true if partition bounds are merged ones
  *		partition_qual - Partition constraint if not the root
  *		part_rels - RelOptInfos for each partition
+ *		all_partrels - Relids set of all partition relids
  *		partexprs, nullable_partexprs - Partition key expressions
  *		partitioned_child_rels - RT indexes of unpruned partitions of
  *								 this relation that are partitioned tables
@@ -718,9 +720,12 @@ typedef struct RelOptInfo
 	PartitionScheme part_scheme;	/* Partitioning scheme. */
 	int			nparts;			/* number of partitions */
 	struct PartitionBoundInfoData *boundinfo;	/* Partition bounds */
+	bool		merged;			/* true if partition bounds were created by
+								 * partition_bounds_merge() */
 	List	   *partition_qual; /* partition constraint */
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of partitions,
 									 * stored in the same order of bounds */
+	Relids		all_partrels;	/* Relids set of all partition relids */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 	List	   *partitioned_child_rels; /* List of RT indexes. */
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index 0d0fd42b181..9292aa11e5e 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -16,6 +16,7 @@
 #include "nodes/pg_list.h"
 #include "partitioning/partdefs.h"
 #include "utils/relcache.h"
+struct RelOptInfo;				/* avoid including pathnodes.h here */
 
 
 /*
@@ -108,5 +109,11 @@ extern int	partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 										  int nvalues, Datum *values, bool *is_equal);
 extern int	partition_hash_bsearch(PartitionBoundInfo boundinfo,
 								   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   struct RelOptInfo *outer_rel, struct RelOptInfo *inner_rel,
+					   JoinType jointype,
+					   List **outer_parts, List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 8e7127e586d..815a6c90749 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -8,59 +8,86 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
                ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p1 t2_1
                ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
          ->  Hash Join
                Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(21 rows)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-(4 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+(7 rows)
 
 -- left outer join, with whole-row reference; partitionwise join does not apply
 EXPLAIN (COSTS OFF)
@@ -72,35 +99,50 @@ SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER
    ->  Hash Right Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(22 rows)
 
 SELECT t1, t2 FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-      t1      |      t2      
---------------+--------------
- (0,0,0000)   | (0,0,0000)
- (50,0,0050)  | 
- (100,0,0100) | 
- (150,0,0150) | (0,150,0150)
- (200,0,0200) | 
- (250,0,0250) | 
- (300,0,0300) | (0,300,0300)
- (350,0,0350) | 
- (400,0,0400) | 
- (450,0,0450) | (0,450,0450)
- (500,0,0500) | 
- (550,0,0550) | 
-(12 rows)
+       t1       |       t2       
+----------------+----------------
+ (-250,0,-0250) | 
+ (-200,0,-0200) | 
+ (-150,0,-0150) | (0,-150,-0150)
+ (-100,0,-0100) | 
+ (-50,0,-0050)  | 
+ (0,0,0000)     | (0,0,0000)
+ (50,0,0050)    | 
+ (100,0,0100)   | 
+ (150,0,0150)   | (0,150,0150)
+ (200,0,0200)   | 
+ (250,0,0250)   | 
+ (300,0,0300)   | (0,300,0300)
+ (350,0,0350)   | 
+ (400,0,0400)   | 
+ (450,0,0450)   | (0,450,0450)
+ (500,0,0500)   | 
+ (550,0,0550)   | 
+ (600,0,0600)   | (0,600,0600)
+ (650,0,0650)   | 
+ (700,0,0700)   | 
+ (750,0,0750)   | (0,750,0750)
+(21 rows)
 
 -- right outer join
 EXPLAIN (COSTS OFF)
@@ -112,35 +154,53 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHE
    ->  Append
          ->  Hash Right Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Right Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
                      Filter: (a = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t2_2.b)
-(20 rows)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(32 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |   0 | 0000
- 150 | 0150 | 150 | 0150
- 300 | 0300 | 300 | 0300
- 450 | 0450 | 450 | 0450
-     |      |  75 | 0075
-     |      | 225 | 0225
-     |      | 375 | 0375
-     |      | 525 | 0525
-(8 rows)
+  a   |   c   |  b   |   c   
+------+-------+------+-------
+ -150 | -0150 | -150 | -0150
+    0 | 0000  |    0 | 0000
+  150 | 0150  |  150 | 0150
+  300 | 0300  |  300 | 0300
+  450 | 0450  |  450 | 0450
+  600 | 0600  |  600 | 0600
+  750 | 0750  |  750 | 0750
+      |       | -225 | -0225
+      |       |  -75 | -0075
+      |       |   75 | 0075
+      |       |  225 | 0225
+      |       |  375 | 0375
+      |       |  525 | 0525
+      |       |  675 | 0675
+(14 rows)
 
 -- full outer join, with placeholder vars
 EXPLAIN (COSTS OFF)
@@ -153,28 +213,44 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0)
          ->  Hash Full Join
                Hash Cond: (prt1.a = prt2.b)
                Filter: (((50) = prt1.a) OR ((75) = prt2.b))
-               ->  Seq Scan on prt1_p1 prt1
+               ->  Seq Scan on prt1_p0 prt1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 prt2
+                     ->  Seq Scan on prt2_p0 prt2
                            Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_1.a = prt2_1.b)
                Filter: (((50) = prt1_1.a) OR ((75) = prt2_1.b))
-               ->  Seq Scan on prt1_p2 prt1_1
+               ->  Seq Scan on prt1_p1 prt1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 prt2_1
+                     ->  Seq Scan on prt2_p1 prt2_1
                            Filter: (a = 0)
          ->  Hash Full Join
                Hash Cond: (prt1_2.a = prt2_2.b)
                Filter: (((50) = prt1_2.a) OR ((75) = prt2_2.b))
-               ->  Seq Scan on prt1_p3 prt1_2
+               ->  Seq Scan on prt1_p2 prt1_2
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p3 prt2_2
+                     ->  Seq Scan on prt2_p2 prt2_2
                            Filter: (a = 0)
-(27 rows)
+         ->  Hash Full Join
+               Hash Cond: (prt1_3.a = prt2_3.b)
+               Filter: (((50) = prt1_3.a) OR ((75) = prt2_3.b))
+               ->  Seq Scan on prt1_p3 prt1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 prt2_3
+                           Filter: (a = 0)
+         ->  Hash Full Join
+               Hash Cond: (prt1_4.a = prt2_4.b)
+               Filter: (((50) = prt1_4.a) OR ((75) = prt2_4.b))
+               ->  Seq Scan on prt1_p4 prt1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 prt2_4
+                           Filter: (a = 0)
+(43 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
  a  |  c   | b  |  c   
@@ -219,27 +295,36 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JO
                      Filter: (b > 250)
                ->  Seq Scan on prt2_p3 prt2_1
                      Filter: (b > 250)
+               ->  Seq Scan on prt2_p4 prt2_2
+                     Filter: (b > 250)
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 prt1
+                     ->  Seq Scan on prt1_p0 prt1
                            Filter: ((a < 450) AND (b = 0))
-                     ->  Seq Scan on prt1_p2 prt1_1
+                     ->  Seq Scan on prt1_p1 prt1_1
                            Filter: ((a < 450) AND (b = 0))
-(15 rows)
+                     ->  Seq Scan on prt1_p2 prt1_2
+                           Filter: ((a < 450) AND (b = 0))
+(19 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-(9 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+(14 rows)
 
 -- Currently we can't do partitioned join if nullable-side partitions are pruned
 EXPLAIN (COSTS OFF)
@@ -252,9 +337,11 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
          Hash Cond: (prt1.a = prt2.b)
          Filter: ((prt1.b = 0) OR (prt2.a = 0))
          ->  Append
-               ->  Seq Scan on prt1_p1 prt1
+               ->  Seq Scan on prt1_p0 prt1
+                     Filter: (a < 450)
+               ->  Seq Scan on prt1_p1 prt1_1
                      Filter: (a < 450)
-               ->  Seq Scan on prt1_p2 prt1_1
+               ->  Seq Scan on prt1_p2 prt1_2
                      Filter: (a < 450)
          ->  Hash
                ->  Append
@@ -262,64 +349,147 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JO
                            Filter: (b > 250)
                      ->  Seq Scan on prt2_p3 prt2_1
                            Filter: (b > 250)
-(16 rows)
+                     ->  Seq Scan on prt2_p4 prt2_2
+                           Filter: (b > 250)
+(20 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 OR t2.a = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   
------+------+-----+------
-   0 | 0000 |     | 
-  50 | 0050 |     | 
- 100 | 0100 |     | 
- 150 | 0150 |     | 
- 200 | 0200 |     | 
- 250 | 0250 |     | 
- 300 | 0300 | 300 | 0300
- 350 | 0350 |     | 
- 400 | 0400 |     | 
-     |      | 375 | 0375
-     |      | 450 | 0450
-     |      | 525 | 0525
-(12 rows)
+  a   |   c   |  b  |  c   
+------+-------+-----+------
+ -250 | -0250 |     | 
+ -200 | -0200 |     | 
+ -150 | -0150 |     | 
+ -100 | -0100 |     | 
+  -50 | -0050 |     | 
+    0 | 0000  |     | 
+   50 | 0050  |     | 
+  100 | 0100  |     | 
+  150 | 0150  |     | 
+  200 | 0200  |     | 
+  250 | 0250  |     | 
+  300 | 0300  | 300 | 0300
+  350 | 0350  |     | 
+  400 | 0400  |     | 
+      |       | 375 | 0375
+      |       | 450 | 0450
+      |       | 525 | 0525
+      |       | 600 | 0600
+      |       | 675 | 0675
+      |       | 750 | 0750
+(20 rows)
 
 -- Semi-join
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                           
+---------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Hash Semi Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
                            Filter: (a = 0)
          ->  Hash Semi Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                            Filter: (a = 0)
-         ->  Nested Loop Semi Join
-               Join Filter: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
-               ->  Materialize
-                     ->  Seq Scan on prt2_p3 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
                            Filter: (a = 0)
-(24 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+(39 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.b = t2.a)
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t2
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.b = t2_1.a)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t2_1
+                           Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.b = t2_2.a)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t2_2
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: (t1_3.b = t2_3.a)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt1_p3 t2_3
+                     Filter: (b = 0)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.b = t2_4.a)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t2_4
+                           Filter: (b = 0)
+(37 rows)
+
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
 
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
@@ -330,27 +500,82 @@ SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS
    ->  Append
          ->  Hash Anti Join
                Hash Cond: (t1.a = t2.b)
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Hash Anti Join
                Hash Cond: (t1_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Hash Anti Join
                Hash Cond: (t1_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
-(17 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(27 rows)
 
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
-  sum  |         avg          | sum  |         avg         
--------+----------------------+------+---------------------
- 60000 | 300.0000000000000000 | 2400 | 12.0000000000000000
+  sum  |         avg          | sum  |        avg         
+-------+----------------------+------+--------------------
+ 95550 | 273.0000000000000000 | 2200 | 6.2857142857142857
 (1 row)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Append
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p0 t1
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+               Index Cond: (a = t1.b)
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.b = t2_1.a)
+         ->  Seq Scan on prt2_p1 t1_1
+               Filter: (a = 0)
+         ->  Hash
+               ->  Seq Scan on prt1_p1 t2_1
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p2 t1_2
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+               Index Cond: (a = t1_2.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p3 t1_3
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+               Index Cond: (a = t1_3.b)
+   ->  Nested Loop Anti Join
+         ->  Seq Scan on prt2_p4 t1_4
+               Filter: (a = 0)
+         ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+               Index Cond: (a = t1_4.b)
+(27 rows)
+
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+  b   |   c   
+------+-------
+ -225 | -0225
+  -75 | -0075
+   75 | 0075
+  225 | 0225
+  375 | 0375
+  525 | 0525
+  675 | 0675
+(7 rows)
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -362,49 +587,74 @@ SELECT * FROM prt1 t1 LEFT JOIN LATERAL
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2
+                     ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
                            Index Cond: (a = t1.a)
-                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3
+                     ->  Index Scan using iprt2_p0_b on prt2_p0 t3
                            Index Cond: (b = t2.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_1
+                     ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
                            Index Cond: (a = t1_1.a)
-                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_1
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 t3_1
                            Index Cond: (b = t2_1.a)
          ->  Nested Loop Left Join
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
                ->  Nested Loop
-                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_2
+                     ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
                            Index Cond: (a = t1_2.a)
-                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_2
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 t3_2
                            Index Cond: (b = t2_2.a)
-(27 rows)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                           Index Cond: (a = t1_3.a)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t3_3
+                           Index Cond: (b = t2_3.a)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Nested Loop
+                     ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                           Index Cond: (a = t1_4.a)
+                     ->  Index Scan using iprt2_p4_b on prt2_p4 t3_4
+                           Index Cond: (b = t2_4.a)
+(43 rows)
 
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.b) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.a = ss.t2a WHERE t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   | t2a | t3a | least 
------+---+------+-----+-----+-------
-   0 | 0 | 0000 |   0 |   0 |     0
-  50 | 0 | 0050 |     |     |      
- 100 | 0 | 0100 |     |     |      
- 150 | 0 | 0150 | 150 |   0 |   150
- 200 | 0 | 0200 |     |     |      
- 250 | 0 | 0250 |     |     |      
- 300 | 0 | 0300 | 300 |   0 |   300
- 350 | 0 | 0350 |     |     |      
- 400 | 0 | 0400 |     |     |      
- 450 | 0 | 0450 | 450 |   0 |   450
- 500 | 0 | 0500 |     |     |      
- 550 | 0 | 0550 |     |     |      
-(12 rows)
+  a   | b |   c   | t2a  | t3a | least 
+------+---+-------+------+-----+-------
+ -250 | 0 | -0250 |      |     |      
+ -200 | 0 | -0200 |      |     |      
+ -150 | 0 | -0150 | -150 |   0 |  -150
+ -100 | 0 | -0100 |      |     |      
+  -50 | 0 | -0050 |      |     |      
+    0 | 0 | 0000  |    0 |   0 |     0
+   50 | 0 | 0050  |      |     |      
+  100 | 0 | 0100  |      |     |      
+  150 | 0 | 0150  |  150 |   0 |   150
+  200 | 0 | 0200  |      |     |      
+  250 | 0 | 0250  |      |     |      
+  300 | 0 | 0300  |  300 |   0 |   300
+  350 | 0 | 0350  |      |     |      
+  400 | 0 | 0400  |      |     |      
+  450 | 0 | 0450  |  450 |   0 |   450
+  500 | 0 | 0500  |      |     |      
+  550 | 0 | 0550  |      |     |      
+  600 | 0 | 0600  |  600 |   0 |   600
+  650 | 0 | 0650  |      |     |      
+  700 | 0 | 0700  |      |     |      
+  750 | 0 | 0750  |  750 |   0 |   750
+(21 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
@@ -418,46 +668,67 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
          Hash Cond: ((t1.c)::text = (t2.c)::text)
          Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
          ->  Append
-               ->  Seq Scan on prt1_p1 t1
-               ->  Seq Scan on prt1_p2 t1_1
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
          ->  Hash
                ->  Append
                      ->  Hash Join
                            Hash Cond: (t2.a = t3.b)
-                           ->  Seq Scan on prt1_p1 t2
+                           ->  Seq Scan on prt1_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t3
+                                 ->  Seq Scan on prt2_p0 t3
                      ->  Hash Join
                            Hash Cond: (t2_1.a = t3_1.b)
-                           ->  Seq Scan on prt1_p2 t2_1
+                           ->  Seq Scan on prt1_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t3_1
+                                 ->  Seq Scan on prt2_p1 t3_1
                      ->  Hash Join
                            Hash Cond: (t2_2.a = t3_2.b)
-                           ->  Seq Scan on prt1_p3 t2_2
+                           ->  Seq Scan on prt1_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p3 t3_2
-(26 rows)
+                                 ->  Seq Scan on prt2_p2 t3_2
+                     ->  Hash Join
+                           Hash Cond: (t2_3.a = t3_3.b)
+                           ->  Seq Scan on prt1_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p3 t3_3
+                     ->  Hash Join
+                           Hash Cond: (t2_4.a = t3_4.b)
+                           ->  Seq Scan on prt1_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t3_4
+(38 rows)
 
 SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t3.a AS t3a, t2.b t2b, t2.c t2c, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
 			  ON t1.c = ss.t2c WHERE (t1.b + coalesce(ss.t2b, 0)) = 0 ORDER BY t1.a;
-  a  | t2a | t2c  
------+-----+------
-   0 |   0 | 0000
-  50 |     | 
- 100 |     | 
- 150 | 150 | 0150
- 200 |     | 
- 250 |     | 
- 300 | 300 | 0300
- 350 |     | 
- 400 |     | 
- 450 | 450 | 0450
- 500 |     | 
- 550 |     | 
-(12 rows)
+  a   | t2a  |  t2c  
+------+------+-------
+ -250 |      | 
+ -200 |      | 
+ -150 | -150 | -0150
+ -100 |      | 
+  -50 |      | 
+    0 |    0 | 0000
+   50 |      | 
+  100 |      | 
+  150 |  150 | 0150
+  200 |      | 
+  250 |      | 
+  300 |  300 | 0300
+  350 |      | 
+  400 |      | 
+  450 |  450 | 0450
+  500 |      | 
+  550 |      | 
+  600 |  600 | 0600
+  650 |      | 
+  700 |      | 
+  750 |  750 | 0750
+(21 rows)
 
 -- bug with inadequate sort key representation
 SET enable_partitionwise_aggregate TO true;
@@ -481,10 +752,10 @@ SELECT a, b FROM prt1 FULL JOIN prt2 p2(b,a,c) USING(a,b)
                            Filter: ((COALESCE(prt1.a, p2.a) >= 490) AND (COALESCE(prt1.a, p2.a) <= 510))
                            ->  Sort
                                  Sort Key: prt1.a, prt1.b
-                                 ->  Seq Scan on prt1_p1 prt1
+                                 ->  Seq Scan on prt1_p0 prt1
                            ->  Sort
                                  Sort Key: p2.a, p2.b
-                                 ->  Seq Scan on prt2_p1 p2
+                                 ->  Seq Scan on prt2_p0 p2
          ->  Group
                Group Key: (COALESCE(prt1_1.a, p2_1.a)), (COALESCE(prt1_1.b, p2_1.b))
                ->  Sort
@@ -494,10 +765,10 @@ SELECT a, b FROM prt1 FULL JOIN prt2 p2(b,a,c) USING(a,b)
                            Filter: ((COALESCE(prt1_1.a, p2_1.a) >= 490) AND (COALESCE(prt1_1.a, p2_1.a) <= 510))
                            ->  Sort
                                  Sort Key: prt1_1.a, prt1_1.b
-                                 ->  Seq Scan on prt1_p2 prt1_1
+                                 ->  Seq Scan on prt1_p1 prt1_1
                            ->  Sort
                                  Sort Key: p2_1.a, p2_1.b
-                                 ->  Seq Scan on prt2_p2 p2_1
+                                 ->  Seq Scan on prt2_p1 p2_1
          ->  Group
                Group Key: (COALESCE(prt1_2.a, p2_2.a)), (COALESCE(prt1_2.b, p2_2.b))
                ->  Sort
@@ -507,11 +778,37 @@ SELECT a, b FROM prt1 FULL JOIN prt2 p2(b,a,c) USING(a,b)
                            Filter: ((COALESCE(prt1_2.a, p2_2.a) >= 490) AND (COALESCE(prt1_2.a, p2_2.a) <= 510))
                            ->  Sort
                                  Sort Key: prt1_2.a, prt1_2.b
-                                 ->  Seq Scan on prt1_p3 prt1_2
+                                 ->  Seq Scan on prt1_p2 prt1_2
                            ->  Sort
                                  Sort Key: p2_2.a, p2_2.b
-                                 ->  Seq Scan on prt2_p3 p2_2
-(43 rows)
+                                 ->  Seq Scan on prt2_p2 p2_2
+         ->  Group
+               Group Key: (COALESCE(prt1_3.a, p2_3.a)), (COALESCE(prt1_3.b, p2_3.b))
+               ->  Sort
+                     Sort Key: (COALESCE(prt1_3.a, p2_3.a)), (COALESCE(prt1_3.b, p2_3.b))
+                     ->  Merge Full Join
+                           Merge Cond: ((prt1_3.a = p2_3.a) AND (prt1_3.b = p2_3.b))
+                           Filter: ((COALESCE(prt1_3.a, p2_3.a) >= 490) AND (COALESCE(prt1_3.a, p2_3.a) <= 510))
+                           ->  Sort
+                                 Sort Key: prt1_3.a, prt1_3.b
+                                 ->  Seq Scan on prt1_p3 prt1_3
+                           ->  Sort
+                                 Sort Key: p2_3.a, p2_3.b
+                                 ->  Seq Scan on prt2_p3 p2_3
+         ->  Group
+               Group Key: (COALESCE(prt1_4.a, p2_4.a)), (COALESCE(prt1_4.b, p2_4.b))
+               ->  Sort
+                     Sort Key: (COALESCE(prt1_4.a, p2_4.a)), (COALESCE(prt1_4.b, p2_4.b))
+                     ->  Merge Full Join
+                           Merge Cond: ((prt1_4.a = p2_4.a) AND (prt1_4.b = p2_4.b))
+                           Filter: ((COALESCE(prt1_4.a, p2_4.a) >= 490) AND (COALESCE(prt1_4.a, p2_4.a) <= 510))
+                           ->  Sort
+                                 Sort Key: prt1_4.a, prt1_4.b
+                                 ->  Seq Scan on prt1_p4 prt1_4
+                           ->  Sort
+                                 Sort Key: p2_4.a, p2_4.b
+                                 ->  Seq Scan on prt2_p4 p2_4
+(69 rows)
 
 SELECT a, b FROM prt1 FULL JOIN prt2 p2(b,a,c) USING(a,b)
   WHERE a BETWEEN 490 AND 510
@@ -540,19 +837,29 @@ RESET enable_hashjoin;
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
@@ -563,32 +870,49 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 =
    ->  Append
          ->  Hash Join
                Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
-               ->  Seq Scan on prt2_e_p1 t2
+               ->  Seq Scan on prt2_e_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p0 t1
+                           Filter: (c = 0)
+         ->  Hash Join
+               Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+               ->  Seq Scan on prt1_e_p1 t1_1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_e_p1 t2_1
+         ->  Hash Join
+               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
+               ->  Seq Scan on prt2_e_p2 t2_2
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 t1
+                     ->  Seq Scan on prt1_e_p2 t1_2
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
-               ->  Seq Scan on prt2_e_p2 t2_1
+               Hash Cond: (((t2_3.b + t2_3.a) / 2) = ((t1_3.a + t1_3.b) / 2))
+               ->  Seq Scan on prt2_e_p3 t2_3
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 t1_1
+                     ->  Seq Scan on prt1_e_p3 t1_3
                            Filter: (c = 0)
          ->  Hash Join
-               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
-               ->  Seq Scan on prt2_e_p3 t2_2
+               Hash Cond: (((t2_4.b + t2_4.a) / 2) = ((t1_4.a + t1_4.b) / 2))
+               ->  Seq Scan on prt2_e_p4 t2_4
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 t1_2
+                     ->  Seq Scan on prt1_e_p4 t1_4
                            Filter: (c = 0)
-(21 rows)
+(33 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1, prt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.c = 0 ORDER BY t1.a, t2.b;
-  a  | c |  b  | c 
------+---+-----+---
-   0 | 0 |   0 | 0
- 150 | 0 | 150 | 0
- 300 | 0 | 300 | 0
- 450 | 0 | 450 | 0
-(4 rows)
+  a   | c |  b   | c 
+------+---+------+---
+ -250 | 0 | -250 | 0
+ -100 | 0 | -100 | 0
+    0 | 0 |    0 | 0
+    0 | 0 |    0 | 0
+  150 | 0 |  150 | 0
+  300 | 0 |  300 | 0
+  450 | 0 |  450 | 0
+  600 | 0 |  600 | 0
+  750 | 0 |  750 | 0
+(9 rows)
 
 --
 -- N-way join
@@ -601,154 +925,232 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = ((t3.a + t3.b) / 2))
+               Join Filter: (t1.a = t2.b)
                ->  Hash Join
-                     Hash Cond: (t2.b = t1.a)
-                     ->  Seq Scan on prt2_p1 t2
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on prt1_e_p0 t3
                      ->  Hash
-                           ->  Seq Scan on prt1_p1 t1
+                           ->  Seq Scan on prt1_p0 t1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p1_ab2 on prt1_e_p1 t3
-                     Index Cond: (((a + b) / 2) = t2.b)
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
+                     Index Cond: (b = ((t3.a + t3.b) / 2))
          ->  Nested Loop
                Join Filter: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_1.b = t1_1.a)
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on prt1_p2 t1_1
+                           ->  Seq Scan on prt1_p1 t1_1
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_1
+               ->  Index Scan using iprt1_e_p4_ab2 on prt1_e_p1 t3_1
                      Index Cond: (((a + b) / 2) = t2_1.b)
          ->  Nested Loop
                Join Filter: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
                ->  Hash Join
                      Hash Cond: (t2_2.b = t1_2.a)
-                     ->  Seq Scan on prt2_p3 t2_2
+                     ->  Seq Scan on prt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on prt1_p3 t1_2
+                           ->  Seq Scan on prt1_p2 t1_2
                                  Filter: (b = 0)
-               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_2
+               ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t3_2
                      Index Cond: (((a + b) / 2) = t2_2.b)
-(33 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = ((t3_3.a + t3_3.b) / 2))
+               ->  Nested Loop
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t2_3.b)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t2_4.b)
+               ->  Hash Join
+                     Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+                     ->  Seq Scan on prt1_e_p4 t3_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_p4 t1_4
+                                 Filter: (b = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = ((t3_4.a + t3_4.b) / 2))
+(52 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM prt1 t1, prt2 t2, prt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
-(4 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(8 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                          QUERY PLAN                          
---------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Hash Right Join
                Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
-               ->  Seq Scan on prt1_e_p1 t3
+               ->  Seq Scan on prt1_e_p0 t3
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2.b = t1.a)
-                           ->  Seq Scan on prt2_p1 t2
+                           ->  Seq Scan on prt2_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_1.a)
-               ->  Seq Scan on prt1_e_p2 t3_1
+               ->  Seq Scan on prt1_e_p1 t3_1
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_1.b = t1_1.a)
-                           ->  Seq Scan on prt2_p2 t2_1
+                           ->  Seq Scan on prt2_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                                        Filter: (b = 0)
          ->  Hash Right Join
                Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_2.a)
-               ->  Seq Scan on prt1_e_p3 t3_2
+               ->  Seq Scan on prt1_e_p2 t3_2
                ->  Hash
                      ->  Hash Right Join
                            Hash Cond: (t2_2.b = t1_2.a)
-                           ->  Seq Scan on prt2_p3 t2_2
+                           ->  Seq Scan on prt2_p2 t2_2
                            ->  Hash
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                                        Filter: (b = 0)
-(33 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                           Index Cond: (b = t1_3.a)
+               ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t3_3
+                     Index Cond: (((a + b) / 2) = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (((t3_4.a + t3_4.b) / 2) = t1_4.a)
+               ->  Seq Scan on prt1_e_p4 t3_4
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_4.b = t1_4.a)
+                           ->  Seq Scan on prt2_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt1_p4 t1_4
+                                       Filter: (b = 0)
+(51 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) LEFT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-                            QUERY PLAN                             
--------------------------------------------------------------------
+                             QUERY PLAN                              
+---------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.b, ((t3.a + t3.b))
    ->  Append
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p1 t3
+                           ->  Seq Scan on prt1_e_p0 t3
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p1_b on prt2_p1 t2
+               ->  Index Scan using iprt2_p0_b on prt2_p0 t2
                      Index Cond: (b = t1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_1.a = ((t3_1.a + t3_1.b) / 2))
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p2 t3_1
+                           ->  Seq Scan on prt1_e_p1 t3_1
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
+               ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
                      Index Cond: (b = t1_1.a)
          ->  Nested Loop Left Join
                ->  Hash Right Join
                      Hash Cond: (t1_2.a = ((t3_2.a + t3_2.b) / 2))
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                      ->  Hash
-                           ->  Seq Scan on prt1_e_p3 t3_2
+                           ->  Seq Scan on prt1_e_p2 t3_2
                                  Filter: (c = 0)
-               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+               ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
                      Index Cond: (b = t1_2.a)
-(30 rows)
+         ->  Nested Loop Left Join
+               ->  Nested Loop Left Join
+                     ->  Seq Scan on prt1_e_p3 t3_3
+                           Filter: (c = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                           Index Cond: (a = ((t3_3.a + t3_3.b) / 2))
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Nested Loop Left Join
+               ->  Hash Right Join
+                     Hash Cond: (t1_4.a = ((t3_4.a + t3_4.b) / 2))
+                     ->  Seq Scan on prt1_p4 t1_4
+                     ->  Hash
+                           ->  Seq Scan on prt1_e_p4 t3_4
+                                 Filter: (c = 0)
+               ->  Index Scan using iprt2_p4_b on prt2_p4 t2_4
+                     Index Cond: (b = t1_4.a)
+(47 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- Cases with non-nullable expressions in subquery results;
 -- make sure these go to null as expected
@@ -764,41 +1166,67 @@ SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * F
                Filter: ((prt1.a = (50)) OR (prt2.b = (75)) OR (((prt1_e.a + prt1_e.b) / 2) = (50)))
                ->  Hash Full Join
                      Hash Cond: (prt1.a = prt2.b)
-                     ->  Seq Scan on prt1_p1 prt1
+                     ->  Seq Scan on prt1_p0 prt1
                            Filter: (b = 0)
                      ->  Hash
-                           ->  Seq Scan on prt2_p1 prt2
+                           ->  Seq Scan on prt2_p0 prt2
                                  Filter: (a = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p1 prt1_e
+                     ->  Seq Scan on prt1_e_p0 prt1_e
                            Filter: (c = 0)
          ->  Hash Full Join
-               Hash Cond: (prt1_1.a = ((prt1_e_1.a + prt1_e_1.b) / 2))
+               Hash Cond: (((prt1_e_1.a + prt1_e_1.b) / 2) = prt1_1.a)
                Filter: ((prt1_1.a = (50)) OR (prt2_1.b = (75)) OR (((prt1_e_1.a + prt1_e_1.b) / 2) = (50)))
+               ->  Seq Scan on prt1_e_p1 prt1_e_1
+                     Filter: (c = 0)
+               ->  Hash
+                     ->  Hash Full Join
+                           Hash Cond: (prt1_1.a = prt2_1.b)
+                           ->  Seq Scan on prt1_p1 prt1_1
+                                 Filter: (b = 0)
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1 prt2_1
+                                       Filter: (a = 0)
+         ->  Hash Full Join
+               Hash Cond: (prt1_2.a = ((prt1_e_2.a + prt1_e_2.b) / 2))
+               Filter: ((prt1_2.a = (50)) OR (prt2_2.b = (75)) OR (((prt1_e_2.a + prt1_e_2.b) / 2) = (50)))
                ->  Hash Full Join
-                     Hash Cond: (prt1_1.a = prt2_1.b)
-                     ->  Seq Scan on prt1_p2 prt1_1
+                     Hash Cond: (prt1_2.a = prt2_2.b)
+                     ->  Seq Scan on prt1_p2 prt1_2
                            Filter: (b = 0)
                      ->  Hash
-                           ->  Seq Scan on prt2_p2 prt2_1
+                           ->  Seq Scan on prt2_p2 prt2_2
                                  Filter: (a = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p2 prt1_e_1
+                     ->  Seq Scan on prt1_e_p2 prt1_e_2
                            Filter: (c = 0)
          ->  Hash Full Join
-               Hash Cond: (prt1_2.a = ((prt1_e_2.a + prt1_e_2.b) / 2))
-               Filter: ((prt1_2.a = (50)) OR (prt2_2.b = (75)) OR (((prt1_e_2.a + prt1_e_2.b) / 2) = (50)))
+               Hash Cond: (prt1_3.a = ((prt1_e_3.a + prt1_e_3.b) / 2))
+               Filter: ((prt1_3.a = (50)) OR (prt2_3.b = (75)) OR (((prt1_e_3.a + prt1_e_3.b) / 2) = (50)))
                ->  Hash Full Join
-                     Hash Cond: (prt1_2.a = prt2_2.b)
-                     ->  Seq Scan on prt1_p3 prt1_2
+                     Hash Cond: (prt1_3.a = prt2_3.b)
+                     ->  Seq Scan on prt1_p3 prt1_3
+                           Filter: (b = 0)
+                     ->  Hash
+                           ->  Seq Scan on prt2_p3 prt2_3
+                                 Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_e_p3 prt1_e_3
+                           Filter: (c = 0)
+         ->  Hash Full Join
+               Hash Cond: (prt1_4.a = ((prt1_e_4.a + prt1_e_4.b) / 2))
+               Filter: ((prt1_4.a = (50)) OR (prt2_4.b = (75)) OR (((prt1_e_4.a + prt1_e_4.b) / 2) = (50)))
+               ->  Hash Full Join
+                     Hash Cond: (prt1_4.a = prt2_4.b)
+                     ->  Seq Scan on prt1_p4 prt1_4
                            Filter: (b = 0)
                      ->  Hash
-                           ->  Seq Scan on prt2_p3 prt2_2
+                           ->  Seq Scan on prt2_p4 prt2_4
                                  Filter: (a = 0)
                ->  Hash
-                     ->  Seq Scan on prt1_e_p3 prt1_e_2
+                     ->  Seq Scan on prt1_e_p4 prt1_e_4
                            Filter: (c = 0)
-(42 rows)
+(68 rows)
 
 SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.b = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.a = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.c = 0) t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a = t1.phv OR t2.b = t2.phv OR (t3.a + t3.b)/2 = t3.phv ORDER BY t1.a, t2.b, t3.a + t3.b;
  a  | phv | b  | phv | ?column? | phv 
@@ -816,172 +1244,260 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHER
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
-               Join Filter: (t1.a = t1_3.b)
+               Join Filter: (t1.a = t1_5.b)
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Join
-                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
-                           ->  Seq Scan on prt1_e_p1 t2
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on prt1_e_p0 t2
                            ->  Hash
-                                 ->  Seq Scan on prt2_p1 t1_3
+                                 ->  Seq Scan on prt2_p0 t1_5
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
                      Index Cond: (a = ((t2.a + t2.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_1.a = t1_4.b)
+               Join Filter: (t1_1.a = t1_6.b)
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Join
-                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
-                           ->  Seq Scan on prt1_e_p2 t2_1
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_6.b)
+                           ->  Seq Scan on prt1_e_p1 t2_1
                            ->  Hash
-                                 ->  Seq Scan on prt2_p2 t1_4
+                                 ->  Seq Scan on prt2_p1 t1_6
                                        Filter: (a = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
                      Index Cond: (a = ((t2_1.a + t2_1.b) / 2))
                      Filter: (b = 0)
          ->  Nested Loop
-               Join Filter: (t1_2.a = t1_5.b)
+               Join Filter: (t1_2.a = t1_7.b)
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Nested Loop
-                           ->  Seq Scan on prt2_p3 t1_5
+                           ->  Seq Scan on prt2_p2 t1_7
                                  Filter: (a = 0)
-                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_2
-                                 Index Cond: (((a + b) / 2) = t1_5.b)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
+                           ->  Index Scan using iprt1_e_p2_ab2 on prt1_e_p2 t2_2
+                                 Index Cond: (((a + b) / 2) = t1_7.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
                      Index Cond: (a = ((t2_2.a + t2_2.b) / 2))
                      Filter: (b = 0)
-(41 rows)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t1_8.b)
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Nested Loop
+                           ->  Seq Scan on prt2_p3 t1_8
+                                 Filter: (a = 0)
+                           ->  Index Scan using iprt1_e_p3_ab2 on prt1_e_p3 t2_3
+                                 Index Cond: (((a + b) / 2) = t1_8.b)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = ((t2_3.a + t2_3.b) / 2))
+                     Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: (t1_4.a = t1_9.b)
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Join
+                           Hash Cond: (((t2_4.a + t2_4.b) / 2) = t1_9.b)
+                           ->  Seq Scan on prt1_e_p4 t2_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t1_9
+                                       Filter: (a = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = ((t2_4.a + t2_4.b) / 2))
+                     Filter: (b = 0)
+(66 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.a = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                               QUERY PLAN                                
--------------------------------------------------------------------------
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
  Sort
    Sort Key: t1.a
    ->  Append
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_3.b
+                     Group Key: t1_5.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
-                           ->  Seq Scan on prt2_p1 t1_3
+                           Hash Cond: (t1_5.b = ((t1_10.a + t1_10.b) / 2))
+                           ->  Seq Scan on prt2_p0 t1_5
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p1 t1_6
+                                 ->  Seq Scan on prt1_e_p0 t1_10
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p1_a on prt1_p1 t1
-                     Index Cond: (a = t1_3.b)
+               ->  Index Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t1_5.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_4.b
+                     Group Key: t1_6.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
-                           ->  Seq Scan on prt2_p2 t1_4
+                           Hash Cond: (t1_6.b = ((t1_11.a + t1_11.b) / 2))
+                           ->  Seq Scan on prt2_p1 t1_6
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p2 t1_7
+                                 ->  Seq Scan on prt1_e_p1 t1_11
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_1
-                     Index Cond: (a = t1_4.b)
+               ->  Index Scan using iprt1_p1_a on prt1_p1 t1_1
+                     Index Cond: (a = t1_6.b)
                      Filter: (b = 0)
          ->  Nested Loop
                ->  HashAggregate
-                     Group Key: t1_5.b
+                     Group Key: t1_7.b
                      ->  Hash Semi Join
-                           Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
-                           ->  Seq Scan on prt2_p3 t1_5
+                           Hash Cond: (t1_7.b = ((t1_12.a + t1_12.b) / 2))
+                           ->  Seq Scan on prt2_p2 t1_7
                            ->  Hash
-                                 ->  Seq Scan on prt1_e_p3 t1_8
+                                 ->  Seq Scan on prt1_e_p2 t1_12
                                        Filter: (c = 0)
-               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_2
-                     Index Cond: (a = t1_5.b)
+               ->  Index Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t1_7.b)
                      Filter: (b = 0)
-(39 rows)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_8.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_8.b = ((t1_13.a + t1_13.b) / 2))
+                           ->  Seq Scan on prt2_p3 t1_8
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p3 t1_13
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t1_8.b)
+                     Filter: (b = 0)
+         ->  Nested Loop
+               ->  HashAggregate
+                     Group Key: t1_9.b
+                     ->  Hash Semi Join
+                           Hash Cond: (t1_9.b = ((t1_14.a + t1_14.b) / 2))
+                           ->  Seq Scan on prt2_p4 t1_9
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p4 t1_14
+                                       Filter: (c = 0)
+               ->  Index Scan using iprt1_p4_a on prt1_p4 t1_4
+                     Index Cond: (a = t1_9.b)
+                     Filter: (b = 0)
+(63 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 -- test merge joins
 SET enable_hashjoin TO off;
 SET enable_nestloop TO off;
 EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-                           QUERY PLAN                           
-----------------------------------------------------------------
+                            QUERY PLAN                            
+------------------------------------------------------------------
  Merge Append
    Sort Key: t1.a
    ->  Merge Semi Join
-         Merge Cond: (t1.a = t1_3.b)
+         Merge Cond: (t1.a = t1_5.b)
          ->  Sort
                Sort Key: t1.a
-               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p0 t1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+               Merge Cond: (t1_5.b = (((t1_10.a + t1_10.b) / 2)))
                ->  Sort
-                     Sort Key: t1_3.b
-                     ->  Seq Scan on prt2_p1 t1_3
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on prt2_p0 t1_5
                ->  Sort
-                     Sort Key: (((t1_6.a + t1_6.b) / 2))
-                     ->  Seq Scan on prt1_e_p1 t1_6
+                     Sort Key: (((t1_10.a + t1_10.b) / 2))
+                     ->  Seq Scan on prt1_e_p0 t1_10
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_1.a = t1_4.b)
+         Merge Cond: (t1_1.a = t1_6.b)
          ->  Sort
                Sort Key: t1_1.a
-               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p1 t1_1
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+               Merge Cond: (t1_6.b = (((t1_11.a + t1_11.b) / 2)))
                ->  Sort
-                     Sort Key: t1_4.b
-                     ->  Seq Scan on prt2_p2 t1_4
+                     Sort Key: t1_6.b
+                     ->  Seq Scan on prt2_p1 t1_6
                ->  Sort
-                     Sort Key: (((t1_7.a + t1_7.b) / 2))
-                     ->  Seq Scan on prt1_e_p2 t1_7
+                     Sort Key: (((t1_11.a + t1_11.b) / 2))
+                     ->  Seq Scan on prt1_e_p1 t1_11
                            Filter: (c = 0)
    ->  Merge Semi Join
-         Merge Cond: (t1_2.a = t1_5.b)
+         Merge Cond: (t1_2.a = t1_7.b)
          ->  Sort
                Sort Key: t1_2.a
-               ->  Seq Scan on prt1_p3 t1_2
+               ->  Seq Scan on prt1_p2 t1_2
                      Filter: (b = 0)
          ->  Merge Semi Join
-               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+               Merge Cond: (t1_7.b = (((t1_12.a + t1_12.b) / 2)))
                ->  Sort
-                     Sort Key: t1_5.b
-                     ->  Seq Scan on prt2_p3 t1_5
+                     Sort Key: t1_7.b
+                     ->  Seq Scan on prt2_p2 t1_7
                ->  Sort
-                     Sort Key: (((t1_8.a + t1_8.b) / 2))
-                     ->  Seq Scan on prt1_e_p3 t1_8
+                     Sort Key: (((t1_12.a + t1_12.b) / 2))
+                     ->  Seq Scan on prt1_e_p2 t1_12
                            Filter: (c = 0)
-(47 rows)
+   ->  Merge Semi Join
+         Merge Cond: (t1_3.a = t1_8.b)
+         ->  Sort
+               Sort Key: t1_3.a
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_8.b = (((t1_13.a + t1_13.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_8.b
+                     ->  Seq Scan on prt2_p3 t1_8
+               ->  Sort
+                     Sort Key: (((t1_13.a + t1_13.b) / 2))
+                     ->  Seq Scan on prt1_e_p3 t1_13
+                           Filter: (c = 0)
+   ->  Merge Semi Join
+         Merge Cond: (t1_4.a = t1_9.b)
+         ->  Sort
+               Sort Key: t1_4.a
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Merge Semi Join
+               Merge Cond: (t1_9.b = (((t1_14.a + t1_14.b) / 2)))
+               ->  Sort
+                     Sort Key: t1_9.b
+                     ->  Seq Scan on prt2_p4 t1_9
+               ->  Sort
+                     Sort Key: (((t1_14.a + t1_14.b) / 2))
+                     ->  Seq Scan on prt1_e_p4 t1_14
+                           Filter: (c = 0)
+(77 rows)
 
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM prt1_e t1 WHERE t1.c = 0)) AND t1.b = 0 ORDER BY t1.a;
-  a  | b |  c   
------+---+------
-   0 | 0 | 0000
- 150 | 0 | 0150
- 300 | 0 | 0300
- 450 | 0 | 0450
-(4 rows)
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
@@ -998,14 +1514,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
                            ->  Sort
                                  Sort Key: (((t3.a + t3.b) / 2))
-                                 ->  Seq Scan on prt1_e_p1 t3
+                                 ->  Seq Scan on prt1_e_p0 t3
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1.a
-                                 ->  Seq Scan on prt1_p1 t1
+                                 ->  Seq Scan on prt1_p0 t1
                ->  Sort
                      Sort Key: t2.b
-                     ->  Seq Scan on prt2_p1 t2
+                     ->  Seq Scan on prt2_p0 t2
          ->  Merge Left Join
                Merge Cond: (t1_1.a = t2_1.b)
                ->  Sort
@@ -1014,14 +1530,14 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_1.a)
                            ->  Sort
                                  Sort Key: (((t3_1.a + t3_1.b) / 2))
-                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                 ->  Seq Scan on prt1_e_p1 t3_1
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_1.a
-                                 ->  Seq Scan on prt1_p2 t1_1
+                                 ->  Seq Scan on prt1_p1 t1_1
                ->  Sort
                      Sort Key: t2_1.b
-                     ->  Seq Scan on prt2_p2 t2_1
+                     ->  Seq Scan on prt2_p1 t2_1
          ->  Merge Left Join
                Merge Cond: (t1_2.a = t2_2.b)
                ->  Sort
@@ -1030,32 +1546,74 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_2.a)
                            ->  Sort
                                  Sort Key: (((t3_2.a + t3_2.b) / 2))
-                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                 ->  Seq Scan on prt1_e_p2 t3_2
                                        Filter: (c = 0)
                            ->  Sort
                                  Sort Key: t1_2.a
-                                 ->  Seq Scan on prt1_p3 t1_2
+                                 ->  Seq Scan on prt1_p2 t1_2
                ->  Sort
                      Sort Key: t2_2.b
-                     ->  Seq Scan on prt2_p3 t2_2
-(51 rows)
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Merge Left Join
+               Merge Cond: (t1_3.a = t2_3.b)
+               ->  Sort
+                     Sort Key: t1_3.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_3.a + t3_3.b) / 2)) = t1_3.a)
+                           ->  Sort
+                                 Sort Key: (((t3_3.a + t3_3.b) / 2))
+                                 ->  Seq Scan on prt1_e_p3 t3_3
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_3.a
+                                 ->  Seq Scan on prt1_p3 t1_3
+               ->  Sort
+                     Sort Key: t2_3.b
+                     ->  Seq Scan on prt2_p3 t2_3
+         ->  Merge Left Join
+               Merge Cond: (t1_4.a = t2_4.b)
+               ->  Sort
+                     Sort Key: t1_4.a
+                     ->  Merge Left Join
+                           Merge Cond: ((((t3_4.a + t3_4.b) / 2)) = t1_4.a)
+                           ->  Sort
+                                 Sort Key: (((t3_4.a + t3_4.b) / 2))
+                                 ->  Seq Scan on prt1_e_p4 t3_4
+                                       Filter: (c = 0)
+                           ->  Sort
+                                 Sort Key: t1_4.a
+                                 ->  Seq Scan on prt1_p4 t1_4
+               ->  Sort
+                     Sort Key: t2_4.b
+                     ->  Seq Scan on prt2_p4 t2_4
+(83 rows)
 
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.c = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
-  a  |  c   |  b  |  c   | ?column? | c 
------+------+-----+------+----------+---
-   0 | 0000 |   0 | 0000 |        0 | 0
-  50 | 0050 |     |      |      100 | 0
- 100 | 0100 |     |      |      200 | 0
- 150 | 0150 | 150 | 0150 |      300 | 0
- 200 | 0200 |     |      |      400 | 0
- 250 | 0250 |     |      |      500 | 0
- 300 | 0300 | 300 | 0300 |      600 | 0
- 350 | 0350 |     |      |      700 | 0
- 400 | 0400 |     |      |      800 | 0
- 450 | 0450 | 450 | 0450 |      900 | 0
- 500 | 0500 |     |      |     1000 | 0
- 550 | 0550 |     |      |     1100 | 0
-(12 rows)
+  a   |   c   |  b   |   c   | ?column? | c 
+------+-------+------+-------+----------+---
+ -250 | -0250 |      |       |     -500 | 0
+ -200 | -0200 |      |       |     -400 | 0
+ -150 | -0150 | -150 | -0150 |     -300 | 0
+ -100 | -0100 |      |       |     -200 | 0
+  -50 | -0050 |      |       |     -100 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+    0 | 0000  |    0 | 0000  |        0 | 0
+   50 | 0050  |      |       |      100 | 0
+  100 | 0100  |      |       |      200 | 0
+  150 | 0150  |  150 | 0150  |      300 | 0
+  200 | 0200  |      |       |      400 | 0
+  250 | 0250  |      |       |      500 | 0
+  300 | 0300  |  300 | 0300  |      600 | 0
+  350 | 0350  |      |       |      700 | 0
+  400 | 0400  |      |       |      800 | 0
+  450 | 0450  |  450 | 0450  |      900 | 0
+  500 | 0500  |      |       |     1000 | 0
+  550 | 0550  |      |       |     1100 | 0
+  600 | 0600  |  600 | 0600  |     1200 | 0
+  650 | 0650  |      |       |     1300 | 0
+  700 | 0700  |      |       |     1400 | 0
+  750 | 0750  |  750 | 0750  |     1500 | 0
+(22 rows)
 
 -- MergeAppend on nullable column
 -- This should generate a partitionwise join, but currently fails to
@@ -1070,9 +1628,11 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
          ->  Sort
                Sort Key: prt1.a
                ->  Append
-                     ->  Seq Scan on prt1_p1 prt1
+                     ->  Seq Scan on prt1_p0 prt1
+                           Filter: ((a < 450) AND (b = 0))
+                     ->  Seq Scan on prt1_p1 prt1_1
                            Filter: ((a < 450) AND (b = 0))
-                     ->  Seq Scan on prt1_p2 prt1_1
+                     ->  Seq Scan on prt1_p2 prt1_2
                            Filter: ((a < 450) AND (b = 0))
          ->  Sort
                Sort Key: prt2.b
@@ -1081,21 +1641,28 @@ SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT *
                            Filter: (b > 250)
                      ->  Seq Scan on prt2_p3 prt2_1
                            Filter: (b > 250)
-(18 rows)
+                     ->  Seq Scan on prt2_p4 prt2_2
+                           Filter: (b > 250)
+(22 rows)
 
 SELECT t1.a, t2.b FROM (SELECT * FROM prt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a, t2.b;
-  a  |  b  
------+-----
-   0 |    
-  50 |    
- 100 |    
- 150 |    
- 200 |    
- 250 |    
- 300 | 300
- 350 |    
- 400 |    
-(9 rows)
+  a   |  b  
+------+-----
+ -250 |    
+ -200 |    
+ -150 |    
+ -100 |    
+  -50 |    
+    0 |    
+   50 |    
+  100 |    
+  150 |    
+  200 |    
+  250 |    
+  300 | 300
+  350 |    
+  400 |    
+(14 rows)
 
 -- merge join when expression with whole-row reference needs to be sorted;
 -- partitionwise join does not apply
@@ -1109,30 +1676,788 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
          Sort Key: t1.a, ((((t1.*)::prt1))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
    ->  Sort
          Sort Key: t2.b, ((((t2.*)::prt2))::text)
          ->  Result
                ->  Append
-                     ->  Seq Scan on prt2_p1 t2
-                     ->  Seq Scan on prt2_p2 t2_1
-                     ->  Seq Scan on prt2_p3 t2_2
-(16 rows)
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+(20 rows)
 
 SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.b ORDER BY t1.a;
- a  | b  
-----+----
-  0 |  0
-  6 |  6
- 12 | 12
- 18 | 18
- 24 | 24
-(5 rows)
+  a  |  b  
+-----+-----
+ -24 | -24
+ -18 | -18
+ -12 | -12
+  -6 |  -6
+   0 |   0
+   6 |   6
+  12 |  12
+  18 |  18
+  24 |  24
+(9 rows)
 
 RESET enable_hashjoin;
 RESET enable_nestloop;
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches prt2_p3 and
+-- prt2_p4 partition from prt2.
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_4
+                           Filter: (b = 0)
+(22 rows)
+
+-- partitionwise join should be possible when we drop prt2_p4 from prt2.
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+(26 rows)
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ANALYZE prt2;
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -150 | -0150 | 0 | -0150
+    0 | 0000  | 0 | 0000
+  150 | 0150  | 0 | 0150
+  300 | 0300  | 0 | 0300
+  450 | 0450  | 0 | 0450
+  600 | 0600  | 0 | 0600
+  750 | 0750  | 0 | 0750
+(7 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on prt2_p0 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+         ->  Nested Loop Left Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Right Join
+               Hash Cond: (t2_4.b = t1_4.a)
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(32 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+  a   |   c   | a |   c   
+------+-------+---+-------
+ -250 | -0250 |   | 
+ -200 | -0200 |   | 
+ -150 | -0150 | 0 | -0150
+ -100 | -0100 |   | 
+  -50 | -0050 |   | 
+    0 | 0000  | 0 | 0000
+   50 | 0050  |   | 
+  100 | 0100  |   | 
+  150 | 0150  | 0 | 0150
+  200 | 0200  |   | 
+  250 | 0250  |   | 
+  300 | 0300  | 0 | 0300
+  350 | 0350  |   | 
+  400 | 0400  |   | 
+  450 | 0450  | 0 | 0450
+  500 | 0500  |   | 
+  550 | 0550  |   | 
+  600 | 0600  | 0 | 0600
+  650 | 0650  |   | 
+  700 | 0700  |   | 
+  750 | 0750  | 0 | 0750
+(21 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Semi Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Append
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t2
+                     Index Cond: (a = t1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p1_a on prt1_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t2_2
+                     Index Cond: (a = t1_2.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t2_3
+                     Index Cond: (a = t1_3.b)
+         ->  Nested Loop Semi Join
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Index Only Scan using iprt1_p4_a on prt1_p4 t2_4
+                     Index Cond: (a = t1_4.b)
+(28 rows)
+
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+ a |  b   |   c   
+---+------+-------
+ 0 | -150 | -0150
+ 0 |    0 | 0000
+ 0 |  150 | 0150
+ 0 |  300 | 0300
+ 0 |  450 | 0450
+ 0 |  600 | 0600
+ 0 |  750 | 0750
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1.a = t2.b)
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p0 t2
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Index Only Scan using iprt2_p3_b on prt2_p3 t2_3
+                     Index Cond: (b = t1_3.a)
+         ->  Hash Anti Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_p4 t2_4
+(32 rows)
+
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a   | b |   c   
+------+---+-------
+ -250 | 0 | -0250
+ -200 | 0 | -0200
+ -100 | 0 | -0100
+  -50 | 0 | -0050
+   50 | 0 | 0050
+  100 | 0 | 0100
+  200 | 0 | 0200
+  250 | 0 | 0250
+  350 | 0 | 0350
+  400 | 0 | 0400
+  500 | 0 | 0500
+  550 | 0 | 0550
+  650 | 0 | 0650
+  700 | 0 | 0700
+(14 rows)
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+                             QUERY PLAN                              
+---------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t3.c
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p0_a on prt1_p0 t3
+                           Index Cond: (a = t2.b)
+               ->  Index Only Scan using iprt1_p0_a on prt1_p0 t1
+                     Index Cond: (a = t2.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_1.a = t2_1.b)
+                           ->  Seq Scan on prt1_p1 t3_1
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p1 t2_1
+                                       Filter: (a = 0)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p2_a on prt1_p2 t3_2
+                           Index Cond: (a = t2_2.b)
+               ->  Index Only Scan using iprt1_p2_a on prt1_p2 t1_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Index Scan using iprt1_p3_a on prt1_p3 t3_3
+                           Index Cond: (a = t2_3.b)
+               ->  Index Only Scan using iprt1_p3_a on prt1_p3 t1_3
+                     Index Cond: (a = t2_3.b)
+         ->  Hash Right Join
+               Hash Cond: (t1_4.a = t2_4.b)
+               ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_4.a = t2_4.b)
+                           ->  Seq Scan on prt1_p4 t3_4
+                           ->  Hash
+                                 ->  Seq Scan on prt2_p4 t2_4
+                                       Filter: (a = 0)
+(47 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+  a   | a |   c   
+------+---+-------
+ -150 | 0 | -0150
+    0 | 0 | 0000
+  150 | 0 | 0150
+  300 | 0 | 0300
+  450 | 0 | 0450
+  600 | 0 | 0600
+  750 | 0 | 0750
+(7 rows)
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.b
+   ->  Hash Right Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p1 t2_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p2 t2_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p3 t2_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p4 t2_4
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_p5 t2_5
+                           Filter: (a = 0)
+(24 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: (t1.a = t2.b)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.a, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Right Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t2_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
+                     Filter: (a = 0)
+(22 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Join
+         Hash Cond: (t1.a = t2.b)
+         Join Filter: ((t1.b + t2.a) = 0)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+               ->  Seq Scan on prt1_p1 t1_1
+               ->  Seq Scan on prt1_p2 t1_2
+               ->  Seq Scan on prt1_p3 t1_3
+               ->  Seq Scan on prt1_p4 t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(19 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Semi Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_p4 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.b, t1.c
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_p0 t1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t1_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t1_5
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t2
+                     ->  Seq Scan on prt1_p1 t2_1
+                     ->  Seq Scan on prt1_p2 t2_2
+                     ->  Seq Scan on prt1_p3 t2_3
+                     ->  Seq Scan on prt1_p4 t2_4
+(24 rows)
+
 --
 -- partitioned by multiple columns
 --
@@ -1202,82 +2527,1552 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 0011
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 470 | 0 | 0011
+(1 row)
+
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 -- test partition matching with N-way join
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-                                   QUERY PLAN                                   
---------------------------------------------------------------------------------
+                                           QUERY PLAN                                           
+------------------------------------------------------------------------------------------------
  GroupAggregate
    Group Key: t1.c, t2.c, t3.c
    ->  Sort
          Sort Key: t1.c, t3.c
          ->  Append
                ->  Hash Join
-                     Hash Cond: (t1.c = ltrim(t3.c, 'A'::text))
+                     Hash Cond: ((t1.c)::text = ltrim(t3.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1.b = t2.b) AND (t1.c = t2.c))
-                           ->  Seq Scan on plt1_p1 t1
+                           Hash Cond: ((t2.b = t1.b) AND ((t2.c)::text = (t1.c)::text))
+                           ->  Seq Scan on plt2_p4 t2
                            ->  Hash
-                                 ->  Seq Scan on plt2_p1 t2
+                                 ->  Seq Scan on plt1_p4 t1
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p1 t3
+                           ->  Seq Scan on plt1_e_p4 t3
                ->  Hash Join
-                     Hash Cond: (t1_1.c = ltrim(t3_1.c, 'A'::text))
+                     Hash Cond: ((t1_1.c)::text = ltrim(t3_1.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
-                           ->  Seq Scan on plt1_p2 t1_1
+                           Hash Cond: ((t1_1.b = t2_1.b) AND ((t1_1.c)::text = (t2_1.c)::text))
+                           ->  Seq Scan on plt1_p1 t1_1
                            ->  Hash
-                                 ->  Seq Scan on plt2_p2 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_1
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p2 t3_1
+                           ->  Seq Scan on plt1_e_p1 t3_1
                ->  Hash Join
-                     Hash Cond: (t1_2.c = ltrim(t3_2.c, 'A'::text))
+                     Hash Cond: ((t1_2.c)::text = ltrim(t3_2.c, 'A'::text))
                      ->  Hash Join
-                           Hash Cond: ((t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
-                           ->  Seq Scan on plt1_p3 t1_2
+                           Hash Cond: ((t1_2.b = t2_2.b) AND ((t1_2.c)::text = (t2_2.c)::text))
+                           ->  Seq Scan on plt1_p2 t1_2
                            ->  Hash
-                                 ->  Seq Scan on plt2_p3 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_2
                      ->  Hash
-                           ->  Seq Scan on plt1_e_p3 t3_2
-(32 rows)
+                           ->  Seq Scan on plt1_e_p2 t3_2
+               ->  Hash Join
+                     Hash Cond: ((t1_3.c)::text = ltrim(t3_3.c, 'A'::text))
+                     ->  Hash Join
+                           Hash Cond: ((t2_3.b = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text))
+                           ->  Seq Scan on plt2_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p3 t1_3
+                     ->  Hash
+                           ->  Seq Scan on plt1_e_p3 t3_3
+(41 rows)
 
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
-         avg          |         avg          |          avg          |  c   |  c   |   c   
-----------------------+----------------------+-----------------------+------+------+-------
-  24.0000000000000000 |  24.0000000000000000 |   48.0000000000000000 | 0000 | 0000 | A0000
-  75.0000000000000000 |  75.0000000000000000 |  148.0000000000000000 | 0001 | 0001 | A0001
- 123.0000000000000000 | 123.0000000000000000 |  248.0000000000000000 | 0002 | 0002 | A0002
- 174.0000000000000000 | 174.0000000000000000 |  348.0000000000000000 | 0003 | 0003 | A0003
- 225.0000000000000000 | 225.0000000000000000 |  448.0000000000000000 | 0004 | 0004 | A0004
- 273.0000000000000000 | 273.0000000000000000 |  548.0000000000000000 | 0005 | 0005 | A0005
- 324.0000000000000000 | 324.0000000000000000 |  648.0000000000000000 | 0006 | 0006 | A0006
- 375.0000000000000000 | 375.0000000000000000 |  748.0000000000000000 | 0007 | 0007 | A0007
- 423.0000000000000000 | 423.0000000000000000 |  848.0000000000000000 | 0008 | 0008 | A0008
- 474.0000000000000000 | 474.0000000000000000 |  948.0000000000000000 | 0009 | 0009 | A0009
- 525.0000000000000000 | 525.0000000000000000 | 1048.0000000000000000 | 0010 | 0010 | A0010
- 573.0000000000000000 | 573.0000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
-(12 rows)
+         avg          |         avg         |         avg          |  c   |  c   |  c   
+----------------------+---------------------+----------------------+------+------+------
+ 246.5000000000000000 | 22.4666666666666667 | 268.9666666666666667 | 0000 | 0000 | 0000
+ 248.5000000000000000 | 21.3333333333333333 | 269.8333333333333333 | 0002 | 0002 | 0002
+ 249.5000000000000000 | 22.3333333333333333 | 271.8333333333333333 | 0003 | 0003 | 0003
+ 250.5000000000000000 | 23.3333333333333333 | 273.8333333333333333 | 0004 | 0004 | 0004
+ 251.5000000000000000 | 22.7666666666666667 | 274.2666666666666667 | 0005 | 0005 | 0005
+ 252.5000000000000000 | 22.2000000000000000 | 274.7000000000000000 | 0006 | 0006 | 0006
+ 246.0000000000000000 | 23.9655172413793103 | 269.9655172413793103 | 0008 | 0008 | 0008
+ 247.0000000000000000 | 23.3448275862068966 | 270.3448275862068966 | 0009 | 0009 | 0009
+(8 rows)
+
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p4 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p4 t2
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 470 | 0011
+     |      | 235 | 0014
+(8 rows)
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+                     ->  Seq Scan on plt2_p5 t2_4
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      |  47 | 0013
+     |      | 235 | 0014
+     |      | 470 | 0011
+(10 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+  47 | 0 | 0013
+ 235 | 0 | 0014
+ 470 | 0 | 0011
+(3 rows)
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Left Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Right Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+               ->  Seq Scan on plt1_p1 t1_1
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Seq Scan on plt1_p3 t1_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p5 t2_1
+                                 ->  Seq Scan on plt2_p1 t2_2
+                                 ->  Seq Scan on plt2_p2 t2_3
+                                 ->  Seq Scan on plt2_p3 t2_4
+(23 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Nested Loop Anti Join
+         Join Filter: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Materialize
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p5 t2_1
+                     ->  Seq Scan on plt2_p1 t2_2
+                     ->  Seq Scan on plt2_p2 t2_3
+                     ->  Seq Scan on plt2_p3 t2_4
+(20 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p5 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_4
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(21 rows)
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(5 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+(7 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_2
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(6 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+               ->  Seq Scan on plt2_p4 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+  a  |  c   |  a  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  94 | 0009 |  94 | 0009
+ 141 | 0005 | 141 | 0005
+ 188 | 0001 |     | 
+ 282 | 0010 |     | 
+ 329 | 0006 | 329 | 0006
+ 376 | 0002 | 376 | 0002
+     |      | 470 | 
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+  94 | 0 | 0009
+ 141 | 0 | 0005
+ 329 | 0 | 0006
+ 376 | 0 | 0002
+(5 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b |  c   
+-----+---+------
+ 188 | 0 | 0001
+ 282 | 0 | 0010
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
+
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+  a  | b | c 
+-----+---+---
+ 470 | 0 | 
+(1 row)
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Join Filter: ((t1.b + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t2.a
+   ->  Hash Left Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.c)::text = (t1.c)::text)
+         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t2
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Seq Scan on plt2_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t1
+                     ->  Seq Scan on plt1_p1 t1_1
+                     ->  Seq Scan on plt1_p2 t1_2
+                     ->  Seq Scan on plt1_p3 t1_3
+(16 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt2_p4 t2
+                                 ->  Seq Scan on plt2_p1 t2_1
+                                 ->  Seq Scan on plt2_p2 t2_2
+                                 ->  Seq Scan on plt2_p3 t2_3
+(22 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Result
+                           ->  Append
+                                 ->  Seq Scan on plt1_p4 t2
+                                 ->  Seq Scan on plt1_p1 t2_1
+                                 ->  Seq Scan on plt1_p2 t2_2
+                                 ->  Seq Scan on plt1_p3 t2_3
+(22 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt1_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+(19 rows)
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+                    QUERY PLAN                    
+--------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.c
+   ->  Hash Anti Join
+         Hash Cond: ((t1.c)::text = (t2.c)::text)
+         ->  Append
+               ->  Seq Scan on plt2_p4 t1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on plt2_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_p4 t2
+                     ->  Seq Scan on plt1_p1 t2_1
+                     ->  Seq Scan on plt1_p2 t2_2
+                     ->  Seq Scan on plt1_p3 t2_3
+(19 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
@@ -1302,22 +4097,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
 --------------------------------------------------
  Hash Left Join
    Hash Cond: (t2.b = a)
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t3.a = t2.b)
-               ->  Seq Scan on prt1_p1 t3
-               ->  Hash
-                     ->  Seq Scan on prt2_p1 t2
-         ->  Hash Join
-               Hash Cond: (t3_1.a = t2_1.b)
-               ->  Seq Scan on prt1_p2 t3_1
-               ->  Hash
-                     ->  Seq Scan on prt2_p2 t2_1
-         ->  Hash Join
-               Hash Cond: (t3_2.a = t2_2.b)
-               ->  Seq Scan on prt1_p3 t3_2
-               ->  Hash
-                     ->  Seq Scan on prt2_p3 t2_2
+   ->  Hash Join
+         Hash Cond: (t3.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_p0 t3
+               ->  Seq Scan on prt1_p1 t3_1
+               ->  Seq Scan on prt1_p2 t3_2
+               ->  Seq Scan on prt1_p3 t3_3
+               ->  Seq Scan on prt1_p4 t3_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_p0 t2
+                     ->  Seq Scan on prt2_p1 t2_1
+                     ->  Seq Scan on prt2_p2 t2_2
+                     ->  Seq Scan on prt2_p3 t2_3
+                     ->  Seq Scan on prt2_p4 t2_4
+                     ->  Seq Scan on prt2_p5 t2_5
    ->  Hash
          ->  Result
                One-Time Filter: false
@@ -1332,16 +4127,22 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
    ->  Hash Left Join
          Hash Cond: (t2.b = a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
+               ->  Seq Scan on prt2_p0 t2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p1 t2_1
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
                      Filter: (a = 0)
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p4 t2_4
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_p5 t2_5
                      Filter: (a = 0)
          ->  Hash
                ->  Result
                      One-Time Filter: false
-(14 rows)
+(20 rows)
 
 --
 -- tests for hash partitioned tables.
@@ -1417,41 +4218,9 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  273.0000000000000000 | 273.0000000000000000 | 548.0000000000000000 | 0005 | 0005 | A0005
 (6 rows)
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
- Sort
-   Sort Key: t1.a
-   ->  Append
-         ->  Hash Join
-               Hash Cond: (t2.b = t1.a)
-               ->  Seq Scan on prt2_p1 t2
-               ->  Hash
-                     ->  Seq Scan on prt1_p1 t1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_1.b = t1_1.a)
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Hash
-                     ->  Seq Scan on prt1_p2 t1_1
-                           Filter: (b = 0)
-         ->  Hash Join
-               Hash Cond: (t2_2.b = t1_2.a)
-               ->  Seq Scan on prt2_p3 t2_2
-               ->  Hash
-                     ->  Seq Scan on prt1_p3 t1_2
-                           Filter: (b = 0)
-(21 rows)
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
@@ -1466,26 +4235,24 @@ SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c
    Sort Key: t1.c
    ->  HashAggregate
          Group Key: t1.c, t2.c
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t2.c = t1.c)
-                     ->  Seq Scan on plt2_p1 t2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p1 t1
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               ->  Append
+                     ->  Seq Scan on plt2_p4 t2
+                     ->  Seq Scan on plt2_p1 t2_1
+                     ->  Seq Scan on plt2_p2 t2_2
+                     ->  Seq Scan on plt2_p3 t2_3
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on plt1_p4 t1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_1.c = t1_1.c)
-                     ->  Seq Scan on plt2_p2 t2_1
-                     ->  Hash
-                           ->  Seq Scan on plt1_p2 t1_1
+                           ->  Seq Scan on plt1_p1 t1_1
                                  Filter: ((a % 25) = 0)
-               ->  Hash Join
-                     Hash Cond: (t2_2.c = t1_2.c)
-                     ->  Seq Scan on plt2_p3 t2_2
-                     ->  Hash
-                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Seq Scan on plt1_p2 t1_2
                                  Filter: ((a % 25) = 0)
-(23 rows)
+                           ->  Seq Scan on plt1_p3 t1_3
+                                 Filter: ((a % 25) = 0)
+(21 rows)
 
 --
 -- multiple levels of partitioning
@@ -1881,64 +4648,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
  Hash Join
    Hash Cond: (t1.a = t2.a)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt4_n_p1 t2
                ->  Seq Scan on prt4_n_p2 t2_1
                ->  Seq Scan on prt4_n_p3 t2_2
-(11 rows)
+(13 rows)
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt2 t3 WHERE t1.a = t2.a and t1.a = t3.b;
-                       QUERY PLAN                       
---------------------------------------------------------
+                        QUERY PLAN                        
+----------------------------------------------------------
  Hash Join
-   Hash Cond: (t2.a = t1.a)
+   Hash Cond: (t3.b = t1.a)
    ->  Append
-         ->  Seq Scan on prt4_n_p1 t2
-         ->  Seq Scan on prt4_n_p2 t2_1
-         ->  Seq Scan on prt4_n_p3 t2_2
+         ->  Seq Scan on prt2_p0 t3
+         ->  Seq Scan on prt2_p1 t3_1
+         ->  Seq Scan on prt2_p2 t3_2
+         ->  Seq Scan on prt2_p3 t3_3
+         ->  Seq Scan on prt2_p4 t3_4
+         ->  Seq Scan on prt2_p5 t3_5
    ->  Hash
-         ->  Append
-               ->  Hash Join
-                     Hash Cond: (t1.a = t3.b)
-                     ->  Seq Scan on prt1_p1 t1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p1 t3
-               ->  Hash Join
-                     Hash Cond: (t1_1.a = t3_1.b)
-                     ->  Seq Scan on prt1_p2 t1_1
-                     ->  Hash
-                           ->  Seq Scan on prt2_p2 t3_1
-               ->  Hash Join
-                     Hash Cond: (t1_2.a = t3_2.b)
-                     ->  Seq Scan on prt1_p3 t1_2
-                     ->  Hash
-                           ->  Seq Scan on prt2_p3 t3_2
+         ->  Hash Join
+               Hash Cond: (t1.a = t2.a)
+               ->  Append
+                     ->  Seq Scan on prt1_p0 t1
+                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_p4 t1_4
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt4_n_p1 t2
+                           ->  Seq Scan on prt4_n_p2 t2_1
+                           ->  Seq Scan on prt4_n_p3 t2_2
 (23 rows)
 
 -- partitionwise join can not be applied if there are no equi-join conditions
 -- between partition keys
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON (t1.a < t2.b);
-                       QUERY PLAN                        
----------------------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
    ->  Append
-         ->  Seq Scan on prt1_p1 t1
-         ->  Seq Scan on prt1_p2 t1_1
-         ->  Seq Scan on prt1_p3 t1_2
-   ->  Append
-         ->  Index Scan using iprt2_p1_b on prt2_p1 t2
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_1
-               Index Cond: (b > t1.a)
-         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
-               Index Cond: (b > t1.a)
-(12 rows)
+         ->  Seq Scan on prt1_p0 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p2 t1_2
+         ->  Seq Scan on prt1_p3 t1_3
+         ->  Seq Scan on prt1_p4 t1_4
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
+(16 rows)
 
 -- equi-join with join condition on partial keys does not qualify for
 -- partitionwise join
@@ -2024,16 +4797,17 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
          ->  Seq Scan on prt2_n_p2 t2_1
    ->  Hash
          ->  Hash Join
-               Hash Cond: (t3.c = (t1.c)::text)
+               Hash Cond: ((t3.c)::text = (t1.c)::text)
                ->  Append
-                     ->  Seq Scan on plt1_p1 t3
-                     ->  Seq Scan on plt1_p2 t3_1
-                     ->  Seq Scan on plt1_p3 t3_2
+                     ->  Seq Scan on plt1_p4 t3
+                     ->  Seq Scan on plt1_p1 t3_1
+                     ->  Seq Scan on plt1_p2 t3_2
+                     ->  Seq Scan on plt1_p3 t3_3
                ->  Hash
                      ->  Append
                            ->  Seq Scan on prt1_n_p1 t1
                            ->  Seq Scan on prt1_n_p2 t1_1
-(16 rows)
+(17 rows)
 
 -- partitionwise join can not be applied for a join between list and range
 -- partitioned tables
@@ -2044,14 +4818,16 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
  Hash Full Join
    Hash Cond: ((t2.c)::text = (t1.c)::text)
    ->  Append
-         ->  Seq Scan on prt1_p1 t2
-         ->  Seq Scan on prt1_p2 t2_1
-         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p0 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p2 t2_2
+         ->  Seq Scan on prt1_p3 t2_3
+         ->  Seq Scan on prt1_p4 t2_4
    ->  Hash
          ->  Append
                ->  Seq Scan on prt1_n_p1 t1
                ->  Seq Scan on prt1_n_p2 t1_1
-(10 rows)
+(12 rows)
 
 -- partitionwise join can not be applied if only one of joining tables has
 -- default partition
@@ -2067,16 +4843,279 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b =
    ->  Hash Join
          Hash Cond: (t2.b = t1.a)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2
-               ->  Seq Scan on prt2_p2 t2_1
-               ->  Seq Scan on prt2_p3 t2_2
+               ->  Seq Scan on prt2_p0 t2
+               ->  Seq Scan on prt2_p1 t2_1
+               ->  Seq Scan on prt2_p2 t2_2
+               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt2_p4 t2_4
+               ->  Seq Scan on prt2_p5 t2_5
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1
+                     ->  Seq Scan on prt1_p0 t1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_1
+                     ->  Seq Scan on prt1_p1 t1_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_2
+                     ->  Seq Scan on prt1_p2 t1_2
                            Filter: (b = 0)
-(16 rows)
+                     ->  Seq Scan on prt1_p3 t1_3
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p4 t1_4
+                           Filter: (b = 0)
+(23 rows)
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007');
+INSERT INTO plt1 SELECT i, i, to_char(i % 8, 'FM0000') FROM generate_series(0, 39) i WHERE i % 8 IN (0, 2, 3, 4, 6, 7);
+ANALYZE plt1;
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0005', '0007');
+INSERT INTO plt2 SELECT i, i, to_char(i % 8, 'FM0000') FROM generate_series(0, 39) i WHERE i % 8 IN (1, 2, 3, 4, 5, 7);
+ANALYZE plt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 8 != 2 AND COALESCE(t1.a, 0) % 8 != 3 AND COALESCE(t1.a, 0) % 8 != 4 AND COALESCE(t1.a, 0) % 8 != 7 ORDER BY t1.c, t2.c, t1.a, t2.a;
+                                                                               QUERY PLAN                                                                                
+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t2.c, t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.c = t2.c)
+               Filter: (((COALESCE(t1.a, 0) % 8) <> 2) AND ((COALESCE(t1.a, 0) % 8) <> 3) AND ((COALESCE(t1.a, 0) % 8) <> 4) AND ((COALESCE(t1.a, 0) % 8) <> 7))
+               ->  Seq Scan on plt1_p1 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t2_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 8) <> 2) AND ((COALESCE(t1_1.a, 0) % 8) <> 3) AND ((COALESCE(t1_1.a, 0) % 8) <> 4) AND ((COALESCE(t1_1.a, 0) % 8) <> 7))
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_1
+         ->  Hash Full Join
+               Hash Cond: (t1_2.c = t2_2.c)
+               Filter: (((COALESCE(t1_2.a, 0) % 8) <> 2) AND ((COALESCE(t1_2.a, 0) % 8) <> 3) AND ((COALESCE(t1_2.a, 0) % 8) <> 4) AND ((COALESCE(t1_2.a, 0) % 8) <> 7))
+               ->  Seq Scan on plt1_p3 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_2
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 8 != 2 AND COALESCE(t1.a, 0) % 8 != 3 AND COALESCE(t1.a, 0) % 8 != 4 AND COALESCE(t1.a, 0) % 8 != 7 ORDER BY t1.c, t2.c, t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+  0 | 0000 |    | 
+  8 | 0000 |    | 
+ 16 | 0000 |    | 
+ 24 | 0000 |    | 
+ 32 | 0000 |    | 
+  6 | 0006 |    | 
+ 14 | 0006 |    | 
+ 22 | 0006 |    | 
+ 30 | 0006 |    | 
+ 38 | 0006 |    | 
+    |      |  1 | 0001
+    |      |  9 | 0001
+    |      | 17 | 0001
+    |      | 25 | 0001
+    |      | 33 | 0001
+    |      |  5 | 0005
+    |      | 13 | 0005
+    |      | 21 | 0005
+    |      | 29 | 0005
+    |      | 37 | 0005
+(20 rows)
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN (NULL, '0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
+ANALYZE plt1;
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 DEFAULT;
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+ANALYZE plt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t2_1.c, t1.a, t2_1.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.c = t2_1.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 2) AND ((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt1_p1 t1
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t2.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 2) AND ((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt1_p2 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+  0 | 0000 |    | 
+  5 | 0000 |    | 
+ 10 | 0000 |    | 
+ 15 | 0000 |    | 
+ 20 | 0000 |    | 
+    |      |  1 | 0001
+    |      |  6 | 0001
+    |      | 11 | 0001
+    |      | 16 | 0001
+    |      | 21 | 0001
+(10 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2 t2 FULL JOIN plt1 t1 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t2_1.c, t1.a, t2_1.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t2_1.c = t1.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 2) AND ((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_p1 t1
+         ->  Hash Full Join
+               Hash Cond: (t2.c = t1_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 2) AND ((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Seq Scan on plt2_p2 t2
+               ->  Hash
+                     ->  Seq Scan on plt1_p2 t1_1
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2 t2 FULL JOIN plt1 t1 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+  0 | 0000 |    | 
+  5 | 0000 |    | 
+ 10 | 0000 |    | 
+ 15 | 0000 |    | 
+ 20 | 0000 |    | 
+    |      |  1 | 0001
+    |      |  6 | 0001
+    |      | 11 | 0001
+    |      | 16 | 0001
+    |      | 21 | 0001
+(10 rows)
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1;
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2;
+CREATE TABLE plt3 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt3_p1 PARTITION OF plt3 FOR VALUES IN ('0001');
+CREATE TABLE plt3_p2 PARTITION OF plt3 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2.a, t3.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1.c = t3.c)
+               Filter: (((COALESCE(t1.a, 0) % 5) <> 3) AND ((COALESCE(t1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1.c = t2.c)
+                     ->  Seq Scan on plt1_p1 t1
+                     ->  Hash
+                           ->  Seq Scan on plt2_p1 t2
+               ->  Hash
+                     ->  Seq Scan on plt3_p1 t3
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t3_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_1.c = t2_1.c)
+                     ->  Seq Scan on plt1_p2 t1_1
+                     ->  Hash
+                           ->  Seq Scan on plt2_p2 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt3_p2 t3_1
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+ a  |  c   | a  |  c   | a  |  c   
+----+------+----+------+----+------
+  0 | 0000 |    |      |    | 
+  5 | 0000 |    |      |    | 
+ 10 | 0000 |    |      |    | 
+ 15 | 0000 |    |      |    | 
+ 20 | 0000 |    |      |    | 
+  1 | 0001 |    |      |  1 | 0001
+  1 | 0001 |    |      |  6 | 0001
+  1 | 0001 |    |      | 11 | 0001
+  1 | 0001 |    |      | 16 | 0001
+  1 | 0001 |    |      | 21 | 0001
+  6 | 0001 |    |      |  1 | 0001
+  6 | 0001 |    |      |  6 | 0001
+  6 | 0001 |    |      | 11 | 0001
+  6 | 0001 |    |      | 16 | 0001
+  6 | 0001 |    |      | 21 | 0001
+ 11 | 0001 |    |      |  1 | 0001
+ 11 | 0001 |    |      |  6 | 0001
+ 11 | 0001 |    |      | 11 | 0001
+ 11 | 0001 |    |      | 16 | 0001
+ 11 | 0001 |    |      | 21 | 0001
+ 16 | 0001 |    |      |  1 | 0001
+ 16 | 0001 |    |      |  6 | 0001
+ 16 | 0001 |    |      | 11 | 0001
+ 16 | 0001 |    |      | 16 | 0001
+ 16 | 0001 |    |      | 21 | 0001
+ 21 | 0001 |    |      |  1 | 0001
+ 21 | 0001 |    |      |  6 | 0001
+ 21 | 0001 |    |      | 11 | 0001
+ 21 | 0001 |    |      | 16 | 0001
+ 21 | 0001 |    |      | 21 | 0001
+  2 | 0002 |  2 | 0002 |    | 
+  2 | 0002 |  7 | 0002 |    | 
+  2 | 0002 | 12 | 0002 |    | 
+  2 | 0002 | 17 | 0002 |    | 
+  2 | 0002 | 22 | 0002 |    | 
+  7 | 0002 |  2 | 0002 |    | 
+  7 | 0002 |  7 | 0002 |    | 
+  7 | 0002 | 12 | 0002 |    | 
+  7 | 0002 | 17 | 0002 |    | 
+  7 | 0002 | 22 | 0002 |    | 
+ 12 | 0002 |  2 | 0002 |    | 
+ 12 | 0002 |  7 | 0002 |    | 
+ 12 | 0002 | 12 | 0002 |    | 
+ 12 | 0002 | 17 | 0002 |    | 
+ 12 | 0002 | 22 | 0002 |    | 
+ 17 | 0002 |  2 | 0002 |    | 
+ 17 | 0002 |  7 | 0002 |    | 
+ 17 | 0002 | 12 | 0002 |    | 
+ 17 | 0002 | 17 | 0002 |    | 
+ 17 | 0002 | 22 | 0002 |    | 
+ 22 | 0002 |  2 | 0002 |    | 
+ 22 | 0002 |  7 | 0002 |    | 
+ 22 | 0002 | 12 | 0002 |    | 
+ 22 | 0002 | 17 | 0002 |    | 
+ 22 | 0002 | 22 | 0002 |    | 
+(55 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 92994b479bb..d8434f6b1ad 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -10,25 +10,39 @@ SET enable_partitionwise_join to true;
 -- partitioned by a single column
 --
 CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p0 PARTITION OF prt1 FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
 CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
-INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 2 = 0;
+CREATE TABLE prt1_p4 PARTITION OF prt1 FOR VALUES FROM (600) TO (800);
+INSERT INTO prt1 SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 2 = 0;
+CREATE INDEX iprt1_p0_a on prt1_p0(a);
 CREATE INDEX iprt1_p1_a on prt1_p1(a);
 CREATE INDEX iprt1_p2_a on prt1_p2(a);
 CREATE INDEX iprt1_p3_a on prt1_p3(a);
+CREATE INDEX iprt1_p4_a on prt1_p4(a);
 ANALYZE prt1;
 
+-- prt2 have missing starting MINVALUE to -250 range and
+-- extra bounds from 800 to MAXVALUE
 CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p0 PARTITION OF prt2 FOR VALUES FROM (-250) TO (0);
 CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
-INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(0, 599) i WHERE i % 3 = 0;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (MAXVALUE);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(-250, 799) i WHERE i % 3 = 0;
+CREATE INDEX iprt2_p0_b on prt2_p0(b);
 CREATE INDEX iprt2_p1_b on prt2_p1(b);
 CREATE INDEX iprt2_p2_b on prt2_p2(b);
 CREATE INDEX iprt2_p3_b on prt2_p3(b);
+CREATE INDEX iprt2_p4_b on prt2_p4(b);
 ANALYZE prt2;
 
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
+
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
@@ -69,11 +83,19 @@ EXPLAIN (COSTS OFF)
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t2.b FROM prt2 t2 WHERE t2.a = 0) AND t1.b = 0 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+SELECT t1.* FROM prt2 t1 WHERE t1.b IN (SELECT t2.a FROM prt1 t2 WHERE t2.b = 0) AND t1.a = 0 ORDER BY t1.b;
+
 -- Anti-join with aggregates
 EXPLAIN (COSTS OFF)
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 SELECT sum(t1.a), avg(t1.a), sum(t1.b), avg(t1.b) FROM prt1 t1 WHERE NOT EXISTS (SELECT 1 FROM prt2 t2 WHERE t1.a = t2.b);
 
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+SELECT t1.b, t1.c FROM prt2 t1 WHERE NOT EXISTS (SELECT 1 FROM prt1 t2 WHERE t1.b = t2.a) and t1.a = 0;
+
 -- lateral reference
 EXPLAIN (COSTS OFF)
 SELECT * FROM prt1 t1 LEFT JOIN LATERAL
@@ -110,20 +132,30 @@ RESET enable_hashjoin;
 -- partitioned by expression
 --
 CREATE TABLE prt1_e (a int, b int, c int) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p0 PARTITION OF prt1_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt1_e_p4 PARTITION OF prt1_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(0, 599, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 2) i;
+INSERT INTO prt1_e SELECT i, i, i % 25 FROM generate_series(600, 799, 2) i;
+CREATE INDEX iprt1_e_p0_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p1_ab2 on prt1_e_p1(((a+b)/2));
 CREATE INDEX iprt1_e_p2_ab2 on prt1_e_p2(((a+b)/2));
 CREATE INDEX iprt1_e_p3_ab2 on prt1_e_p3(((a+b)/2));
+CREATE INDEX iprt1_e_p4_ab2 on prt1_e_p1(((a+b)/2));
 ANALYZE prt1_e;
 
 CREATE TABLE prt2_e (a int, b int, c int) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p0 PARTITION OF prt2_e FOR VALUES FROM (MINVALUE) TO (0);
 CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES FROM (250) TO (500);
 CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES FROM (500) TO (600);
+CREATE TABLE prt2_e_p4 PARTITION OF prt2_e FOR VALUES FROM (600) TO (MAXVALUE);
 INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(0, 599, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(-250, 0, 3) i;
+INSERT INTO prt2_e SELECT i, i, i % 25 FROM generate_series(600, 799, 3) i;
 ANALYZE prt2_e;
 
 EXPLAIN (COSTS OFF)
@@ -187,6 +219,114 @@ SELECT t1.a, t2.b FROM prt1 t1, prt2 t2 WHERE t1::text = t2::text AND t1.a = t2.
 RESET enable_hashjoin;
 RESET enable_nestloop;
 
+-- test default partition behavior for range, partition-wise join is not
+-- possible since more than one partition on one side matches default partition
+-- on the other side. Default partition from prt1 matches prt2_p3 and
+-- prt2_p4 partition from prt2.
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ANALYZE prt1;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- partitionwise join should be possible when we drop prt2_p4 from prt2.
+ALTER TABLE prt2 DETACH PARTITION prt2_p4;
+ANALYZE prt2;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- restore the partitioned tables for rest of the tests
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 FOR VALUES FROM (500) TO (600);
+ANALYZE prt1;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p4 FOR VALUES FROM (600) TO (MAXVALUE);
+ANALYZE prt2;
+
+-- Add an extra partition to prt2 , Partition-wise join is possible with
+-- extra partitions on inner side are allowed
+DROP TABLE prt2_p4;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (800);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (800) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999) i WHERE i % 3 = 0;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- 3-way join when not every pair of joining relation can use partition-wise
+-- join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON (t1.a = t2.b) INNER JOIN prt1 t3 ON (t2.b = t3.a) WHERE t2.a = 0 ORDER BY t1.a, t2.a, t3.c;
+
+-- partition-wise join can not handle missing partition on the inner side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.b;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE coalesce(t1.b, 0) + coalesce(t2.a, 0) = 0 ORDER BY t1.a, t2.a;
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE prt2_p4;
+DROP TABLE prt2_p5;
+CREATE TABLE prt2_p4 PARTITION OF prt2 FOR VALUES FROM (600) TO (700);
+CREATE TABLE prt2_p5 PARTITION OF prt2 FOR VALUES FROM (700) TO (1000);
+INSERT INTO prt2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(600, 999, 3) i;
+ANALYZE prt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 INNER JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM prt1 t1 FULL JOIN prt2 t2 ON t1.a = t2.b WHERE t1.b + t2.a = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt1 t1 where not exists (select 1 from prt2 t2 WHERE t1.a = t2.b) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from prt2 t1 where not exists (select 1 from prt1 t2 WHERE t1.b = t2.a) and t1.a = 0 order by t1.a, t1.b, t1.c;
+
 --
 -- partitioned by multiple columns
 --
@@ -211,28 +351,79 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.c = 0) t1
 --
 -- tests for list partitioned tables.
 --
-CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
-ANALYZE plt1;
+\set part_mod 17
+\set cond_mod 47
+\set num_rows 500
+
+CREATE TABLE plt1 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0001','0002','0003');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0008','0009');
+CREATE TABLE plt1_p4 PARTITION OF plt1 FOR VALUES IN ('0000','0010');
+INSERT INTO plt1 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (7, 11, 12, 13, 14, 15, 16);
+ANALYSE plt1;
+
+-- plt2 have missing starting 0001, additional 0007, missing ending 0010
+-- and additional 0011 and 0012 bounds
+CREATE TABLE plt2 (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002','0003');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0004','0005','0006');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0007','0008','0009');
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000','0011','0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 10, 13, 14, 15, 16);
+ANALYSE plt2;
+
+-- Partition-wise-join is possible with some partition bounds overlap
+-- with each other completely and some partialy for inner,left,right,
+-- full, semi and anti joins
 
-CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
-CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
-ANALYZE plt2;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
 --
 -- list partitioned by expression
 --
 CREATE TABLE plt1_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
-CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0000', '0003', '0004', '0010');
-CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0001', '0005', '0002', '0009');
-CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0006', '0007', '0008', '0011');
-INSERT INTO plt1_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE plt1_e_p1 PARTITION OF plt1_e FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt1_e_p2 PARTITION OF plt1_e FOR VALUES IN ('0004', '0005', '0006');
+CREATE TABLE plt1_e_p3 PARTITION OF plt1_e FOR VALUES IN ('0008', '0009');
+CREATE TABLE plt1_e_p4 PARTITION OF plt1_e FOR VALUES IN ('0000');
+INSERT INTO plt1_e SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod NOT IN (1, 7, 10, 11, 12, 13, 14, 15, 16);
 ANALYZE plt1_e;
 
 -- test partition matching with N-way join
@@ -240,6 +431,175 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
+-- Add an extra partition to plt2 , Partition-wise join is possible with
+-- partitions on inner side are allowed
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join, partition-wise join can not handle extra partition on the outer
+-- side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt2_p5;
+CREATE TABLE plt2_p5 PARTITION OF plt2 FOR VALUES IN ('0001','0013','0014');
+INSERT INTO plt2 SELECT i, i % :cond_mod, to_char(i % :part_mod, 'FM0000') FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (1, 13, 14);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on one side, Partition-wise join is possible with
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- in this case NULL will be treated as addition partition bounds.
+DROP TABLE plt2_p5;
+DROP TABLE plt2_p4;
+CREATE TABLE plt2_p4 PARTITION OF plt2 FOR VALUES IN ('0000',NULL,'0012');
+INSERT INTO plt2 SELECT i, i % :cond_mod, case when i % :part_mod = 11 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (0,11,12);
+ANALYZE plt2;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- partition have a NULL on both side with different partition bounds w.r.t other side
+-- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
+-- Partition-wise join can not handle the case when one partition from one side
+-- matches with multiple partitions on the other side
+DROP TABLE plt1_p3;
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
+INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
+ANALYZE plt1;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
+
+-- right join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
+EXPLAIN (COSTS OFF)
+select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
+
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
@@ -285,27 +645,18 @@ EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, pht2 t2, pht1_e t3 WHERE t1.b = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c GROUP BY t1.c, t2.c, t3.c ORDER BY t1.c, t2.c, t3.c;
 
--- test default partition behavior for range
-ALTER TABLE prt1 DETACH PARTITION prt1_p3;
-ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
-ANALYZE prt1;
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
-ANALYZE prt2;
-
-EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-
--- test default partition behavior for list
+-- test default partition behavior for list, should not use partition-wise join
+-- since default partition from one side matches multiple partitions on the
+-- other
 ALTER TABLE plt1 DETACH PARTITION plt1_p3;
 ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
 ANALYZE plt1;
 ALTER TABLE plt2 DETACH PARTITION plt2_p3;
 ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
 ANALYZE plt2;
-
 EXPLAIN (COSTS OFF)
 SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.a % 25 = 0 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
 --
 -- multiple levels of partitioning
 --
@@ -450,3 +801,72 @@ ANALYZE prt2;
 
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007');
+INSERT INTO plt1 SELECT i, i, to_char(i % 8, 'FM0000') FROM generate_series(0, 39) i WHERE i % 8 IN (0, 2, 3, 4, 6, 7);
+ANALYZE plt1;
+
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0005', '0007');
+INSERT INTO plt2 SELECT i, i, to_char(i % 8, 'FM0000') FROM generate_series(0, 39) i WHERE i % 8 IN (1, 2, 3, 4, 5, 7);
+ANALYZE plt2;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 8 != 2 AND COALESCE(t1.a, 0) % 8 != 3 AND COALESCE(t1.a, 0) % 8 != 4 AND COALESCE(t1.a, 0) % 8 != 7 ORDER BY t1.c, t2.c, t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 8 != 2 AND COALESCE(t1.a, 0) % 8 != 3 AND COALESCE(t1.a, 0) % 8 != 4 AND COALESCE(t1.a, 0) % 8 != 7 ORDER BY t1.c, t2.c, t1.a, t2.a;
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN (NULL, '0000', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (0, 2, 3, 4);
+ANALYZE plt1;
+
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 DEFAULT;
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 2, 3, 4);
+ANALYZE plt2;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2 t2 FULL JOIN plt1 t1 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2 t2 FULL JOIN plt1 t1 ON (t1.c = t2.c) WHERE COALESCE(t1.a, 0) % 5 != 2 AND COALESCE(t1.a, 0) % 5 != 3 AND COALESCE(t1.a, 0) % 5 != 4 ORDER BY t1.c, t2.c, t1.a, t2.a;
+
+DROP TABLE plt1;
+DROP TABLE plt2;
+
+CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1;
+
+CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0002');
+CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2;
+
+CREATE TABLE plt3 (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE plt3_p1 PARTITION OF plt3 FOR VALUES IN ('0001');
+CREATE TABLE plt3_p2 PARTITION OF plt3 FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3 SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON (t1.c = t2.c)) FULL JOIN plt3 t3 ON (t1.c = t3.c) WHERE COALESCE(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
-- 
2.18.0

v29-0002-Modify-merge_null_partitions.patchapplication/octet-stream; name=v29-0002-Modify-merge_null_partitions.patchDownload
From 9d00b41479b7bc172e25b776ee93d066e011061f Mon Sep 17 00:00:00 2001
From: Amul Sul <amul.sul@enterprisedb.com>
Date: Mon, 9 Dec 2019 04:07:54 -0500
Subject: [PATCH 2/2] v29-0002-Modify-merge_null_partitions

---
 src/backend/partitioning/partbounds.c        | 490 ++++++++++---------
 src/test/regress/expected/partition_join.out | 292 ++++++-----
 src/test/regress/sql/partition_join.sql      |   5 +-
 3 files changed, 427 insertions(+), 360 deletions(-)

diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index a362998e574..494632fc7c7 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -141,7 +141,7 @@ static int map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner
 						 int outer_part, int inner_part, int *next_index);
 static int merge_partition_with_dummy(PartitionMap *map, int index,
 						   int *next_index);
-static bool process_outer_partition(PartitionMap *outer_map,
+static int process_outer_partition(PartitionMap *outer_map,
 						PartitionMap *inner_map,
 						bool outer_has_default,
 						bool inner_has_default,
@@ -149,9 +149,8 @@ static bool process_outer_partition(PartitionMap *outer_map,
 						int inner_default,
 						JoinType jointype,
 						int *next_index,
-						int *default_index,
-						int *merged_index);
-static bool process_inner_partition(PartitionMap *outer_map,
+						int *default_index);
+static int process_inner_partition(PartitionMap *outer_map,
 						PartitionMap *inner_map,
 						bool outer_has_default,
 						bool inner_has_default,
@@ -159,8 +158,7 @@ static bool process_inner_partition(PartitionMap *outer_map,
 						int outer_default,
 						JoinType jointype,
 						int *next_index,
-						int *default_index,
-						int *merged_index);
+						int *default_index);
 static void fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
 				   int nmerged, List *merged_indexes);
 static void generate_matching_part_pairs(RelOptInfo *outer_rel,
@@ -200,21 +198,24 @@ static void add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
 						List **merged_datums,
 						List **merged_kinds,
 						List **merged_indexes);
-static bool merge_default_partitions(PartitionMap *outer_map,
+static void merge_default_partitions(PartitionMap *outer_map,
 						 PartitionMap *inner_map,
 						 bool outer_has_default,
-						 int outer_default,
 						 bool inner_has_default,
+						 int outer_default,
 						 int inner_default,
 						 JoinType jointype,
 						 int *next_index,
 						 int *default_index);
-static bool merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
-					  PartitionMap *outer_map, PartitionMap *inner_map,
-					  bool outer_has_default, bool inner_has_default,
-					  bool outer_has_null, bool inner_has_null,
-					  JoinType jointype, int *next_index,
-					  int *default_index, int *null_index);
+static void merge_null_partitions(PartitionMap *outer_map,
+					  PartitionMap *inner_map,
+					  bool outer_has_null,
+					  bool inner_has_null,
+					  int outer_null,
+					  int inner_null,
+					  JoinType jointype,
+					  int *next_index,
+					  int *null_index);
 
 /*
  * get_qual_from_partbound
@@ -3554,11 +3555,11 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 			if (ub_cmpval > 0 && inner_part >= 0 &&
 				compare_range_bounds(partnatts, partsupfuncs, partcollations,
 									 &save_outer_ub, &inner_lb) > 0)
-				return NULL;
+				goto cleanup;
 			if (ub_cmpval < 0 && outer_part >= 0 &&
 				compare_range_bounds(partnatts, partsupfuncs, partcollations,
 									 &outer_lb, &save_inner_ub) < 0)
-				return NULL;
+				goto cleanup;
 
 			/*
 			 * A row from a non-overlapping portion (if any) of a partition
@@ -3568,7 +3569,7 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 			 */
 			if ((outer_has_default && (lb_cmpval > 0 || ub_cmpval < 0)) ||
 				(inner_has_default && (lb_cmpval < 0 || ub_cmpval > 0)))
-				return NULL;
+				goto cleanup;
 		}
 		else if (ub_cmpval < 0)
 		{
@@ -3579,17 +3580,17 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 
 			if (inner_has_default || IS_OUTER_JOIN(jointype))
 			{
-				if (!process_outer_partition(&outer_map,
-											 &inner_map,
-											 outer_has_default,
-											 inner_has_default,
-											 outer_part,
-											 inner_default,
-											 jointype,
-											 &next_index,
-											 &default_index,
-											 &merged_index))
-					return NULL;
+				merged_index = process_outer_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   outer_part,
+													   inner_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
 			}
 
 			/* Move to the next partition on the outer side. */
@@ -3606,17 +3607,17 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 
 			if (outer_has_default || jointype == JOIN_FULL)
 			{
-				if (!process_inner_partition(&outer_map,
-											 &inner_map,
-											 outer_has_default,
-											 inner_has_default,
-											 inner_part,
-											 outer_default,
-											 jointype,
-											 &next_index,
-											 &default_index,
-											 &merged_index))
-					return NULL;
+				merged_index = process_inner_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   inner_part,
+													   outer_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
 			}
 
 			/* Move to the next partition on the inner side. */
@@ -3624,7 +3625,7 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 											 &inner_lb, &inner_ub);
 		}
 
-		if (merged_index >= 0)
+		if (merged_index >= 0 && merged_index != default_index)
 		{
 			/* Add the range bounds of the merged partition. */
 			add_merged_range_bounds(partnatts, partsupfuncs, partcollations,
@@ -3636,18 +3637,10 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 
 	/* Merge default partitions if any. */
 	if (outer_has_default || inner_has_default)
-	{
-		if (!merge_default_partitions(&outer_map,
-									  &inner_map,
-									  outer_has_default,
-									  outer_default,
-									  inner_has_default,
-									  inner_default,
-									  jointype,
-									  &next_index,
-									  &default_index))
-			return NULL;
-	}
+		merge_default_partitions(&outer_map, &inner_map,
+								 outer_has_default, inner_has_default,
+								 outer_default, inner_default,
+								 jointype, &next_index, &default_index);
 	else
 		Assert(default_index == -1);
 
@@ -3675,7 +3668,8 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 													  -1, default_index);
 	}
 
-	/* Free any memory we used in this function. */
+cleanup:
+	/* Free local memory before returning. */
 	free_partition_map(&outer_map);
 	free_partition_map(&inner_map);
 	list_free(merged_datums);
@@ -3789,8 +3783,8 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 													o_index, i_index,
 													&next_index);
 
-			if (merged_index < 0)
-				return NULL;
+			if (merged_index == -1)
+				goto cleanup;
 
 			/* Move to the next pair of bounds. */
 			cnto++;
@@ -3808,17 +3802,17 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 				int			o_index = outer_bi->indexes[cnto];
 
 				Assert(o_index >= 0);
-				if (!process_outer_partition(&outer_map,
-											 &inner_map,
-											 outer_has_default,
-											 inner_has_default,
-											 o_index,
-											 inner_default,
-											 jointype,
-											 &next_index,
-											 &default_index,
-											 &merged_index))
-					return NULL;
+				merged_index = process_outer_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   o_index,
+													   inner_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
 			}
 
 			/* Move to the next datum on the outer side. */
@@ -3837,17 +3831,17 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 				int			i_index = inner_bi->indexes[cnti];
 
 				Assert(i_index >= 0);
-				if (!process_inner_partition(&outer_map,
-											 &inner_map,
-											 outer_has_default,
-											 inner_has_default,
-											 i_index,
-											 outer_default,
-											 jointype,
-											 &next_index,
-											 &default_index,
-											 &merged_index))
-					return NULL;
+				merged_index = process_inner_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   i_index,
+													   outer_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
 			}
 
 			/* Move to the next datum on the inner side. */
@@ -3858,7 +3852,7 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 		 * Add the list value with appropriate index in the list of datums, if
 		 * we have associated a partition with this list value.
 		 */
-		if (merged_index >= 0)
+		if (merged_index >= 0 && merged_index != default_index)
 		{
 			merged_indexes = lappend_int(merged_indexes, merged_index);
 			merged_datums = lappend(merged_datums, merged_datum);
@@ -3867,27 +3861,19 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 
 	/* Merge null partitions if any. */
 	if (outer_has_null || inner_has_null)
-	{
-		if (!merge_null_partitions(outer_bi, inner_bi,
-								   &outer_map, &inner_map,
-								   outer_has_default, inner_has_default,
-								   outer_has_null, inner_has_null,
-								   jointype, &next_index, &default_index,
-								   &null_index))
-			return NULL;
-	}
+		merge_null_partitions(&outer_map, &inner_map,
+							  outer_has_null, inner_has_null,
+							  outer_bi->null_index, inner_bi->null_index,
+							  jointype, &next_index, &null_index);
 	else
 		Assert(null_index == -1);
 
 	/* Merge default partitions if any. */
 	if (outer_has_default || inner_has_default)
-	{
-		if (!merge_default_partitions(&outer_map, &inner_map,
-									  outer_has_default, outer_default,
-									  inner_has_default, inner_default,
-									  jointype, &next_index, &default_index))
-			return NULL;
-	}
+		merge_default_partitions(&outer_map, &inner_map,
+								 outer_has_default, inner_has_default,
+								 outer_default, inner_default,
+								 jointype, &next_index, &default_index);
 	else
 		Assert(default_index == -1);
 
@@ -3916,7 +3902,8 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 													  null_index, default_index);
 	}
 
-	/* Free up all extra memory before returning from this function. */
+cleanup:
+	/* Free local memory before returning. */
 	free_partition_map(&outer_map);
 	free_partition_map(&inner_map);
 	list_free(merged_datums);
@@ -4099,7 +4086,7 @@ merge_partition_with_dummy(PartitionMap *map, int index, int *next_index)
  * *next_index is incremented when creating a new merged partition associated
  * with the given outer partition.
  */
-static bool
+static int
 process_outer_partition(PartitionMap *outer_map,
 						PartitionMap *inner_map,
 						bool outer_has_default,
@@ -4108,9 +4095,10 @@ process_outer_partition(PartitionMap *outer_map,
 						int inner_default,
 						JoinType jointype,
 						int *next_index,
-						int *default_index,
-						int *merged_index)
+						int *default_index)
 {
+	int 		merged_index = -1;
+
 	Assert(outer_index >= 0);
 
 	/*
@@ -4131,13 +4119,13 @@ process_outer_partition(PartitionMap *outer_map,
 		 * up on it.
 		 */
 		if (outer_has_default)
-			return false;
+			return -1;
 
-		*merged_index = map_and_merge_partitions(outer_map, inner_map,
-												 outer_index, inner_default,
-												 next_index);
-		if (*merged_index == -1)
-			return false;
+		merged_index = map_and_merge_partitions(outer_map, inner_map,
+												outer_index, inner_default,
+												next_index);
+		if (merged_index == -1)
+			return -1;
 
 		/*
 		 * If this is a FULL join, the merged partition would act as the
@@ -4147,11 +4135,9 @@ process_outer_partition(PartitionMap *outer_map,
 		if (jointype == JOIN_FULL)
 		{
 			if (*default_index == -1)
-				*default_index = *merged_index;
+				*default_index = merged_index;
 			else
-				Assert(*merged_index == *default_index);
-			/* Don't add this index to the list of merged indexes. */
-			*merged_index = -1;
+				Assert(*default_index == merged_index);
 		}
 	}
 	else
@@ -4159,13 +4145,12 @@ process_outer_partition(PartitionMap *outer_map,
 		Assert(IS_OUTER_JOIN(jointype));
 		Assert(jointype != JOIN_RIGHT);
 
-		if (outer_map->merged_indexes[outer_index] >= 0)
-			*merged_index = outer_map->merged_indexes[outer_index];
-		else
-			*merged_index = merge_partition_with_dummy(outer_map, outer_index,
-													   next_index);
+		merged_index = outer_map->merged_indexes[outer_index];
+		if (merged_index == -1)
+			merged_index = merge_partition_with_dummy(outer_map, outer_index,
+													  next_index);
 	}
-	return true;
+	return merged_index;
 }
 
 /*
@@ -4176,7 +4161,7 @@ process_outer_partition(PartitionMap *outer_map,
  * *next_index is incremented when creating a new merged partition associated
  * with the given inner partition.
  */
-static bool
+static int
 process_inner_partition(PartitionMap *outer_map,
 						PartitionMap *inner_map,
 						bool outer_has_default,
@@ -4185,9 +4170,10 @@ process_inner_partition(PartitionMap *outer_map,
 						int outer_default,
 						JoinType jointype,
 						int *next_index,
-						int *default_index,
-						int *merged_index)
+						int *default_index)
 {
+	int 		merged_index = -1;
+
 	Assert(inner_index >= 0);
 
 	/*
@@ -4208,13 +4194,13 @@ process_inner_partition(PartitionMap *outer_map,
 		 * up on it.
 		 */
 		if (inner_has_default)
-			return false;
+			return -1;
 
-		*merged_index = map_and_merge_partitions(outer_map, inner_map,
-												 outer_default, inner_index,
-												 next_index);
-		if (*merged_index == -1)
-			return false;
+		merged_index = map_and_merge_partitions(outer_map, inner_map,
+												outer_default, inner_index,
+												next_index);
+		if (merged_index == -1)
+			return -1;
 
 		/*
 		 * If this is an outer join, the merged partition would act as the
@@ -4225,24 +4211,21 @@ process_inner_partition(PartitionMap *outer_map,
 		{
 			Assert(jointype != JOIN_RIGHT);
 			if (*default_index == -1)
-				*default_index = *merged_index;
+				*default_index = merged_index;
 			else
-				Assert(*merged_index == *default_index);
-			/* Don't add this index to the list of merged indexes. */
-			*merged_index = -1;
+				Assert(*default_index == merged_index);
 		}
 	}
 	else
 	{
 		Assert(jointype == JOIN_FULL);
 
-		if (inner_map->merged_indexes[inner_index] >= 0)
-			*merged_index = inner_map->merged_indexes[inner_index];
-		else
-			*merged_index = merge_partition_with_dummy(inner_map, inner_index,
-													   next_index);
+		merged_index = inner_map->merged_indexes[inner_index];
+		if (merged_index == -1)
+			merged_index = merge_partition_with_dummy(inner_map, inner_index,
+													  next_index);
 	}
-	return true;
+	return merged_index;
 }
 
 /*
@@ -4430,175 +4413,200 @@ build_merged_partition_bounds(char strategy, List *merged_datums,
 }
 
 /*
- * Merge default partitions from both sides, if any, and assign the default
- * partition for the join result, if necessary.
- *
- * If both the relations have default partitions, try mapping those to each
- * other. If the mapping succeeds corresponding merged partition will act as
- * the default partition of the join result.
+ * merge_default_partitions
+ *		Merge the default partitions from both sides, if any
  *
- * If inner side of the join has default but not the outer side, rows in it
- * won't appear in the join result. So don't create a default partition. If
- * outer side of the join has default but not the inner side, rows in it will
- * appear in the join result, so create a default merged partition.
+ * *default_index is set to the index of the default partition of the joinrel,
+ * if appropriate.
  */
-static bool
-merge_default_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
-						 bool outer_has_default, int outer_default,
-						 bool inner_has_default, int inner_default,
-						 JoinType jointype, int *next_index,
+static void
+merge_default_partitions(PartitionMap *outer_map,
+						 PartitionMap *inner_map,
+						 bool outer_has_default,
+						 bool inner_has_default,
+						 int outer_default,
+						 int inner_default,
+						 JoinType jointype,
+						 int *next_index,
 						 int *default_index)
 {
+	int 		outer_merged_index = -1;
+	int 		inner_merged_index = -1;
+
 	Assert(outer_has_default || inner_has_default);
 
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0 && outer_default < outer_map->nparts);
+		outer_merged_index = outer_map->merged_indexes[outer_default];
+	}
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0 && inner_default < inner_map->nparts);
+		inner_merged_index = inner_map->merged_indexes[inner_default];
+	}
+
 	if (outer_has_default && !inner_has_default)
 	{
+		/*
+		 * If this is an outer join, assign a new merged partition if not
+		 * already done.  The resulting partition should be the default
+		 * partition of the joinrel.
+		 */
 		if (IS_OUTER_JOIN(jointype))
 		{
-			int			merged_index;
-
 			Assert(jointype != JOIN_RIGHT);
-			Assert(outer_default >= 0 && outer_default < outer_map->nparts);
-			merged_index = outer_map->merged_indexes[outer_default];
-			if (merged_index == -1)
-			{
-				Assert(*default_index == -1);
+			if (outer_merged_index == -1)
 				*default_index = merge_partition_with_dummy(outer_map,
 															outer_default,
 															next_index);
-			}
 			else
-				Assert(*default_index == merged_index);
+				Assert(*default_index == outer_merged_index);
 		}
 		else
-			Assert(*default_index < 0);
+			Assert(*default_index == -1);
 	}
 	else if (!outer_has_default && inner_has_default)
 	{
+		/*
+		 * If this is a FULL join, assign a new merged partition if not
+		 * already done.  The resulting partition should be the default
+		 * partition of the joinrel.
+		 */
 		if (jointype == JOIN_FULL)
 		{
-			int			merged_index;
-
-			Assert(inner_default >= 0 && inner_default < inner_map->nparts);
-			merged_index = inner_map->merged_indexes[inner_default];
-			if (merged_index == -1)
-			{
-				Assert(*default_index == -1);
+			if (inner_merged_index == -1)
 				*default_index = merge_partition_with_dummy(inner_map,
 															inner_default,
 															next_index);
-			}
 			else
-				Assert(*default_index == merged_index);
+				Assert(*default_index == inner_merged_index);
 		}
 		else
-			Assert(*default_index < 0);
+			Assert(*default_index == -1);
 	}
 	else
 	{
 		Assert(outer_has_default && inner_has_default);
+		Assert(outer_merged_index == -1);
+		Assert(inner_merged_index == -1);
+		Assert(*default_index == -1);
 
+		/*
+		 * Merge the default partitions.  Both partitions aren't merged yet,
+		 * so the partitions should be merged successfully.  The resulting
+		 * partition should be the default partition of the joinrel.
+		 */
 		*default_index = map_and_merge_partitions(outer_map,
 												  inner_map,
 												  outer_default,
 												  inner_default,
 												  next_index);
-		if (*default_index == -1)
-			return false;
+		Assert(*default_index >= 0);
 	}
-
-	return true;
 }
 
 /*
  * merge_null_partitions
+ *		Merge the NULL partitions from both sides, if any
  *
- * Merge NULL partitions, i.e. a partition that can hold NULL values for a list
- * partitioned table, if any. Find the index of merged partition to which the
- * NULL values would belong in the join result. If one joining relation has a
- * NULL partition but not the other, try matching it with the default partition
- * from the other relation since the default partition may have rows with NULL
- * partition key. We can eliminate a NULL partition when it appears only on the
- * inner side of the join and the outer side doesn't have a default partition.
- *
- * When the equality operator used for join is strict, two NULL values will not
- * be considered as equal, and thus a NULL partition can be eliminated for an
- * inner join. But we don't check the strictness operator here.
+ * *null_index is set to the index of the NULL partition of the joinrel, if
+ * appropriate.
  */
-static bool
-merge_null_partitions(PartitionBoundInfo outer_bi, PartitionBoundInfo inner_bi,
-					  PartitionMap *outer_map, PartitionMap *inner_map,
-					  bool outer_has_default, bool inner_has_default,
-					  bool outer_has_null, bool inner_has_null,
-					  JoinType jointype, int *next_index,
-					  int *default_index, int *null_index)
+static void
+merge_null_partitions(PartitionMap *outer_map,
+					  PartitionMap *inner_map,
+					  bool outer_has_null,
+					  bool inner_has_null,
+					  int outer_null,
+					  int inner_null,
+					  JoinType jointype,
+					  int *next_index,
+					  int *null_index)
 {
+	bool 		merge_outer_null = false;
+	bool 		merge_inner_null = false;
+
 	Assert(outer_has_null || inner_has_null);
 	Assert(*null_index == -1);
 
-	if (outer_has_null && !inner_has_null)
+	/*
+	 * If this is an INNER or SEMI join, two NULL values won't be considered
+	 * to be equal, in which case we don't need to do anything.
+	 */
+	if (jointype == JOIN_INNER || jointype == JOIN_SEMI)
+		return;
+
+	Assert(IS_OUTER_JOIN(jointype));
+	Assert(jointype != JOIN_RIGHT);
+
+	if (outer_has_null)
 	{
-		int			merged_index = -1;
+		/*
+		 * Since this is an outer join, if the NULL partition of the outer
+		 * side isn't merged yet, it means that the partition contains only
+		 * NULL values as the key values, in which case we need to create a
+		 * merged partition below.
+		 */
+		Assert(outer_null >= 0 && outer_null < outer_map->nparts);
+		if (outer_map->merged_indexes[outer_null] == -1)
+			merge_outer_null = true;
+	}
+	if (inner_has_null)
+	{
+		/*
+		 * If this is a FULL join, and the NULL partition of the inner side
+		 * isn't merged yet, it means that the partition contains only NULL
+		 * values as the key values, in which case we need to create a merged
+		 * partition below.
+		 */
+		Assert(inner_null >= 0 && inner_null < inner_map->nparts);
+		if (jointype == JOIN_FULL &&
+			inner_map->merged_indexes[inner_null] == -1)
+			merge_inner_null = true;
+	}
+
+	if (!merge_outer_null && !merge_inner_null)
+		return;
+
+	if (merge_outer_null && !merge_inner_null)
+	{
+		Assert(outer_has_null);
 
 		/*
-		 * If the NULL partition was missing from the inner side of the join,
-		 * the partition of the join to which the NULL partition matches will
-		 * contain the NULL values and thus become the NULL partition of the
-		 * the join.
+		 * Assign a new merged partition.  The resulting partition should be
+		 * the NULL partition of the joinrel.
 		 */
-		if (inner_has_default || IS_OUTER_JOIN(jointype))
-		{
-			if (!process_outer_partition(outer_map,
-										 inner_map,
-										 outer_has_default,
-										 inner_has_default,
-										 outer_bi->null_index,
-										 inner_bi->default_index,
-										 jointype,
-										 next_index,
-										 default_index,
-										 &merged_index))
-				return false;
-		}
-		*null_index = merged_index;
+		*null_index = merge_partition_with_dummy(outer_map, outer_null,
+												 next_index);
 	}
-	else if (!outer_has_null && inner_has_null)
+	else if (!merge_outer_null && merge_inner_null)
 	{
-		int			merged_index = -1;
+		Assert(inner_has_null);
+		Assert(jointype == JOIN_FULL);
 
 		/*
-		 * If the NULL partition was missing from the outer side of the join,
-		 * the partition of the join to which the NULL partition matches will
-		 * contain the NULL values and thus become the NULL partition of the
-		 * the join.
+		 * Assign a new merged partition.  The resulting partition should be
+		 * the NULL partition of the joinrel.
 		 */
-		if (outer_has_default || jointype == JOIN_FULL)
-		{
-			if (!process_inner_partition(outer_map,
-										 inner_map,
-										 outer_has_default,
-										 inner_has_default,
-										 inner_bi->null_index,
-										 outer_bi->default_index,
-										 jointype,
-										 next_index,
-										 default_index,
-										 &merged_index))
-				return false;
-		}
-		*null_index = merged_index;
+		*null_index = merge_partition_with_dummy(inner_map, inner_null,
+												 next_index);
 	}
 	else
 	{
-		/* Both the relations have NULL partitions, try merging them. */
-		*null_index = map_and_merge_partitions(outer_map,
-											   inner_map,
-											   outer_bi->null_index,
-											   inner_bi->null_index,
+		Assert(merge_outer_null && merge_inner_null);
+		Assert(outer_has_null);
+		Assert(inner_has_null);
+		Assert(jointype == JOIN_FULL);
+
+		/*
+		 * Merge the NULL partitions.  Both partitions aren't merged yet, so
+		 * the partitions should be merged successfully.  The resulting
+		 * partition should be the NULL partition of the joinrel.
+		 */
+		*null_index = map_and_merge_partitions(outer_map, inner_map,
+											   outer_null, inner_null,
 											   next_index);
-		if (*null_index == -1)
-			return false;
+		Assert(*null_index >= 0);
 	}
-
-	return true;
 }
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 815a6c90749..01fe9278cc9 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -3866,10 +3866,7 @@ select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHE
  470 | 0 | 
 (1 row)
 
--- partition have a NULL on both side with different partition bounds w.r.t other side
--- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
--- Partition-wise join can not handle the case when one partition from one side
--- matches with multiple partitions on the other side
+-- test the null-partition handling in the new partition-matching algorithm
 DROP TABLE plt1_p3;
 CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
 INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
@@ -3877,94 +3874,138 @@ ANALYZE plt1;
 -- inner join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 INNER JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + t2.b = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                         QUERY PLAN                         
+------------------------------------------------------------
  Sort
    Sort Key: t1.a
-   ->  Hash Join
-         Hash Cond: ((t2.c)::text = (t1.c)::text)
-         Join Filter: ((t1.b + t2.b) = 0)
-         ->  Append
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Join Filter: ((t1.b + t2.b) = 0)
                ->  Seq Scan on plt2_p4 t2
-               ->  Seq Scan on plt2_p1 t2_1
-               ->  Seq Scan on plt2_p2 t2_2
-               ->  Seq Scan on plt2_p3 t2_3
-         ->  Hash
-               ->  Append
+               ->  Hash
                      ->  Seq Scan on plt1_p4 t1
+         ->  Hash Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Join Filter: ((t1_1.b + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
                      ->  Seq Scan on plt1_p1 t1_1
-                     ->  Seq Scan on plt1_p2 t1_2
-                     ->  Seq Scan on plt1_p3 t1_3
-(16 rows)
+         ->  Hash Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Join Filter: ((t1_2.b + t2_2.b) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Join Filter: ((t1_3.b + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
 
 -- left join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.c = t2.c WHERE t1.b + coalesce(t2.b, 0) = 0 ORDER BY t1.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                         QUERY PLAN                         
+------------------------------------------------------------
  Sort
    Sort Key: t1.a
-   ->  Hash Right Join
-         Hash Cond: ((t2.c)::text = (t1.c)::text)
-         Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
-         ->  Append
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((t1.b + COALESCE(t2.b, 0)) = 0)
                ->  Seq Scan on plt2_p4 t2
-               ->  Seq Scan on plt2_p1 t2_1
-               ->  Seq Scan on plt2_p2 t2_2
-               ->  Seq Scan on plt2_p3 t2_3
-         ->  Hash
-               ->  Append
+               ->  Hash
                      ->  Seq Scan on plt1_p4 t1
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((t1_1.b + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
                      ->  Seq Scan on plt1_p1 t1_1
-                     ->  Seq Scan on plt1_p2 t1_2
-                     ->  Seq Scan on plt1_p3 t1_3
-(16 rows)
+         ->  Hash Left Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((t1_2.b + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Left Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((t1_3.b + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
 
 -- right join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + t2.b = 0 ORDER BY t2.a;
-                    QUERY PLAN                    
---------------------------------------------------
+                         QUERY PLAN                         
+------------------------------------------------------------
  Sort
    Sort Key: t2.a
-   ->  Hash Left Join
-         Hash Cond: ((t2.c)::text = (t1.c)::text)
-         Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
-         ->  Append
+   ->  Append
+         ->  Hash Left Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + t2.b) = 0)
                ->  Seq Scan on plt2_p4 t2
-               ->  Seq Scan on plt2_p1 t2_1
-               ->  Seq Scan on plt2_p2 t2_2
-               ->  Seq Scan on plt2_p3 t2_3
-         ->  Hash
-               ->  Append
+               ->  Hash
                      ->  Seq Scan on plt1_p4 t1
+         ->  Hash Left Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + t2_1.b) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
                      ->  Seq Scan on plt1_p1 t1_1
+         ->  Hash Left Join
+               Hash Cond: ((t2_2.c)::text = (t1_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + t2_2.b) = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+               ->  Hash
                      ->  Seq Scan on plt1_p2 t1_2
-                     ->  Seq Scan on plt1_p3 t1_3
-(16 rows)
+         ->  Hash Right Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + t2_3.b) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
 
 -- full join
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.a, t2.c FROM plt1 t1 FULL JOIN plt2 t2 ON t1.c = t2.c WHERE coalesce(t1.b, 0) + coalesce(t2.b, 0) = 0 ORDER BY t1.a, t2.a;
-                          QUERY PLAN                           
----------------------------------------------------------------
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
  Sort
    Sort Key: t1.a, t2.a
-   ->  Hash Full Join
-         Hash Cond: ((t2.c)::text = (t1.c)::text)
-         Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
-         ->  Append
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t2.c)::text = (t1.c)::text)
+               Filter: ((COALESCE(t1.b, 0) + COALESCE(t2.b, 0)) = 0)
                ->  Seq Scan on plt2_p4 t2
-               ->  Seq Scan on plt2_p1 t2_1
-               ->  Seq Scan on plt2_p2 t2_2
-               ->  Seq Scan on plt2_p3 t2_3
-         ->  Hash
-               ->  Append
+               ->  Hash
                      ->  Seq Scan on plt1_p4 t1
+         ->  Hash Full Join
+               Hash Cond: ((t2_1.c)::text = (t1_1.c)::text)
+               Filter: ((COALESCE(t1_1.b, 0) + COALESCE(t2_1.b, 0)) = 0)
+               ->  Seq Scan on plt2_p1 t2_1
+               ->  Hash
                      ->  Seq Scan on plt1_p1 t1_1
-                     ->  Seq Scan on plt1_p2 t1_2
-                     ->  Seq Scan on plt1_p3 t1_3
-(16 rows)
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.c)::text = (t2_2.c)::text)
+               Filter: ((COALESCE(t1_2.b, 0) + COALESCE(t2_2.b, 0)) = 0)
+               ->  Seq Scan on plt1_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.c)::text = (t2_3.c)::text)
+               Filter: ((COALESCE(t1_3.b, 0) + COALESCE(t2_3.b, 0)) = 0)
+               ->  Seq Scan on plt1_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_p3 t2_3
+(27 rows)
 
 -- semi join
 EXPLAIN (COSTS OFF)
@@ -3973,27 +4014,34 @@ select t1.a, t1.b, t1.c from plt1 t1 where exists (select 1 from plt2 t2 WHERE t
 --------------------------------------------------------------
  Sort
    Sort Key: t1.a, t1.c
-   ->  Hash Join
-         Hash Cond: ((t1.c)::text = (t2.c)::text)
-         ->  Append
-               ->  Seq Scan on plt1_p4 t1
-                     Filter: (b = 0)
-               ->  Seq Scan on plt1_p1 t1_1
-                     Filter: (b = 0)
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt2_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt1_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2_1.c)::text
+                     ->  Seq Scan on plt2_p1 t2_1
+               ->  Materialize
+                     ->  Seq Scan on plt1_p1 t1_1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
                ->  Seq Scan on plt1_p2 t1_2
                      Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
                ->  Seq Scan on plt1_p3 t1_3
                      Filter: (b = 0)
-         ->  Hash
-               ->  HashAggregate
-                     Group Key: (t2.c)::text
-                     ->  Result
-                           ->  Append
-                                 ->  Seq Scan on plt2_p4 t2
-                                 ->  Seq Scan on plt2_p1 t2_1
-                                 ->  Seq Scan on plt2_p2 t2_2
-                                 ->  Seq Scan on plt2_p3 t2_3
-(22 rows)
+               ->  Seq Scan on plt2_p3 t2_3
+(29 rows)
 
 EXPLAIN (COSTS OFF)
 select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
@@ -4001,78 +4049,92 @@ select t1.a, t1.b, t1.c from plt2 t1 where exists (select 1 from plt1 t2 WHERE t
 --------------------------------------------------------------
  Sort
    Sort Key: t1.a, t1.c
-   ->  Hash Join
-         Hash Cond: ((t1.c)::text = (t2.c)::text)
-         ->  Append
-               ->  Seq Scan on plt2_p4 t1
-                     Filter: (b = 0)
+   ->  Append
+         ->  Nested Loop
+               Join Filter: ((t1.c)::text = (t2.c)::text)
+               ->  HashAggregate
+                     Group Key: (t2.c)::text
+                     ->  Seq Scan on plt1_p4 t2
+               ->  Materialize
+                     ->  Seq Scan on plt2_p4 t1
+                           Filter: (b = 0)
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
                ->  Seq Scan on plt2_p1 t1_1
                      Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
                ->  Seq Scan on plt2_p2 t1_2
                      Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
                ->  Seq Scan on plt2_p3 t1_3
                      Filter: (b = 0)
-         ->  Hash
-               ->  HashAggregate
-                     Group Key: (t2.c)::text
-                     ->  Result
-                           ->  Append
-                                 ->  Seq Scan on plt1_p4 t2
-                                 ->  Seq Scan on plt1_p1 t2_1
-                                 ->  Seq Scan on plt1_p2 t2_2
-                                 ->  Seq Scan on plt1_p3 t2_3
-(22 rows)
+               ->  Seq Scan on plt1_p3 t2_3
+(26 rows)
 
 -- anti join
 EXPLAIN (COSTS OFF)
 select t1.a, t1.b, t1.c from plt1 t1 where not exists (select 1 from plt2 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                          
+--------------------------------------------------------------
  Sort
    Sort Key: t1.a, t1.c
-   ->  Hash Anti Join
-         Hash Cond: ((t1.c)::text = (t2.c)::text)
-         ->  Append
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1.c)::text = (t2.c)::text)
                ->  Seq Scan on plt1_p4 t1
                      Filter: (b = 0)
+               ->  Seq Scan on plt2_p4 t2
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.c)::text = (t2_1.c)::text)
                ->  Seq Scan on plt1_p1 t1_1
                      Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt2_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
                ->  Seq Scan on plt1_p2 t1_2
                      Filter: (b = 0)
+               ->  Seq Scan on plt2_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
                ->  Seq Scan on plt1_p3 t1_3
                      Filter: (b = 0)
-         ->  Hash
-               ->  Append
-                     ->  Seq Scan on plt2_p4 t2
-                     ->  Seq Scan on plt2_p1 t2_1
-                     ->  Seq Scan on plt2_p2 t2_2
-                     ->  Seq Scan on plt2_p3 t2_3
-(19 rows)
+               ->  Seq Scan on plt2_p3 t2_3
+(24 rows)
 
 EXPLAIN (COSTS OFF)
 select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
-                    QUERY PLAN                    
---------------------------------------------------
+                          QUERY PLAN                          
+--------------------------------------------------------------
  Sort
    Sort Key: t1.a, t1.c
-   ->  Hash Anti Join
-         Hash Cond: ((t1.c)::text = (t2.c)::text)
-         ->  Append
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1.c)::text = (t2.c)::text)
                ->  Seq Scan on plt2_p4 t1
                      Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on plt1_p4 t2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.c)::text = (t2_1.c)::text)
                ->  Seq Scan on plt2_p1 t1_1
                      Filter: (b = 0)
+               ->  Seq Scan on plt1_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.c)::text = (t2_2.c)::text)
                ->  Seq Scan on plt2_p2 t1_2
                      Filter: (b = 0)
+               ->  Seq Scan on plt1_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.c)::text = (t2_3.c)::text)
                ->  Seq Scan on plt2_p3 t1_3
                      Filter: (b = 0)
-         ->  Hash
-               ->  Append
-                     ->  Seq Scan on plt1_p4 t2
-                     ->  Seq Scan on plt1_p1 t2_1
-                     ->  Seq Scan on plt1_p2 t2_2
-                     ->  Seq Scan on plt1_p3 t2_3
-(19 rows)
+               ->  Seq Scan on plt1_p3 t2_3
+(24 rows)
 
 -- joins where one of the relations is proven empty
 EXPLAIN (COSTS OFF)
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index d8434f6b1ad..a552874742f 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -561,10 +561,7 @@ EXPLAIN (COSTS OFF)
 select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 select t1.a, t1.b, t1.c from plt2 t1 where not exists (select 1 from plt1 t2 WHERE t1.c = t2.c) and t1.b = 0 order by t1.a, t1.b, t1.c;
 
--- partition have a NULL on both side with different partition bounds w.r.t other side
--- NULL when NULL comparision is not strict i.e. NULL=NULL allowed
--- Partition-wise join can not handle the case when one partition from one side
--- matches with multiple partitions on the other side
+-- test the null-partition handling in the new partition-matching algorithm
 DROP TABLE plt1_p3;
 CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN (NULL,'0008','0009');
 INSERT INTO plt1 SELECT i, i % :cond_mod, case when i % :part_mod = 7 then NULL else to_char(i % :part_mod, 'FM0000') end FROM generate_series(0, :num_rows) i WHERE i % :part_mod IN (7,8,9);
-- 
2.18.0

#110amul sul
sulamul@gmail.com
In reply to: amul sul (#109)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Mon, Dec 9, 2019 at 3:08 PM amul sul <sulamul@gmail.com> wrote:

Attached is the rebase version atop the latest master head(2d0fdfaccec).

Hi Fujita san,

I have been through your changes proposed in [1] -- the changes make sense
to me &
I didn't see any unobvious behaviour in testing as well.

Regards,
Amul

1]
/messages/by-id/CAPmGK15kXh76EnRn=B64u=+qLxZOKWROd4uMjMBnWcsZPdQ_mw@mail.gmail.com

#111Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: amul sul (#110)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Hi Amul,

On Tue, Dec 10, 2019 at 3:49 PM amul sul <sulamul@gmail.com> wrote:

On Mon, Dec 9, 2019 at 3:08 PM amul sul <sulamul@gmail.com> wrote:

Attached is the rebase version atop the latest master head(2d0fdfaccec).

Thanks for that!

I have been through your changes proposed in [1] -- the changes make sense to me &
I didn't see any unobvious behaviour in testing as well.

Thanks for reviewing! I'll merge the changes into the main patch,
then. I don't see any issues in the latest version, but I think we
need to polish the patch, so I'll do that.

Best regards,
Etsuro Fujita

#112Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Etsuro Fujita (#111)
2 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Tue, Dec 10, 2019 at 7:30 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

I don't see any issues in the latest version, but I think we
need to polish the patch, so I'll do that.

I noticed some issues. :-( I think we should address it before
polishing the patch. One thing I noticed is: the patch heavily
modifies the existing test cases in partition_join.sql to test the new
partition-matching algorithm, but I think we should leave those test
cases alone because we would handle the exiting test cases (except one
negative test case) as before (see the try_partitionwise_join()
change in the patch), so those test cases would be still needed to
test that. Attached is a proposed patch for that
(v30-0001-Improve-partition-matching-for-partitionwise-join.patch)
that 1) avoids modifying the existing test cases and 2) adds a
slightly modified version of the test cases proposed in the previous
patch to test the new algorithm. Though I omitted some test cases
that seem redundant to me and added a bit more test cases involving
NULL partitions and/or default partitions. The elapsed time to run
the partition_join.sql regression test increased from 741 ms (HEAD) to
1086 ms in my environment, but I think that would be acceptable. I
fixed one white space issue, but other than that, no code/comment
changes.

Another thing I noticed while working on the above is: the patch fails
to apply PWJ to this case:

CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0001', '0003');
CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0004', '0006');
CREATE TABLE plt1_ad_p3 PARTITION OF plt1_ad FOR VALUES IN ('0008', '0009');
CREATE TABLE plt1_ad_extra PARTITION OF plt1_ad FOR VALUES IN (NULL);
INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM
generate_series(1, 299) i WHERE i % 10 NOT IN (0, 2, 5, 7);
INSERT INTO plt1_ad VALUES (-1, -1, NULL);
ANALYZE plt1_ad;
CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c);
CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0002', '0003');
CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0004', '0006');
CREATE TABLE plt2_ad_p3 PARTITION OF plt2_ad FOR VALUES IN ('0007', '0009');
CREATE TABLE plt2_ad_extra PARTITION OF plt2_ad FOR VALUES IN (NULL);
INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM
generate_series(1, 299) i WHERE i % 10 NOT IN (0, 1, 5, 8);
INSERT INTO plt2_ad VALUES (-1, -1, NULL);
ANALYZE plt2_ad;

EXPLAIN (COSTS OFF)
SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON
(t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
QUERY PLAN
--------------------------------------------------------
Sort
Sort Key: t1.a
-> Hash Right Join
Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
-> Append
-> Seq Scan on plt2_ad_p1 t2_1
-> Seq Scan on plt2_ad_p2 t2_2
-> Seq Scan on plt2_ad_p3 t2_3
-> Seq Scan on plt2_ad_extra t2_4
-> Hash
-> Append
-> Seq Scan on plt1_ad_p1 t1_1
Filter: (b < 10)
-> Seq Scan on plt1_ad_p2 t1_2
Filter: (b < 10)
-> Seq Scan on plt1_ad_p3 t1_3
Filter: (b < 10)
-> Seq Scan on plt1_ad_extra t1_4
Filter: (b < 10)
(19 rows)

because merge_null_partitions() does not consider matching the NULL
partitions from both sides, but matches the NULL partition on the
plt1_ad side and a dummy partition, resulting in a non-PWJ plan (see
[1]: https://git.postgresql.org/gitweb/?p=postgresql.git;a=commit;h=7ad6498fd5a654de6e743814c36cf619a3b5ddb6
Another patch attached to fix this issue
(v30-0002-Fix-handling-of-NULL-partitions.patch). (We would not need
to fix this, if we could handle the case where a dummy partition is on
the nullable side of an outer join [1]https://git.postgresql.org/gitweb/?p=postgresql.git;a=commit;h=7ad6498fd5a654de6e743814c36cf619a3b5ddb6, but we can't, so I think it
would be a good idea at least for now to match the NULL partitions
from both sides to do PWJ.)

Best regards,
Etsuro Fujita

[1]: https://git.postgresql.org/gitweb/?p=postgresql.git;a=commit;h=7ad6498fd5a654de6e743814c36cf619a3b5ddb6

Attachments:

v30-0001-Improve-partition-matching-for-partitionwise-join.patchapplication/octet-stream; name=v30-0001-Improve-partition-matching-for-partitionwise-join.patchDownload
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index d76fae44b8..e42a52fb9b 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -2285,6 +2285,8 @@ _outRelOptInfo(StringInfo str, const RelOptInfo *node)
 	WRITE_BOOL_FIELD(has_eclass_joins);
 	WRITE_BOOL_FIELD(consider_partitionwise_join);
 	WRITE_BITMAPSET_FIELD(top_parent_relids);
+	WRITE_BOOL_FIELD(merged);
+	WRITE_BITMAPSET_FIELD(all_partrels);
 	WRITE_NODE_FIELD(partitioned_child_rels);
 }
 
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index a21c295b99..77e6ff5376 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -48,6 +48,9 @@ static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 												Relids left_relids, Relids right_relids);
 static int	match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel,
 										 bool strict_op);
+static void get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2);
 
 
 /*
@@ -1357,25 +1360,30 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	bool		merged = false;
+	List	   *parts1 = NIL;
+	List	   *parts2 = NIL;
+	ListCell   *lcr1 = NULL;
+	ListCell   *lcr2 = NULL;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1390,34 +1398,107 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		   joinrel->part_scheme == rel2->part_scheme);
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * If we don't have the partition bounds for the join rel yet, try to
+	 * create it along with pairs of partitions to be joined; else generate
+	 * those using the partitioning info for the join rel we already have.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	if (joinrel->nparts == -1)
+	{
+		PartitionScheme part_scheme = joinrel->part_scheme;
+		PartitionBoundInfo boundinfo = NULL;
+		int			nparts = 0;
+
+		Assert(joinrel->boundinfo == NULL);
+		Assert(joinrel->part_rels == NULL);
+
+		/*
+		 * See if the partition bounds for inputs are exactly the same, in
+		 * which case we don't need to work hard: partitions with the same
+		 * partition indexes will form join pairs, and the join rel will have
+		 * the same partition bounds as inputs; otherwise try to merge the
+		 * partition bounds along with generating join pairs.
+		 *
+		 * Even if one or both inputs have merged partition bounds, it'd be
+		 * possible for the partition bounds to be exactly the same, but it
+		 * seems unlikely to be worth the cycles to check; do this check only
+		 * if both inputs have non-merged partition bounds.
+		 */
+		if (!rel1->merged &&
+			!rel2->merged &&
+			rel1->nparts == rel2->nparts &&
+			partition_bounds_equal(part_scheme->partnatts,
+								   part_scheme->parttyplen,
+								   part_scheme->parttypbyval,
+								   rel1->boundinfo, rel2->boundinfo))
+		{
+			boundinfo = rel1->boundinfo;
+			nparts = rel1->nparts;
+		}
+		else
+		{
+			boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											   part_scheme->parttyplen,
+											   part_scheme->parttypbyval,
+											   part_scheme->partsupfunc,
+											   part_scheme->partcollation,
+											   rel1, rel2,
+											   parent_sjinfo->jointype,
+											   &parts1, &parts2);
+			if (boundinfo == NULL)
+			{
+				joinrel->nparts = 0;
+				return;
+			}
+			nparts = list_length(parts1);
+			merged = true;
+		}
+
+		Assert(nparts > 0);
+		joinrel->boundinfo = boundinfo;
+		joinrel->merged = merged;
+		joinrel->nparts = nparts;
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+	}
+	else
+	{
+		Assert(joinrel->nparts > 0);
+		Assert(joinrel->boundinfo);
+		Assert(joinrel->part_rels);
+
+		/*
+		 * If the partition bounds for the join rel are not merged ones,
+		 * inputs are guaranteed to have the same partition bounds, so
+		 * partitions with the same partition indexes will form join pairs;
+		 * else let get_matching_part_pairs() do the work.
+		 */
+		if (joinrel->merged)
+		{
+			get_matching_part_pairs(root, joinrel, rel1, rel2,
+									&parts1, &parts2);
+			Assert(list_length(parts1) == joinrel->nparts);
+			Assert(list_length(parts2) == joinrel->nparts);
+			merged = true;
+		}
+	}
 
-	nparts = joinrel->nparts;
+	if (merged)
+	{
+		lcr1 = list_head(parts1);
+		lcr2 = list_head(parts2);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
-		bool		rel1_empty = (child_rel1 == NULL ||
-								  IS_DUMMY_REL(child_rel1));
-		bool		rel2_empty = (child_rel2 == NULL ||
-								  IS_DUMMY_REL(child_rel2));
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		bool		rel1_empty;
+		bool		rel2_empty;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1425,6 +1506,22 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		if (merged)
+		{
+			child_rel1 = lfirst_node(RelOptInfo, lcr1);
+			child_rel2 = lfirst_node(RelOptInfo, lcr2);
+			lcr1 = lnext(parts1, lcr1);
+			lcr2 = lnext(parts2, lcr2);
+		}
+		else
+		{
+			child_rel1 = rel1->part_rels[cnt_parts];
+			child_rel2 = rel2->part_rels[cnt_parts];
+		}
+
+		rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1));
+		rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2));
+
 		/*
 		 * Check for cases where we can prove that this segment of the join
 		 * returns no rows, due to one or both inputs being empty (including
@@ -1522,6 +1619,8 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 												 child_sjinfo,
 												 child_sjinfo->jointype);
 			joinrel->part_rels[cnt_parts] = child_joinrel;
+			joinrel->all_partrels = bms_add_members(joinrel->all_partrels,
+													child_joinrel->relids);
 		}
 
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
@@ -1738,3 +1837,99 @@ match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op)
 
 	return -1;
 }
+
+/*
+ * get_matching_part_pairs
+ *		Generate join pairs of partitions for the two inputs
+ */
+static void
+get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2)
+{
+	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
+	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
+	int 		cnt_parts;
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
+	{
+		RelOptInfo *child_joinrel = joinrel->part_rels[cnt_parts];
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		Relids		child_relids1;
+		Relids		child_relids2;
+
+		/*
+		 * If this segment of the join is empty, it means that this segment
+		 * was ignored when previously creating child-join paths for it in
+		 * try_partitionwise_join() as it would not contribute to the join
+		 * result, due to one or both inputs being empty; add NULL to each of
+		 * the given lists so that this segment will be ignored again in that
+		 * function.
+		 */
+		if (!child_joinrel)
+		{
+			*parts1 = lappend(*parts1, NULL);
+			*parts2 = lappend(*parts2, NULL);
+			continue;
+		}
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel1 side.
+		 */
+		child_relids1 = bms_intersect(child_joinrel->relids,
+									  rel1->all_partrels);
+		Assert(bms_num_members(child_relids1) == bms_num_members(rel1->relids));
+
+		/*
+		 * Get a child rel for rel1 with the relids.  Note that we should have
+		 * the child rel even if rel1 is a join rel, because in that case the
+		 * partitions specified in the relids would have matching/overlapping
+		 * boundaries, so those partitions should be considered as ones to be
+		 * joined even when planning partitionwise joins of rel1, meaning that
+		 * the child rel would have been built by the time we get here.
+		 */
+		if (rel1_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids1);
+
+			child_rel1 = find_base_rel(root, varno);
+		}
+		else
+			child_rel1 = find_join_rel(root, child_relids1);
+		Assert(child_rel1);
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel2 side.
+		 */
+		child_relids2 = bms_intersect(child_joinrel->relids,
+									  rel2->all_partrels);
+		Assert(bms_num_members(child_relids2) == bms_num_members(rel2->relids));
+
+		/*
+		 * Get a child rel for rel2 with the relids.  See above comments.
+		 */
+		if (rel2_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids2);
+
+			child_rel2 = find_base_rel(root, varno);
+		}
+		else
+			child_rel2 = find_join_rel(root, child_relids2);
+		Assert(child_rel2);
+
+		/*
+		 * The join of rel1 and rel2 is legal, so is the join of the child
+		 * rels obtained above; add them to the given lists as a join pair
+		 * producing this join segment.
+		 */
+		*parts1 = lappend(*parts1, child_rel1);
+		*parts2 = lappend(*parts2, child_rel2);
+	}
+}
diff --git a/src/backend/optimizer/util/inherit.c b/src/backend/optimizer/util/inherit.c
index 7db67fdf34..3132fd35a5 100644
--- a/src/backend/optimizer/util/inherit.c
+++ b/src/backend/optimizer/util/inherit.c
@@ -376,6 +376,8 @@ expand_partitioned_rtentry(PlannerInfo *root, RelOptInfo *relinfo,
 		/* Create the otherrel RelOptInfo too. */
 		childrelinfo = build_simple_rel(root, childRTindex, relinfo);
 		relinfo->part_rels[i] = childrelinfo;
+		relinfo->all_partrels = bms_add_members(relinfo->all_partrels,
+												childrelinfo->relids);
 
 		/* If this child is itself partitioned, recurse */
 		if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 374f93890b..1df6da64ed 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -240,10 +240,12 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->has_eclass_joins = false;
 	rel->consider_partitionwise_join = false;	/* might get changed later */
 	rel->part_scheme = NULL;
-	rel->nparts = 0;
+	rel->nparts = -1;
 	rel->boundinfo = NULL;
+	rel->merged = false;
 	rel->partition_qual = NIL;
 	rel->part_rels = NULL;
+	rel->all_partrels = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->partitioned_child_rels = NIL;
@@ -653,10 +655,12 @@ build_join_rel(PlannerInfo *root,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -829,10 +833,12 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -1639,7 +1645,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1652,24 +1658,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1681,17 +1669,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 95af37b9c7..d619ab4486 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -25,6 +25,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/pathnodes.h"
 #include "parser/parse_coerce.h"
 #include "partitioning/partbounds.h"
 #include "partitioning/partdesc.h"
@@ -68,6 +69,23 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+typedef struct PartitionMap
+{
+	int			nparts;			/* number of partitions */
+	int		   *merged_indexes;	/* indexes of merged partitions */
+	bool	   *merged;			/* flags to indicate whether partitions are
+								 * merged with non-dummy partitions */
+	bool		did_remapping;	/* did we remap partitions? */
+	int		   *old_indexes;	/* old indexes of merged partitions if
+								 * did_remapping */
+} PartitionMap;
+
+#define compare_range_bounds(partnatts, partsupfunc, partcollations, \
+							 bound1, bound2) \
+	(partition_rbound_cmp(partnatts, partsupfunc, partcollations, \
+						  (bound1)->datums, (bound1)->kind, (bound1)->lower, \
+						  bound2))
+
 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);
@@ -107,6 +125,96 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 									 Expr **keyCol,
 									 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static PartitionBoundInfo partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts);
+static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							JoinType jointype,
+							List **outer_parts, List **inner_parts);
+static void init_partition_map(RelOptInfo *rel, PartitionMap *map);
+static void free_partition_map(PartitionMap *map);
+static int map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_part, int inner_part, int *next_index);
+static int merge_partition_with_dummy(PartitionMap *map, int index,
+						   int *next_index);
+static int process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index);
+static int process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index);
+static void fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes);
+static void generate_matching_part_pairs(RelOptInfo *outer_rel,
+							 RelOptInfo *inner_rel,
+							 PartitionMap *outer_map,
+							 PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_part_list,
+							 List **inner_part_list);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int get_range_partition(PartitionBoundInfo bi, int *lb_pos,
+					PartitionRangeBound *lb, PartitionRangeBound *ub);
+static bool compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+						 Oid *partcollations,
+						 PartitionRangeBound *outer_lb,
+						 PartitionRangeBound *outer_ub,
+						 PartitionRangeBound *inner_lb,
+						 PartitionRangeBound *inner_ub,
+						 int *lb_cmpval, int *ub_cmpval);
+static void get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations, JoinType jointype,
+						PartitionRangeBound *outer_lb,
+						PartitionRangeBound *outer_ub,
+						PartitionRangeBound *inner_lb,
+						PartitionRangeBound *inner_ub,
+						int	lb_cmpval, int ub_cmpval,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub);
+static void add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub,
+						int merged_index,
+						List **merged_datums,
+						List **merged_kinds,
+						List **merged_indexes);
+static void merge_default_partitions(PartitionMap *outer_map,
+						 PartitionMap *inner_map,
+						 bool outer_has_default,
+						 bool inner_has_default,
+						 int outer_default,
+						 int inner_default,
+						 JoinType jointype,
+						 int *next_index,
+						 int *default_index);
+static void merge_null_partitions(PartitionMap *outer_map,
+					  PartitionMap *inner_map,
+					  bool outer_has_null,
+					  bool inner_has_null,
+					  int outer_null,
+					  int inner_null,
+					  JoinType jointype,
+					  int *next_index,
+					  int *null_index);
 
 /*
  * get_qual_from_partbound
@@ -2999,3 +3107,1510 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * This function builds and returns the partition bounds for a join relation
+ * between input relations, creating two lists of partitions, which are
+ * returned to *outer_parts and *inner_parts respectively.  The lists contain
+ * the same number of partitions, and the partitions at the same positions in
+ * the lists indicate join pairs used for partitioned join.
+ *
+ * This function returns NULL, setting *outer_parts and *inner_parts to NIL,
+ * if a partition on one side matches multiple partitions on the other side,
+ * in which case we currently don't support partitioned join.
+ */
+PartitionBoundInfo
+partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
+{
+	PartitionBoundInfo 	merged_bounds;
+	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
+						inner_binfo = inner_rel->boundinfo;
+	char				strategy = outer_binfo->strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = NULL;
+
+			break;
+
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partsupfunc,
+														partcollation,
+														outer_rel,
+														inner_rel,
+														jointype,
+														outer_parts,
+														inner_parts);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(partnatts,
+														 partsupfunc,
+														 partcollation,
+														 outer_rel,
+														 inner_rel,
+														 jointype,
+														 outer_parts,
+														 inner_parts);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	return merged_bounds;
+}
+
+/*
+ * get_range_partition
+ *		Returns the index of the range partition with the given lower bound
+ *
+ * *lb and *ub are set to the lower and upper bounds of the range partition
+ * respectively, and *lb_index is advanced to the next lower bound, if any.
+ */
+static int
+get_range_partition(PartitionBoundInfo bi, int *lb_index,
+					PartitionRangeBound *lb, PartitionRangeBound *ub)
+{
+	/* Return the index as -1 if we've exhausted all the lower bounds. */
+	if (*lb_index >= bi->ndatums)
+		return -1;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(*lb_index + 1 < bi->ndatums);
+
+	lb->index = bi->indexes[*lb_index];
+	lb->kind = bi->kind[*lb_index];
+	lb->datums = bi->datums[*lb_index];
+	lb->lower = true;
+	ub->index = bi->indexes[*lb_index + 1];
+	ub->kind = bi->kind[*lb_index + 1];
+	ub->datums = bi->datums[*lb_index + 1];
+	ub->lower = false;
+
+	/* The partition index of an upper bound should be valid. */
+	Assert(ub->index >= 0);
+
+	/*
+	 * Advance the lower bound; if there are no bounds left beyond the upper
+	 * bound, we have reached the last lower bound.
+	 */
+	if (*lb_index + 2 >= bi->ndatums)
+		*lb_index = bi->ndatums;
+	else
+	{
+		/*
+		 * If the index assigned to the bound next to the upper bound isn't
+		 * valid, that is the lower bound of the next range partition; else,
+		 * the upper bound of the current range partition is also the lower
+		 * bound of the next range partition.
+		 */
+		if (bi->indexes[*lb_index + 2] < 0)
+			*lb_index = *lb_index + 2;
+		else
+			*lb_index = *lb_index + 1;
+	}
+
+	return ub->index;
+}
+
+/*
+ * compare_range_partitions
+ *		Compares the bounds of two range partitions, and returns true if the
+ *		ranges of the partitions overlap, false otherwise
+ *
+ * *lb_cmpval is set to -1, 0 or 1 if the outer partition's lower bound is
+ * lower than, equal to or higher than the inner partition's lower bound
+ * respectively.  Likewise, *ub_cmpval is set to -1, 0 or 1 if the outer
+ * partition's upper bound is lower than, equal to or higher than the inner
+ * partition's upper bound respectively.
+ */
+static bool
+compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+						 Oid *partcollations,
+						 PartitionRangeBound *outer_lb,
+						 PartitionRangeBound *outer_ub,
+						 PartitionRangeBound *inner_lb,
+						 PartitionRangeBound *inner_ub,
+						 int *lb_cmpval, int *ub_cmpval)
+{
+	/*
+	 * Check to see if the upper bound of the outer partition is lower than
+	 * the lower bound of the inner partition; in which case the partitions
+	 * aren't overlapping.
+	 */
+	if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+							 outer_ub, inner_lb) < 0)
+	{
+		*lb_cmpval = -1;
+		*ub_cmpval = -1;
+		return false;
+	}
+
+	/*
+	 * Check to see if the lower bound of the outer partition is higher than
+	 * the upper bound of the inner partition; in which case the partitions
+	 * aren't overlapping.
+	 */
+	if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+							 outer_lb, inner_ub) > 0)
+	{
+		*lb_cmpval = 1;
+		*ub_cmpval = 1;
+		return false;
+	}
+
+	/* All other cases indicate overlapping partitions. */
+	*lb_cmpval = compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									  outer_lb, inner_lb);
+	*ub_cmpval = compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									  outer_ub, inner_ub);
+	return true;
+}
+
+/*
+ * get_merged_range_bounds
+ *		Given the bounds of range partitions to be join, determine the range
+ *		bounds of the merged partition produced from the range partitions
+ *
+ * *merged_lb and *merged_ub are set to the lower and upper bounds of the
+ * merged partition.
+ */
+static void
+get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations, JoinType jointype,
+						PartitionRangeBound *outer_lb,
+						PartitionRangeBound *outer_ub,
+						PartitionRangeBound *inner_lb,
+						PartitionRangeBound *inner_ub,
+						int	lb_cmpval, int ub_cmpval,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub)
+{
+	Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								outer_lb, inner_lb) == lb_cmpval);
+	Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								outer_ub, inner_ub) == ub_cmpval);
+
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			*merged_lb = (lb_cmpval > 0) ? *outer_lb : *inner_lb;
+			*merged_ub = (ub_cmpval < 0) ? *outer_ub : *inner_ub;
+			break;
+
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = *outer_ub;
+			*merged_lb = *outer_lb;
+			break;
+
+		case JOIN_FULL:
+			*merged_lb = (lb_cmpval < 0) ? *outer_lb : *inner_lb;
+			*merged_ub = (ub_cmpval > 0) ? *outer_ub : *inner_ub;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+}
+
+/*
+ * add_merged_range_bounds
+ *		Add the range bounds of a merged partition to the lists of range
+ *		bounds
+ */
+static void
+add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub,
+						int merged_index,
+						List **merged_datums,
+						List **merged_kinds,
+						List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		/* First merged partition */
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		Assert(*merged_kinds && *merged_indexes);
+
+		/* Get the last upper bound. */
+		prev_ub.index = llast_int(*merged_indexes);
+		prev_ub.datums = (Datum *) llast(*merged_datums);
+		prev_ub.kind = (PartitionRangeDatumKind *) llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		/*
+		 * We pass to partition_rbound_cmp() lower1 as false to prevent it
+		 * from considering the last upper bound to be smaller than the lower
+		 * bound of the merged partition when the values of the two range
+		 * bounds compare equal.
+		 */
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  merged_lb->datums, merged_lb->kind,
+									  false, &prev_ub);
+		Assert(cmpval >= 0);
+	}
+
+	/*
+	 * If the lower bound is higher than the last upper bound, add the lower
+	 * bound with the index as -1 indicating that that is a lower bound; else,
+	 * the last upper bound will be reused as the lower bound of the merged
+	 * partition, so skip this.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, merged_lb->datums);
+		*merged_kinds = lappend(*merged_kinds, merged_lb->kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	/* Add the upper bound and index of the merged partition. */
+	*merged_datums = lappend(*merged_datums, merged_ub->datums);
+	*merged_kinds = lappend(*merged_kinds, merged_ub->kind);
+	*merged_indexes = lappend_int(*merged_indexes, merged_index);
+}
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			outer_part;
+	int			inner_part;
+	int			outer_lb_index;
+	int			inner_lb_index;
+	PartitionRangeBound outer_lb;
+	PartitionRangeBound outer_ub;
+	PartitionRangeBound inner_lb;
+	PartitionRangeBound inner_ub;
+	int			next_index = 0;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_kinds = NIL;
+	List	   *merged_indexes = NIL;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = inner_lb_index = 0;
+	outer_part = get_range_partition(outer_bi, &outer_lb_index,
+									 &outer_lb, &outer_ub);
+	inner_part = get_range_partition(inner_bi, &inner_lb_index,
+									 &inner_lb, &inner_ub);
+	while (outer_part >= 0 || inner_part >= 0)
+	{
+		PartitionRangeBound merged_lb;
+		PartitionRangeBound merged_ub;
+		int			merged_index = -1;
+		bool		overlap;
+		int			ub_cmpval;
+		int			lb_cmpval;
+
+		if (outer_part >= 0)
+			Assert(outer_map.merged_indexes[outer_part] == -1 &&
+				   outer_map.merged[outer_part] == false);
+		if (inner_part >= 0)
+			Assert(inner_map.merged_indexes[inner_part] == -1 &&
+				   inner_map.merged[inner_part] == false);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (outer_part == -1)
+		{
+			overlap = false;
+			lb_cmpval = 1;
+			ub_cmpval = 1;
+		}
+		else if (inner_part == -1)
+		{
+			overlap = false;
+			lb_cmpval = -1;
+			ub_cmpval = -1;
+		}
+		else
+			overlap = compare_range_partitions(partnatts, partsupfuncs,
+											   partcollations,
+											   &outer_lb, &outer_ub,
+											   &inner_lb, &inner_ub,
+											   &lb_cmpval, &ub_cmpval);
+
+		if (overlap)
+		{
+			/* The ranges of partitions overlap; form a join pair of them. */
+
+			PartitionRangeBound save_outer_ub;
+			PartitionRangeBound save_inner_ub;
+
+			/* Get the bounds of the merged partition. */
+			get_merged_range_bounds(partnatts, partsupfuncs,
+									partcollations, jointype,
+									&outer_lb, &outer_ub,
+									&inner_lb, &inner_ub,
+									lb_cmpval, ub_cmpval,
+									&merged_lb, &merged_ub);
+
+			/*
+			 * Get the index of the merged partition.  Both partitions aren't
+			 * merged yet, so the partitions should be merged successfully.
+			 */
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
+													outer_part, inner_part,
+													&next_index);
+			Assert(merged_index >= 0);
+
+			/* Save the upper bounds of both partitions for use below. */
+			save_outer_ub = outer_ub;
+			save_inner_ub = inner_ub;
+
+			/* Move to the next pair of partitions. */
+			outer_part = get_range_partition(outer_bi, &outer_lb_index,
+											 &outer_lb, &outer_ub);
+			inner_part = get_range_partition(inner_bi, &inner_lb_index,
+											 &inner_lb, &inner_ub);
+
+			/*
+			 * If the range of a partition on one side overlaps the range of
+			 * the next partition on the other side, that will cause the
+			 * partition on one side to match at least two partitions on the
+			 * other side, which is the case that we currently don't support
+			 * partitioned join for; give up.
+			 */
+			if (ub_cmpval > 0 && inner_part >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &save_outer_ub, &inner_lb) > 0)
+				goto cleanup;
+			if (ub_cmpval < 0 && outer_part >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &outer_lb, &save_inner_ub) < 0)
+				goto cleanup;
+
+			/*
+			 * A row from a non-overlapping portion (if any) of a partition
+			 * on one side might find its join partner in the default
+			 * partition (if any) on the other side, causing the same
+			 * situation as above; if so, give up.
+			 */
+			if ((outer_has_default && (lb_cmpval > 0 || ub_cmpval < 0)) ||
+				(inner_has_default && (lb_cmpval < 0 || ub_cmpval > 0)))
+				goto cleanup;
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			merged_lb = outer_lb;
+			merged_ub = outer_ub;
+
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				merged_index = process_outer_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   outer_part,
+													   inner_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+			}
+
+			/* Move to the next partition on the outer side. */
+			outer_part = get_range_partition(outer_bi, &outer_lb_index,
+											 &outer_lb, &outer_ub);
+		}
+		else
+		{
+			/* Upper bound of outer range higher than that of the inner. */
+			Assert(ub_cmpval > 0);
+
+			merged_lb = inner_lb;
+			merged_ub = inner_ub;
+
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				merged_index = process_inner_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   inner_part,
+													   outer_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+			}
+
+			/* Move to the next partition on the inner side. */
+			inner_part = get_range_partition(inner_bi, &inner_lb_index,
+											 &inner_lb, &inner_ub);
+		}
+
+		if (merged_index >= 0 && merged_index != default_index)
+		{
+			/* Add the range bounds of the merged partition. */
+			add_merged_range_bounds(partnatts, partsupfuncs, partcollations,
+									&merged_lb, &merged_ub, merged_index,
+									&merged_datums, &merged_kinds,
+									&merged_indexes);
+		}
+	}
+
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+		merge_default_partitions(&outer_map, &inner_map,
+								 outer_has_default, inner_has_default,
+								 outer_default, inner_default,
+								 jointype, &next_index, &default_index);
+	else
+		Assert(default_index == -1);
+
+	/*
+	 * Unlike the case for list partitioning, we wouldn't have re-merged
+	 * partitions, so did_remapping should be left alone.
+	 */
+	Assert(!outer_map.did_remapping && !inner_map.did_remapping);
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 &outer_map, &inner_map,
+								 next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes,
+													  merged_kinds,
+													  -1, default_index);
+	}
+
+cleanup:
+	/* Free local memory before returning. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							JoinType jointype,
+							List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			next_index = 0;
+	int			null_index = -1;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			cnto;
+	int			cnti;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *merged_datum = NULL;
+		int			merged_index = -1;
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			cmpval;
+
+		/* Get the list datums of the next pair of partitions. */
+		odatums = cnto < outer_bi->ndatums ? outer_bi->datums[cnto] : NULL;
+		idatums = cnti < inner_bi->ndatums ? inner_bi->datums[cnti] : NULL;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+			cmpval = 1;
+		else if (cnti >= inner_bi->ndatums)
+			cmpval = -1;
+		else
+		{
+			Assert(odatums != NULL && idatums != NULL);
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0],
+													 idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			int			o_index = outer_bi->indexes[cnto];
+			int			i_index = inner_bi->indexes[cnti];
+
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			Assert(o_index >= 0 && i_index >= 0);
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
+													o_index, i_index,
+													&next_index);
+
+			if (merged_index == -1)
+				goto cleanup;
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			Assert(cnto < outer_bi->ndatums);
+
+			/* A datum missing from the inner side. */
+			merged_datum = odatums;
+
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				int			o_index = outer_bi->indexes[cnto];
+
+				Assert(o_index >= 0);
+				merged_index = process_outer_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   o_index,
+													   inner_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+			}
+
+			/* Move to the next datum on the outer side. */
+			cnto++;
+		}
+		else
+		{
+			Assert(cmpval > 0);
+			Assert(cnti < inner_bi->ndatums);
+
+			/* A datum missing from the outer side. */
+			merged_datum = idatums;
+
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				int			i_index = inner_bi->indexes[cnti];
+
+				Assert(i_index >= 0);
+				merged_index = process_inner_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   i_index,
+													   outer_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+			}
+
+			/* Move to the next datum on the inner side. */
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0 && merged_index != default_index)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	/* Merge null partitions if any. */
+	if (outer_has_null || inner_has_null)
+		merge_null_partitions(&outer_map, &inner_map,
+							  outer_has_null, inner_has_null,
+							  outer_bi->null_index, inner_bi->null_index,
+							  jointype, &next_index, &null_index);
+	else
+		Assert(null_index == -1);
+
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+		merge_default_partitions(&outer_map, &inner_map,
+								 outer_has_default, inner_has_default,
+								 outer_default, inner_default,
+								 jointype, &next_index, &default_index);
+	else
+		Assert(default_index == -1);
+
+	/* Fix the merged_indexes list if necessary. */
+	if (outer_map.did_remapping || inner_map.did_remapping)
+	{
+		Assert(jointype == JOIN_FULL);
+		fix_merged_indexes(&outer_map, &inner_map,
+						   next_index, merged_indexes);
+	}
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 &outer_map, &inner_map,
+								 next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) <= next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes, NIL,
+													  null_index, default_index);
+	}
+
+cleanup:
+	/* Free local memory before returning. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+	list_free(merged_datums);
+	list_free(merged_indexes);
+
+	return merged_bounds;
+}
+
+/*
+ * init_partition_map
+ *
+ * Initialize a PartitionMap struct for given relation.
+ */
+static void
+init_partition_map(RelOptInfo *rel, PartitionMap *map)
+{
+	int			nparts = rel->nparts;
+	int			i;
+
+	map->nparts = nparts;
+	map->merged_indexes = (int *) palloc(sizeof(int) * nparts);
+	map->merged = (bool *) palloc(sizeof(bool) * nparts);
+	map->did_remapping = false;
+	map->old_indexes = (int *) palloc(sizeof(int) * nparts);
+	for (i = 0; i < nparts; i++)
+	{
+		map->merged_indexes[i] = map->old_indexes[i] = -1;
+		map->merged[i] = false;
+	}
+}
+
+/*
+ * free_partition_map
+ */
+static void
+free_partition_map(PartitionMap *map)
+{
+	pfree(map->merged_indexes);
+	pfree(map->merged);
+	pfree(map->old_indexes);
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given partitions.
+ */
+static int
+map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_index, int inner_index, int *next_index)
+{
+	int 		outer_merged_index;
+	bool 		outer_merged;
+	int 		inner_merged_index;
+	bool 		inner_merged;
+
+	Assert(outer_index >= 0 && outer_index < outer_map->nparts);
+	outer_merged_index = outer_map->merged_indexes[outer_index];
+	outer_merged = outer_map->merged[outer_index];
+	Assert(inner_index >= 0 && inner_index < inner_map->nparts);
+	inner_merged_index = inner_map->merged_indexes[inner_index];
+	inner_merged = inner_map->merged[inner_index];
+
+	/*
+	 * Handle cases where both partitions are mapped to merged partitions.
+	 */
+	if (outer_merged_index >= 0 && inner_merged_index >= 0)
+	{
+		/*
+		 * If the mereged partitions are the same, no need to do anything;
+		 * return the index of the merged partition.  Otherwise, if both
+		 * partitions are merged with dummy partitions, re-merge them; map
+		 * them to the merged partition with the smaller of the two merged
+		 * indexes and return the smaller index.  Otherwise they can't be
+		 * merged, so return -1.
+		 */
+		if (outer_merged_index == inner_merged_index)
+		{
+			Assert(outer_merged);
+			Assert(inner_merged);
+			return outer_merged_index;
+		}
+		if (!outer_merged && !inner_merged)
+		{
+			/*
+			 * Note that we will fix the larger index that have been added to
+			 * the merged_indexes list so far in fix_merged_indexes().
+			 */
+			if (outer_merged_index < inner_merged_index)
+			{
+				outer_map->merged[outer_index] = true;
+				inner_map->merged_indexes[inner_index] = outer_merged_index;
+				inner_map->merged[inner_index] = true;
+				inner_map->did_remapping = true;
+				inner_map->old_indexes[inner_index] = inner_merged_index;
+				return outer_merged_index;
+			}
+			else
+			{
+				inner_map->merged[inner_index] = true;
+				outer_map->merged_indexes[outer_index] = inner_merged_index;
+				outer_map->merged[outer_index] = true;
+				outer_map->did_remapping = true;
+				outer_map->old_indexes[outer_index] = outer_merged_index;
+				return inner_merged_index;
+			}
+		}
+		return -1;
+	}
+
+	/* At least one partition isn't mapped to a merged partition. */
+	Assert(outer_merged_index == -1 || inner_merged_index == -1);
+
+	/*
+	 * If neither of partitions isn't mapped, assign them a new merged
+	 * partition and return the index of the merged partition.  Otherwise, if
+	 * one of partitions is merged with a dummy relation (and the other isn't
+	 * merged), re-merge it with the other, with the same index, and return
+	 * the index.  Otherwise they can't be merged, so return -1.
+	 */
+	if (outer_merged_index == -1 && inner_merged_index == -1)
+	{
+		int 		merged_index = *next_index;
+
+		Assert(!outer_merged);
+		Assert(!inner_merged);
+		outer_map->merged_indexes[outer_index] = merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged_indexes[inner_index] = merged_index;
+		inner_map->merged[inner_index] = true;
+		*next_index = *next_index + 1;
+		return merged_index;
+	}
+	if (outer_merged_index >= 0 && !outer_map->merged[outer_index])
+	{
+		Assert(inner_merged_index == -1);
+		Assert(!inner_merged);
+		inner_map->merged_indexes[inner_index] = outer_merged_index;
+		inner_map->merged[inner_index] = true;
+		outer_map->merged[outer_index] = true;
+		return outer_merged_index;
+	}
+	if (inner_merged_index >= 0 && !inner_map->merged[inner_index])
+	{
+		Assert(outer_merged_index == -1);
+		Assert(!outer_merged);
+		outer_map->merged_indexes[outer_index] = inner_merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged[inner_index] = true;
+		return inner_merged_index;
+	}
+	return -1;
+}
+
+/*
+ * merge_partition_with_dummy
+ *
+ * *next_index is incremented.
+ */
+static int
+merge_partition_with_dummy(PartitionMap *map, int index, int *next_index)
+{
+	int 		merged_index = *next_index;
+
+	Assert(index >= 0 && index < map->nparts);
+	Assert(map->merged_indexes[index] == -1);
+	Assert(!map->merged[index]);
+	map->merged_indexes[index] = merged_index;
+	/* Leave the merged flag alone! */
+	*next_index = *next_index + 1;
+	return merged_index;
+}
+
+/*
+ * process_outer_partition
+ *
+ * Determine the merged partition associated with the given outer partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given outer partition.
+ */
+static int
+process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index)
+{
+	int 		merged_index = -1;
+
+	Assert(outer_index >= 0);
+
+	/*
+	 * If the inner side has the default partition, the outer partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an outer join, in which case the outer partition has to be
+	 * scanned all the way anyway; if the outer partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the outer partition with a dummy partition.
+	 */
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0);
+
+		/*
+		 * If the outer side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (outer_has_default)
+			return -1;
+
+		merged_index = map_and_merge_partitions(outer_map, inner_map,
+												outer_index, inner_default,
+												next_index);
+		if (merged_index == -1)
+			return -1;
+
+		/*
+		 * If this is a FULL join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (*default_index == -1)
+				*default_index = merged_index;
+			else
+				Assert(*default_index == merged_index);
+		}
+	}
+	else
+	{
+		Assert(IS_OUTER_JOIN(jointype));
+		Assert(jointype != JOIN_RIGHT);
+
+		merged_index = outer_map->merged_indexes[outer_index];
+		if (merged_index == -1)
+			merged_index = merge_partition_with_dummy(outer_map, outer_index,
+													  next_index);
+	}
+	return merged_index;
+}
+
+/*
+ * process_inner_partition
+ *
+ * Determine the merged partition associated with the given inner partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given inner partition.
+ */
+static int
+process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index)
+{
+	int 		merged_index = -1;
+
+	Assert(inner_index >= 0);
+
+	/*
+	 * If the outer side has the default partition, the inner partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an FULL join, in which case the inner partition has to be
+	 * scanned all the way anyway; if the inner partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the inner partition with a dummy partition.
+	 */
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0);
+
+		/*
+		 * If the inner side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (inner_has_default)
+			return -1;
+
+		merged_index = map_and_merge_partitions(outer_map, inner_map,
+												outer_default, inner_index,
+												next_index);
+		if (merged_index == -1)
+			return -1;
+
+		/*
+		 * If this is an outer join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			if (*default_index == -1)
+				*default_index = merged_index;
+			else
+				Assert(*default_index == merged_index);
+		}
+	}
+	else
+	{
+		Assert(jointype == JOIN_FULL);
+
+		merged_index = inner_map->merged_indexes[inner_index];
+		if (merged_index == -1)
+			merged_index = merge_partition_with_dummy(inner_map, inner_index,
+													  next_index);
+	}
+	return merged_index;
+}
+
+/*
+ * fix_merged_indexes
+ */
+static void
+fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes)
+{
+	int		   *new_indexes;
+	int			merged_index;
+	int			i;
+	ListCell   *lc;
+
+	new_indexes = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		new_indexes[i] = -1;
+
+	/* Build the mapping of old merged indexes to new merged indexes. */
+	if (outer_map->did_remapping)
+	{
+		for (i = 0; i < outer_map->nparts; i++)
+		{
+			merged_index = outer_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = outer_map->merged_indexes[i];
+		}
+	}
+	if (inner_map->did_remapping)
+	{
+		for (i = 0; i < inner_map->nparts; i++)
+		{
+			merged_index = inner_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = inner_map->merged_indexes[i];
+		}
+	}
+
+	/* Fix the merged_indexes list using the mapping. */
+	foreach(lc, merged_indexes)
+	{
+		merged_index = lfirst_int(lc);
+		Assert(merged_index >= 0);
+		if (new_indexes[merged_index] >= 0)
+			lfirst_int(lc) = new_indexes[merged_index];
+	}
+
+	pfree(new_indexes);
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * This function produces the list pairs of partitions which produce merged
+ * partitions in the order of merged partition indexes.
+ *
+ * nmerged is the number of merged partitions.
+ *
+ * If successful, the list pairs are returned as two separate lists,
+ * outer_part_list and inner_part_list, one for each side.  Otherwise, those
+ * lists will be set to NIL.
+ */
+static void
+generate_matching_part_pairs(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 PartitionMap *outer_map, PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_part_list, List **inner_part_list)
+{
+	int			outer_nparts = outer_map->nparts;
+	int			inner_nparts = inner_map->nparts;
+	int		   *outer_part_array;
+	int		   *inner_part_array;
+	int			max_nparts;
+	int 		i;
+
+	Assert(outer_nparts == outer_rel->nparts);
+	Assert(inner_nparts == inner_rel->nparts);
+
+	outer_part_array = (int *) palloc(sizeof(int) * nmerged);
+	inner_part_array = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		outer_part_array[i] = inner_part_array[i] = -1;
+
+	/* Set pairs of matching partitions. */
+	max_nparts = Max(outer_nparts, inner_nparts);
+	for (i = 0; i < max_nparts; i++)
+	{
+		if (i < outer_nparts)
+		{
+			int 		merged_index = outer_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				outer_part_array[merged_index] = i;
+			}
+		}
+		if (i < inner_nparts)
+		{
+			int 		merged_index = inner_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				inner_part_array[merged_index] = i;
+			}
+		}
+	}
+
+	*outer_part_list = NIL;
+	*inner_part_list = NIL;
+
+	/* Build the list pairs. */
+	for (i = 0; i < nmerged; i++)
+	{
+		int			outer_part = outer_part_array[i];
+		int			inner_part = inner_part_array[i];
+
+		/*
+		 * If both partitions are dummy, it means the merged partition was
+		 * removed by partition re-merging in map_and_merge_partitions();
+		 * ignore the merged partition
+		 */
+		if (outer_part == -1 && inner_part == -1)
+			continue;
+
+		*outer_part_list = lappend(*outer_part_list, outer_part >= 0 ?
+								   outer_rel->part_rels[outer_part] : NULL);
+		*inner_part_list = lappend(*inner_part_list, inner_part >= 0 ?
+								   inner_rel->part_rels[inner_part] : NULL);
+	}
+
+	pfree(outer_part_array);
+	pfree(inner_part_array);
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * merge_default_partitions
+ *		Merge the default partitions from both sides, if any
+ *
+ * *default_index is set to the index of the default partition of the joinrel,
+ * if appropriate.
+ */
+static void
+merge_default_partitions(PartitionMap *outer_map,
+						 PartitionMap *inner_map,
+						 bool outer_has_default,
+						 bool inner_has_default,
+						 int outer_default,
+						 int inner_default,
+						 JoinType jointype,
+						 int *next_index,
+						 int *default_index)
+{
+	int 		outer_merged_index = -1;
+	int 		inner_merged_index = -1;
+
+	Assert(outer_has_default || inner_has_default);
+
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0 && outer_default < outer_map->nparts);
+		outer_merged_index = outer_map->merged_indexes[outer_default];
+	}
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0 && inner_default < inner_map->nparts);
+		inner_merged_index = inner_map->merged_indexes[inner_default];
+	}
+
+	if (outer_has_default && !inner_has_default)
+	{
+		/*
+		 * If this is an outer join, assign a new merged partition if not
+		 * already done.  The resulting partition should be the default
+		 * partition of the joinrel.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			if (outer_merged_index == -1)
+				*default_index = merge_partition_with_dummy(outer_map,
+															outer_default,
+															next_index);
+			else
+				Assert(*default_index == outer_merged_index);
+		}
+		else
+			Assert(*default_index == -1);
+	}
+	else if (!outer_has_default && inner_has_default)
+	{
+		/*
+		 * If this is a FULL join, assign a new merged partition if not
+		 * already done.  The resulting partition should be the default
+		 * partition of the joinrel.
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (inner_merged_index == -1)
+				*default_index = merge_partition_with_dummy(inner_map,
+															inner_default,
+															next_index);
+			else
+				Assert(*default_index == inner_merged_index);
+		}
+		else
+			Assert(*default_index == -1);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+		Assert(outer_merged_index == -1);
+		Assert(inner_merged_index == -1);
+		Assert(*default_index == -1);
+
+		/*
+		 * Merge the default partitions.  Both partitions aren't merged yet,
+		 * so the partitions should be merged successfully.  The resulting
+		 * partition should be the default partition of the joinrel.
+		 */
+		*default_index = map_and_merge_partitions(outer_map,
+												  inner_map,
+												  outer_default,
+												  inner_default,
+												  next_index);
+		Assert(*default_index >= 0);
+	}
+}
+
+/*
+ * merge_null_partitions
+ *		Merge the NULL partitions from both sides, if any
+ *
+ * *null_index is set to the index of the NULL partition of the joinrel, if
+ * appropriate.
+ */
+static void
+merge_null_partitions(PartitionMap *outer_map,
+					  PartitionMap *inner_map,
+					  bool outer_has_null,
+					  bool inner_has_null,
+					  int outer_null,
+					  int inner_null,
+					  JoinType jointype,
+					  int *next_index,
+					  int *null_index)
+{
+	bool 		merge_outer_null = false;
+	bool 		merge_inner_null = false;
+
+	Assert(outer_has_null || inner_has_null);
+	Assert(*null_index == -1);
+
+	/*
+	 * If this is an INNER or SEMI join, two NULL values won't be considered
+	 * to be equal, in which case we don't need to do anything.
+	 */
+	if (jointype == JOIN_INNER || jointype == JOIN_SEMI)
+		return;
+
+	Assert(IS_OUTER_JOIN(jointype));
+	Assert(jointype != JOIN_RIGHT);
+
+	if (outer_has_null)
+	{
+		/*
+		 * Since this is an outer join, if the NULL partition of the outer
+		 * side isn't merged yet, it means that the partition contains only
+		 * NULL values as the key values, in which case we need to create a
+		 * merged partition below.
+		 */
+		Assert(outer_null >= 0 && outer_null < outer_map->nparts);
+		if (outer_map->merged_indexes[outer_null] == -1)
+			merge_outer_null = true;
+	}
+	if (inner_has_null)
+	{
+		/*
+		 * If this is a FULL join, and the NULL partition of the inner side
+		 * isn't merged yet, it means that the partition contains only NULL
+		 * values as the key values, in which case we need to create a merged
+		 * partition below.
+		 */
+		Assert(inner_null >= 0 && inner_null < inner_map->nparts);
+		if (jointype == JOIN_FULL &&
+			inner_map->merged_indexes[inner_null] == -1)
+			merge_inner_null = true;
+	}
+
+	if (!merge_outer_null && !merge_inner_null)
+		return;
+
+	if (merge_outer_null && !merge_inner_null)
+	{
+		Assert(outer_has_null);
+
+		/*
+		 * Assign a new merged partition.  The resulting partition should be
+		 * the NULL partition of the joinrel.
+		 */
+		*null_index = merge_partition_with_dummy(outer_map, outer_null,
+												 next_index);
+	}
+	else if (!merge_outer_null && merge_inner_null)
+	{
+		Assert(inner_has_null);
+		Assert(jointype == JOIN_FULL);
+
+		/*
+		 * Assign a new merged partition.  The resulting partition should be
+		 * the NULL partition of the joinrel.
+		 */
+		*null_index = merge_partition_with_dummy(inner_map, inner_null,
+												 next_index);
+	}
+	else
+	{
+		Assert(merge_outer_null && merge_inner_null);
+		Assert(outer_has_null);
+		Assert(inner_has_null);
+		Assert(jointype == JOIN_FULL);
+
+		/*
+		 * Merge the NULL partitions.  Both partitions aren't merged yet, so
+		 * the partitions should be merged successfully.  The resulting
+		 * partition should be the NULL partition of the joinrel.
+		 */
+		*null_index = map_and_merge_partitions(outer_map, inner_map,
+											   outer_null, inner_null,
+											   next_index);
+		Assert(*null_index >= 0);
+	}
+}
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index 3d3be197e0..1545877d8c 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -579,8 +579,10 @@ typedef struct PartitionSchemeData *PartitionScheme;
  *		part_scheme - Partitioning scheme of the relation
  *		nparts - Number of partitions
  *		boundinfo - Partition bounds
+ *		merged - true if partition bounds are merged ones
  *		partition_qual - Partition constraint if not the root
  *		part_rels - RelOptInfos for each partition
+ *		all_partrels - Relids set of all partition relids
  *		partexprs, nullable_partexprs - Partition key expressions
  *		partitioned_child_rels - RT indexes of unpruned partitions of
  *								 this relation that are partitioned tables
@@ -720,9 +722,12 @@ typedef struct RelOptInfo
 	PartitionScheme part_scheme;	/* Partitioning scheme. */
 	int			nparts;			/* number of partitions */
 	struct PartitionBoundInfoData *boundinfo;	/* Partition bounds */
+	bool		merged;			/* true if partition bounds were created by
+								 * partition_bounds_merge() */
 	List	   *partition_qual; /* partition constraint */
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of partitions,
 									 * stored in the same order of bounds */
+	Relids		all_partrels;	/* Relids set of all partition relids */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 	List	   *partitioned_child_rels; /* List of RT indexes. */
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index be940b5192..5442e843af 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -16,6 +16,7 @@
 #include "nodes/pg_list.h"
 #include "partitioning/partdefs.h"
 #include "utils/relcache.h"
+struct RelOptInfo;				/* avoid including pathnodes.h here */
 
 
 /*
@@ -108,5 +109,11 @@ extern int	partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 										  int nvalues, Datum *values, bool *is_equal);
 extern int	partition_hash_bsearch(PartitionBoundInfo boundinfo,
 								   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   struct RelOptInfo *outer_rel, struct RelOptInfo *inner_rel,
+					   JoinType jointype,
+					   List **outer_parts, List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index b3fbe47bde..dc78bc2503 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -2055,30 +2055,2046 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
                ->  Seq Scan on prt1_n_p2 t1_2
 (10 rows)
 
--- partitionwise join can not be applied if only one of joining tables has
--- default partition
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
-ANALYZE prt2;
+--
+-- Test advanced partition-matching algorithm for partitioned join
+--
+-- Tests for range-partitioned tables
+CREATE TABLE prt1_ad (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_ad_p1 PARTITION OF prt1_ad FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_ad_p2 PARTITION OF prt1_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_ad_p3 PARTITION OF prt1_ad FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_ad_a_idx on prt1_ad (a);
+INSERT INTO prt1_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_ad;
+CREATE TABLE prt2_ad (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_ad_p1 PARTITION OF prt2_ad FOR VALUES FROM (100) TO (150);
+CREATE TABLE prt2_ad_p2 PARTITION OF prt2_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt2_ad_p3 PARTITION OF prt2_ad FOR VALUES FROM (350) TO (500);
+CREATE INDEX prt2_ad_b_idx on prt2_ad (b);
+INSERT INTO prt2_ad_p1 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 149) i;
+INSERT INTO prt2_ad_p2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(200, 299) i;
+INSERT INTO prt2_ad_p3 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_ad;
+-- inner join
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                     QUERY PLAN                      
+-----------------------------------------------------
  Sort
    Sort Key: t1.a
-   ->  Hash Join
-         Hash Cond: (t2.b = t1.a)
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_ad_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_ad_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p2 t2_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_ad_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 100 | 0 | 0100
+ 125 | 0 | 0125
+ 200 | 0 | 0200
+ 225 | 0 | 0225
+ 250 | 0 | 0250
+ 275 | 0 | 0275
+ 350 | 0 | 0350
+ 375 | 0 | 0375
+(8 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 |     | 
+ 175 | 0175 |     | 
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 300 | 0300 |     | 
+ 325 | 0325 |     | 
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(12 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+                  QUERY PLAN                   
+-----------------------------------------------
+ Append
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.a = t2_1.b)
+         ->  Seq Scan on prt1_ad_p1 t1_1
+               Filter: (b = 0)
+         ->  Hash
+               ->  Seq Scan on prt2_ad_p1 t2_1
+   ->  Hash Anti Join
+         Hash Cond: (t1_2.a = t2_2.b)
+         ->  Seq Scan on prt1_ad_p2 t1_2
+               Filter: (b = 0)
+         ->  Hash
+               ->  Seq Scan on prt2_ad_p2 t2_2
+   ->  Hash Anti Join
+         Hash Cond: (t1_3.a = t2_3.b)
+         ->  Seq Scan on prt1_ad_p3 t1_3
+               Filter: (b = 0)
+         ->  Hash
+               ->  Seq Scan on prt2_ad_p3 t2_3
+(19 rows)
+
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+  a  | b |  c   
+-----+---+------
+ 150 | 0 | 0150
+ 175 | 0 | 0175
+ 300 | 0 | 0300
+ 325 | 0 | 0325
+(4 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Sort
+   Sort Key: prt1_ad.a, prt2_ad.b
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_ad_1.a = prt2_ad_1.b)
+               Filter: (((175) = prt1_ad_1.a) OR ((425) = prt2_ad_1.b))
+               ->  Seq Scan on prt1_ad_p1 prt1_ad_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p1 prt2_ad_1
+                           Filter: (a = 0)
+         ->  Hash Full Join
+               Hash Cond: (prt1_ad_2.a = prt2_ad_2.b)
+               Filter: (((175) = prt1_ad_2.a) OR ((425) = prt2_ad_2.b))
+               ->  Seq Scan on prt1_ad_p2 prt1_ad_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p2 prt2_ad_2
+                           Filter: (a = 0)
+         ->  Hash Full Join
+               Hash Cond: (prt2_ad_3.b = prt1_ad_3.a)
+               Filter: (((175) = prt1_ad_3.a) OR ((425) = prt2_ad_3.b))
+               ->  Seq Scan on prt2_ad_p3 prt2_ad_3
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p3 prt1_ad_3
+                           Filter: (b = 0)
+(27 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 175 | 0175 |     | 
+     |      | 425 | 0425
+(2 rows)
+
+-- Test cases where one side has an extra partition
+-- Add an extra partition to prt2_ad
+CREATE TABLE prt2_ad_extra PARTITION OF prt2_ad FOR VALUES FROM (500) TO (MAXVALUE);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(500, 599) i;
+ANALYZE prt2_ad;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_ad_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_ad_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p2 t2_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_ad_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 100 | 0 | 0100
+ 125 | 0 | 0125
+ 200 | 0 | 0200
+ 225 | 0 | 0225
+ 250 | 0 | 0250
+ 275 | 0 | 0275
+ 350 | 0 | 0350
+ 375 | 0 | 0375
+(8 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 |     | 
+ 175 | 0175 |     | 
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 300 | 0300 |     | 
+ 325 | 0325 |     | 
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(12 rows)
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c FROM prt2_ad t1 LEFT JOIN prt1_ad t2 ON (t1.b = t2.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t2.a
+   ->  Hash Right Join
+         Hash Cond: (t2.a = t1.b)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2_1
-               ->  Seq Scan on prt2_p2 t2_2
-               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt1_ad_p1 t2_1
+               ->  Seq Scan on prt1_ad_p2 t2_2
+               ->  Seq Scan on prt1_ad_p3 t2_3
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt2_ad_p1 t1_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_ad_p2 t1_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_ad_p3 t1_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_ad_extra t1_4
+                           Filter: (a = 0)
+(18 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+                  QUERY PLAN                   
+-----------------------------------------------
+ Append
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.a = t2_1.b)
+         ->  Seq Scan on prt1_ad_p1 t1_1
+               Filter: (b = 0)
+         ->  Hash
+               ->  Seq Scan on prt2_ad_p1 t2_1
+   ->  Hash Anti Join
+         Hash Cond: (t1_2.a = t2_2.b)
+         ->  Seq Scan on prt1_ad_p2 t1_2
+               Filter: (b = 0)
+         ->  Hash
+               ->  Seq Scan on prt2_ad_p2 t2_2
+   ->  Hash Anti Join
+         Hash Cond: (t1_3.a = t2_3.b)
+         ->  Seq Scan on prt1_ad_p3 t1_3
+               Filter: (b = 0)
+         ->  Hash
+               ->  Seq Scan on prt2_ad_p3 t2_3
+(19 rows)
+
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+  a  | b |  c   
+-----+---+------
+ 150 | 0 | 0150
+ 175 | 0 | 0175
+ 300 | 0 | 0300
+ 325 | 0 | 0325
+(4 rows)
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt1_ad t2 WHERE t1.b = t2.a) AND t1.a = 0;
+                  QUERY PLAN                   
+-----------------------------------------------
+ Hash Anti Join
+   Hash Cond: (t1.b = t2.a)
+   ->  Append
+         ->  Seq Scan on prt2_ad_p1 t1_1
+               Filter: (a = 0)
+         ->  Seq Scan on prt2_ad_p2 t1_2
+               Filter: (a = 0)
+         ->  Seq Scan on prt2_ad_p3 t1_3
+               Filter: (a = 0)
+         ->  Seq Scan on prt2_ad_extra t1_4
+               Filter: (a = 0)
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_ad_p1 t2_1
+               ->  Seq Scan on prt1_ad_p2 t2_2
+               ->  Seq Scan on prt1_ad_p3 t2_3
+(16 rows)
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: prt1_ad.a, prt2_ad.b
+   ->  Hash Full Join
+         Hash Cond: (prt2_ad.b = prt1_ad.a)
+         Filter: (((175) = prt1_ad.a) OR ((425) = prt2_ad.b))
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 prt2_ad_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_ad_p2 prt2_ad_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_ad_p3 prt2_ad_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_ad_extra prt2_ad_4
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_ad_p1 prt1_ad_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_ad_p2 prt1_ad_2
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_ad_p3 prt1_ad_3
                            Filter: (b = 0)
-(16 rows)
+(22 rows)
+
+-- 3-way join where not every pair of relations can do partitioned join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1_ad t1 RIGHT JOIN prt2_ad t2 ON (t1.a = t2.b) INNER JOIN prt1_ad t3 ON (t2.b = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.a, t3.c;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a, t3.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: (t1_1.a = t3_1.a)
+               ->  Hash Join
+                     Hash Cond: (t2_1.b = t1_1.a)
+                     ->  Seq Scan on prt2_ad_p1 t2_1
+                     ->  Hash
+                           ->  Seq Scan on prt1_ad_p1 t1_1
+                                 Filter: (b = 0)
+               ->  Index Scan using prt1_ad_p1_a_idx on prt1_ad_p1 t3_1
+                     Index Cond: (a = t2_1.b)
+         ->  Nested Loop
+               Join Filter: (t1_2.a = t3_2.a)
+               ->  Hash Join
+                     Hash Cond: (t2_2.b = t1_2.a)
+                     ->  Seq Scan on prt2_ad_p2 t2_2
+                     ->  Hash
+                           ->  Seq Scan on prt1_ad_p2 t1_2
+                                 Filter: (b = 0)
+               ->  Index Scan using prt1_ad_p2_a_idx on prt1_ad_p2 t3_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t2_3.b)
+               ->  Hash Join
+                     Hash Cond: (t3_3.a = t1_3.a)
+                     ->  Seq Scan on prt1_ad_p3 t3_3
+                     ->  Hash
+                           ->  Seq Scan on prt1_ad_p3 t1_3
+                                 Filter: (b = 0)
+               ->  Index Scan using prt2_ad_p3_b_idx on prt2_ad_p3 t2_3
+                     Index Cond: (b = t3_3.a)
+(33 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1_ad t1 RIGHT JOIN prt2_ad t2 ON (t1.a = t2.b) INNER JOIN prt1_ad t3 ON (t2.b = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.a, t3.c;
+  a  | a |  c   
+-----+---+------
+ 100 | 0 | 0100
+ 125 | 0 | 0125
+ 200 | 0 | 0200
+ 225 | 0 | 0225
+ 250 | 0 | 0250
+ 275 | 0 | 0275
+ 350 | 0 | 0350
+ 375 | 0 | 0375
+(8 rows)
+
+DROP TABLE prt2_ad_extra;
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in those cases
+-- Split prt2_ad_p3 into two partitions so that prt1_ad_p3 matches both
+-- partitions
+ALTER TABLE prt2_ad DETACH PARTITION prt2_ad_p3;
+CREATE TABLE prt2_ad_p3_350_375 PARTITION OF prt2_ad FOR VALUES FROM (350) TO (375);
+CREATE TABLE prt2_ad_p3_375_500 PARTITION OF prt2_ad FOR VALUES FROM (375) TO (500);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_ad;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Seq Scan on prt2_ad_p3_350_375 t2_3
+               ->  Seq Scan on prt2_ad_p3_375_500 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p3 t1_3
+                           Filter: (b = 0)
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                          
+-------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_ad_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_ad_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_ad_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_ad_p1 t2_1
+                     ->  Seq Scan on prt2_ad_p2 t2_2
+                     ->  Seq Scan on prt2_ad_p3_350_375 t2_3
+                     ->  Seq Scan on prt2_ad_p3_375_500 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Seq Scan on prt2_ad_p3_350_375 t2_3
+               ->  Seq Scan on prt2_ad_p3_375_500 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p3 t1_3
+                           Filter: (b = 0)
+(17 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Hash Anti Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_ad_p1 t1_1
+               Filter: (b = 0)
+         ->  Seq Scan on prt1_ad_p2 t1_2
+               Filter: (b = 0)
+         ->  Seq Scan on prt1_ad_p3 t1_3
+               Filter: (b = 0)
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Seq Scan on prt2_ad_p3_350_375 t2_3
+               ->  Seq Scan on prt2_ad_p3_375_500 t2_4
+(15 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: prt1_ad.a, prt2_ad.b
+   ->  Hash Full Join
+         Hash Cond: (prt2_ad.b = prt1_ad.a)
+         Filter: (((175) = prt1_ad.a) OR ((425) = prt2_ad.b))
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 prt2_ad_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_ad_p2 prt2_ad_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_ad_p3_350_375 prt2_ad_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_ad_p3_375_500 prt2_ad_4
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_ad_p1 prt1_ad_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p2 prt1_ad_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p3 prt1_ad_3
+                           Filter: (b = 0)
+(22 rows)
+
+DROP TABLE prt2_ad_p3_350_375;
+DROP TABLE prt2_ad_p3_375_500;
+-- Test default partitions
+-- Change prt1_ad_p1 to the default partition
+ALTER TABLE prt1_ad DETACH PARTITION prt1_ad_p1;
+ALTER TABLE prt1_ad ATTACH PARTITION prt1_ad_p1 DEFAULT;
+ANALYZE prt1_ad;
+CREATE TABLE prt2_ad_p3_300_400 PARTITION OF prt2_ad FOR VALUES FROM (300) TO (400);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(300, 399) i;
+ANALYZE prt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_3.a)
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p1 t1_3
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p2 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_3.b = t1_2.a)
+               ->  Seq Scan on prt2_ad_p3_300_400 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p3 t1_2
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 300 | 0300 | 300 | 0300
+ 325 | 0325 | 325 | 0325
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(10 rows)
+
+DROP TABLE prt2_ad_p3_300_400;
+-- Restore prt2_ad_p3
+ALTER TABLE prt2_ad ATTACH PARTITION prt2_ad_p3 FOR VALUES FROM (350) TO (500);
+ANALYZE prt2_ad;
+-- Partitioned join can't be applied because the default partition of prt1_ad
+-- prt1_ad_p1 matches prt2_ad_p1 and prt2_ad_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Seq Scan on prt2_ad_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_ad_p2 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p3 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p1 t1_3
+                           Filter: (b = 0)
+(16 rows)
+
+-- Change prt2_ad_p3 to the default partition
+ALTER TABLE prt2_ad DETACH PARTITION prt2_ad_p3;
+ALTER TABLE prt2_ad ATTACH PARTITION prt2_ad_p3 DEFAULT;
+ANALYZE prt2_ad;
+-- Partitioned join can't be applied because the default partition of prt1_ad
+-- prt1_ad_p1 matches prt2_ad_p1 and prt2_ad_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Seq Scan on prt2_ad_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_ad_p2 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p3 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p1 t1_3
+                           Filter: (b = 0)
+(16 rows)
+
+DROP TABLE prt1_ad_p3;
+ANALYZE prt1_ad;
+DROP TABLE prt2_ad_p3;
+ANALYZE prt2_ad;
+CREATE TABLE prt3_ad (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt3_ad_p1 PARTITION OF prt3_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt3_ad_p2 PARTITION OF prt3_ad FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt3_ad_a_idx on prt3_ad (a);
+INSERT INTO prt3_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(200, 399) i;
+ANALYZE prt3_ad;
+-- 3-way join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) LEFT JOIN prt3_ad t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+                           QUERY PLAN                            
+-----------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b, t3.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t3_1.a = t1_1.a)
+               ->  Seq Scan on prt3_ad_p1 t3_1
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_2.b = t1_1.a)
+                           ->  Seq Scan on prt2_ad_p2 t2_2
+                           ->  Hash
+                                 ->  Seq Scan on prt1_ad_p2 t1_1
+                                       Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t3_2.a = t1_2.a)
+               ->  Seq Scan on prt3_ad_p2 t3_2
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_1.b = t1_2.a)
+                           ->  Seq Scan on prt2_ad_p1 t2_1
+                           ->  Hash
+                                 ->  Seq Scan on prt1_ad_p1 t1_2
+                                       Filter: (b = 0)
+(23 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) LEFT JOIN prt3_ad t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+  a  |  c   |  b  |  c   |  a  |  c   
+-----+------+-----+------+-----+------
+ 100 | 0100 | 100 | 0100 |     | 
+ 125 | 0125 | 125 | 0125 |     | 
+ 150 | 0150 |     |      |     | 
+ 175 | 0175 |     |      |     | 
+ 200 | 0200 | 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275 | 275 | 0275
+(8 rows)
+
+-- Tests for list-partitioned tables
+CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0001', '0003');
+CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt1_ad_p3 PARTITION OF plt1_ad FOR VALUES IN ('0008', '0009');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 NOT IN (0, 2, 5, 7);
+ANALYZE plt1_ad;
+CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt2_ad_p3 PARTITION OF plt2_ad FOR VALUES IN ('0007', '0009');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 NOT IN (0, 1, 5, 8);
+ANALYZE plt2_ad;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_ad_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+(6 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_ad_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 1 | 1 | 0001
+ 8 | 8 | 0008
+(2 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p3 t2_3
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+   |      | 2 | 0002
+   |      | 7 | 0007
+(8 rows)
+
+-- Test cases where one side has an extra partition
+-- Add an extra partition to plt2_ad
+CREATE TABLE plt2_ad_extra PARTITION OF plt2_ad FOR VALUES IN ('0000');
+INSERT INTO plt2_ad_extra VALUES (0, 0, '0000');
+ANALYZE plt2_ad;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_ad_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+(6 rows)
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2_ad t1 LEFT JOIN plt1_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt1_ad_p1 t2_1
+               ->  Seq Scan on plt1_ad_p2 t2_2
+               ->  Seq Scan on plt1_ad_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_ad_extra t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_ad_p1 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_ad_p2 t1_3
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_ad_p3 t1_4
+                           Filter: (b < 10)
+(18 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_ad_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 1 | 1 | 0001
+ 8 | 8 | 0008
+(2 rows)
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt2_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt1_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Anti Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt2_ad_extra t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p1 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3 t1_4
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p1 t2_1
+                     ->  Seq Scan on plt1_ad_p2 t2_2
+                     ->  Seq Scan on plt1_ad_p3 t2_3
+(18 rows)
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt2_ad_extra t2_1
+               ->  Seq Scan on plt2_ad_p1 t2_2
+               ->  Seq Scan on plt2_ad_p2 t2_3
+               ->  Seq Scan on plt2_ad_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+(15 rows)
+
+DROP TABLE plt2_ad_extra;
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in those cases
+-- Split plt2_ad_p2 into two partitions so that prt1_ad_p2 matches both
+-- partitions
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2;
+CREATE TABLE plt2_ad_p2_0004 PARTITION OF plt2_ad FOR VALUES IN ('0004');
+CREATE TABLE plt2_ad_p2_0006 PARTITION OF plt2_ad FOR VALUES IN ('0006');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6);
+ANALYZE plt2_ad;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Seq Scan on plt2_ad_p2_0004 t2_2
+               ->  Seq Scan on plt2_ad_p2_0006 t2_3
+               ->  Seq Scan on plt2_ad_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                        QUERY PLAN                        
+----------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Semi Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt1_ad_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+                     ->  Seq Scan on plt2_ad_p2_0004 t2_2
+                     ->  Seq Scan on plt2_ad_p2_0006 t2_3
+                     ->  Seq Scan on plt2_ad_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Seq Scan on plt2_ad_p2_0004 t2_2
+               ->  Seq Scan on plt2_ad_p2_0006 t2_3
+               ->  Seq Scan on plt2_ad_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(17 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                        QUERY PLAN                        
+----------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Anti Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt1_ad_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+                     ->  Seq Scan on plt2_ad_p2_0004 t2_2
+                     ->  Seq Scan on plt2_ad_p2_0006 t2_3
+                     ->  Seq Scan on plt2_ad_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Seq Scan on plt2_ad_p2_0004 t2_2
+               ->  Seq Scan on plt2_ad_p2_0006 t2_3
+               ->  Seq Scan on plt2_ad_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+(15 rows)
+
+DROP TABLE plt2_ad_p2_0004;
+DROP TABLE plt2_ad_p2_0006;
+-- Restore plt2_ad_p2
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_ad;
+-- Test NULL partitions
+-- Change plt1_ad_p1 to the NULL partition
+ALTER TABLE plt1_ad DETACH PARTITION plt1_ad_p1;
+CREATE TABLE plt1_ad_p1_null PARTITION OF plt1_ad FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt1_ad VALUES (-1, -1, NULL);
+ANALYZE plt1_ad;
+-- Change plt2_ad_p3 to the NULL partition
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p3;
+CREATE TABLE plt2_ad_p3_null PARTITION OF plt2_ad FOR VALUES IN (NULL, '0007', '0009');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (7, 9);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1_null t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_ad_p1_null t1_1
+                     Filter: (b < 10)
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3_null t2_3
+(19 rows)
+
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1_null t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a  |  c   | a |  c   
+----+------+---+------
+ -1 |      |   | 
+  1 | 0001 |   | 
+  3 | 0003 | 3 | 0003
+  4 | 0004 | 4 | 0004
+  6 | 0006 | 6 | 0006
+  8 | 0008 |   | 
+  9 | 0009 | 9 | 0009
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_ad_p1_null t1_1
+                     Filter: (b < 10)
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3_null t2_3
+(19 rows)
+
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a  | b  |  c   
+----+----+------
+ -1 | -1 | 
+  1 |  1 | 0001
+  8 |  8 | 0008
+(3 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p1_null t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt2_ad_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+ -1 |      |    | 
+  1 | 0001 |    | 
+  3 | 0003 |  3 | 0003
+  4 | 0004 |  4 | 0004
+  6 | 0006 |  6 | 0006
+  8 | 0008 |    | 
+  9 | 0009 |  9 | 0009
+    |      | -1 | 
+    |      |  2 | 0002
+    |      |  7 | 0007
+(10 rows)
+
+DROP TABLE plt1_ad_p1_null;
+DROP TABLE plt2_ad_p3_null;
+-- Restore plt1_ad_p1
+ALTER TABLE plt1_ad ATTACH PARTITION plt1_ad_p1 FOR VALUES IN ('0001', '0003');
+-- Add to plt1_ad the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt1_ad_extra PARTITION OF plt1_ad FOR VALUES IN (NULL);
+INSERT INTO plt1_ad VALUES (-1, -1, NULL);
+ANALYZE plt1_ad;
+-- Restore plt2_ad_p3
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p3 FOR VALUES IN ('0007', '0009');
+ANALYZE plt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- Partitioned join can't be applied because there isn't any partition on the
+-- plt2_ad side that matches the NULL partition of plt1_ad plt1_ad_extra
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt1_ad_p1 t1_1
+               ->  Seq Scan on plt1_ad_p2 t1_2
+               ->  Seq Scan on plt1_ad_p3 t1_3
+               ->  Seq Scan on plt1_ad_extra t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+                     ->  Seq Scan on plt2_ad_p2 t2_2
+                     ->  Seq Scan on plt2_ad_p3 t2_3
+(15 rows)
+
+-- Add to plt2_ad the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt2_ad_extra PARTITION OF plt2_ad FOR VALUES IN (NULL);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p3 t2_3
+         ->  Hash Full Join
+               Hash Cond: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+               Filter: ((COALESCE(t1_4.b, 0) < 10) AND (COALESCE(t2_4.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_extra t1_4
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_extra t2_4
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+ -1 |      |    | 
+  1 | 0001 |    | 
+  3 | 0003 |  3 | 0003
+  4 | 0004 |  4 | 0004
+  6 | 0006 |  6 | 0006
+  8 | 0008 |    | 
+  9 | 0009 |  9 | 0009
+    |      | -1 | 
+    |      |  2 | 0002
+    |      |  7 | 0007
+(10 rows)
+
+DROP TABLE plt1_ad_extra;
+DROP TABLE plt2_ad_extra;
+-- Test default partitions
+-- Change plt1_ad_p1 to the default partition
+ALTER TABLE plt1_ad DETACH PARTITION plt1_ad_p1;
+ALTER TABLE plt1_ad ATTACH PARTITION plt1_ad_p1 DEFAULT;
+DROP TABLE plt1_ad_p3;
+ANALYZE plt1_ad;
+DROP TABLE plt2_ad_p3;
+ANALYZE plt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_1
+                           Filter: (b < 10)
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(3 rows)
+
+-- Change plt2_ad_p2 to contain '0005' in addition to '0004' and '0006' as
+-- the key values
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2;
+CREATE TABLE plt2_ad_p2_ext PARTITION OF plt2_ad FOR VALUES IN ('0004', '0005', '0006');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 5, 6);
+ANALYZE plt2_ad;
+-- Partitioned join can't be applied because the default partition of plt1_ad
+-- plt1_ad_p1 matches plt2_ad_p1 and plt2_ad_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Seq Scan on plt2_ad_p2_ext t2_2
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p2 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p1 t1_2
+                           Filter: (b < 10)
+(13 rows)
+
+-- Change plt2_ad_p2_ext to the default partition
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2_ext;
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2_ext DEFAULT;
+ANALYZE plt2_ad;
+-- Partitioned join can't be applied because the default partition of plt1_ad
+-- plt1_ad_p1 matches plt2_ad_p1 and plt2_ad_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Seq Scan on plt2_ad_p2_ext t2_2
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p2 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p1 t1_2
+                           Filter: (b < 10)
+(13 rows)
+
+DROP TABLE plt2_ad_p2_ext;
+-- Restore plt2_ad_p2
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_ad;
+CREATE TABLE plt3_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_ad_p1 PARTITION OF plt3_ad FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt3_ad_p2 PARTITION OF plt3_ad FOR VALUES IN ('0007', '0009');
+INSERT INTO plt3_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6, 7, 9);
+ANALYZE plt3_ad;
+-- 3-way join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_ad t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t3_1.a = t1_1.a) AND (t3_1.c = t1_1.c))
+               ->  Seq Scan on plt3_ad_p1 t3_1
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+                           ->  Seq Scan on plt2_ad_p2 t2_2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_ad_p2 t1_1
+                                       Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t3_2.a = t1_2.a) AND (t3_2.c = t1_2.c))
+               ->  Seq Scan on plt3_ad_p2 t3_2
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+                           ->  Seq Scan on plt2_ad_p1 t2_1
+                           ->  Hash
+                                 ->  Seq Scan on plt1_ad_p1 t1_2
+                                       Filter: (b < 10)
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_ad t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   | a |  c   
+---+------+---+------+---+------
+ 1 | 0001 |   |      |   | 
+ 3 | 0003 | 3 | 0003 |   | 
+ 4 | 0004 | 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006 | 6 | 0006
+(4 rows)
+
+-- Test cases where one side has the default partition while the other side
+-- has the NULL partition
+-- Change plt2_ad_p1 to the NULL partition
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p1;
+CREATE TABLE plt2_ad_p1_null PARTITION OF plt2_ad FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p1_null t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_1
+                           Filter: (b < 10)
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 | 1 | 0001
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(4 rows)
+
+-- Change the NULL partition to contain only NULL values as the key values
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p1_null;
+CREATE TABLE plt2_ad_p1_nullonly PARTITION OF plt2_ad FOR VALUES IN (NULL);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Seq Scan on plt2_ad_p2 t2
+         ->  Hash
+               ->  Seq Scan on plt1_ad_p2 t1
+                     Filter: (b < 10)
+(8 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(2 rows)
+
+DROP TABLE plt2_ad_p1_nullonly;
+DROP TABLE plt2_ad_p1_null;
+DROP TABLE plt2_ad_p1;
+DROP TABLE plt1_ad;
+DROP TABLE plt2_ad;
+DROP TABLE plt3_ad;
+CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1_ad;
+CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0002');
+CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2_ad;
+CREATE TABLE plt3_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_ad_p1 PARTITION OF plt3_ad FOR VALUES IN ('0001');
+CREATE TABLE plt3_ad_p2 PARTITION OF plt3_ad FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.c = t2.c)) FULL JOIN plt3_ad t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2.a, t3.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t3_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_1.c = t2_1.c)
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                     ->  Hash
+                           ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt3_ad_p1 t3_1
+         ->  Hash Full Join
+               Hash Cond: (t1_2.c = t3_2.c)
+               Filter: (((COALESCE(t1_2.a, 0) % 5) <> 3) AND ((COALESCE(t1_2.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_2.c = t2_2.c)
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                     ->  Hash
+                           ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt3_ad_p2 t3_2
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.c = t2.c)) FULL JOIN plt3_ad t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+ a  |  c   | a  |  c   | a  |  c   
+----+------+----+------+----+------
+  0 | 0000 |    |      |    | 
+  5 | 0000 |    |      |    | 
+ 10 | 0000 |    |      |    | 
+ 15 | 0000 |    |      |    | 
+ 20 | 0000 |    |      |    | 
+  1 | 0001 |    |      |  1 | 0001
+  1 | 0001 |    |      |  6 | 0001
+  1 | 0001 |    |      | 11 | 0001
+  1 | 0001 |    |      | 16 | 0001
+  1 | 0001 |    |      | 21 | 0001
+  6 | 0001 |    |      |  1 | 0001
+  6 | 0001 |    |      |  6 | 0001
+  6 | 0001 |    |      | 11 | 0001
+  6 | 0001 |    |      | 16 | 0001
+  6 | 0001 |    |      | 21 | 0001
+ 11 | 0001 |    |      |  1 | 0001
+ 11 | 0001 |    |      |  6 | 0001
+ 11 | 0001 |    |      | 11 | 0001
+ 11 | 0001 |    |      | 16 | 0001
+ 11 | 0001 |    |      | 21 | 0001
+ 16 | 0001 |    |      |  1 | 0001
+ 16 | 0001 |    |      |  6 | 0001
+ 16 | 0001 |    |      | 11 | 0001
+ 16 | 0001 |    |      | 16 | 0001
+ 16 | 0001 |    |      | 21 | 0001
+ 21 | 0001 |    |      |  1 | 0001
+ 21 | 0001 |    |      |  6 | 0001
+ 21 | 0001 |    |      | 11 | 0001
+ 21 | 0001 |    |      | 16 | 0001
+ 21 | 0001 |    |      | 21 | 0001
+  2 | 0002 |  2 | 0002 |    | 
+  2 | 0002 |  7 | 0002 |    | 
+  2 | 0002 | 12 | 0002 |    | 
+  2 | 0002 | 17 | 0002 |    | 
+  2 | 0002 | 22 | 0002 |    | 
+  7 | 0002 |  2 | 0002 |    | 
+  7 | 0002 |  7 | 0002 |    | 
+  7 | 0002 | 12 | 0002 |    | 
+  7 | 0002 | 17 | 0002 |    | 
+  7 | 0002 | 22 | 0002 |    | 
+ 12 | 0002 |  2 | 0002 |    | 
+ 12 | 0002 |  7 | 0002 |    | 
+ 12 | 0002 | 12 | 0002 |    | 
+ 12 | 0002 | 17 | 0002 |    | 
+ 12 | 0002 | 22 | 0002 |    | 
+ 17 | 0002 |  2 | 0002 |    | 
+ 17 | 0002 |  7 | 0002 |    | 
+ 17 | 0002 | 12 | 0002 |    | 
+ 17 | 0002 | 17 | 0002 |    | 
+ 17 | 0002 | 22 | 0002 |    | 
+ 22 | 0002 |  2 | 0002 |    | 
+ 22 | 0002 |  7 | 0002 |    | 
+ 22 | 0002 | 12 | 0002 |    | 
+ 22 | 0002 | 17 | 0002 |    | 
+ 22 | 0002 | 22 | 0002 |    | 
+(55 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 575ba7b8d4..c3e47ee549 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -445,11 +445,503 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
 
--- partitionwise join can not be applied if only one of joining tables has
--- default partition
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
-ANALYZE prt2;
 
+--
+-- Test advanced partition-matching algorithm for partitioned join
+--
+
+-- Tests for range-partitioned tables
+CREATE TABLE prt1_ad (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_ad_p1 PARTITION OF prt1_ad FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_ad_p2 PARTITION OF prt1_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_ad_p3 PARTITION OF prt1_ad FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_ad_a_idx on prt1_ad (a);
+INSERT INTO prt1_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_ad;
+
+CREATE TABLE prt2_ad (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_ad_p1 PARTITION OF prt2_ad FOR VALUES FROM (100) TO (150);
+CREATE TABLE prt2_ad_p2 PARTITION OF prt2_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt2_ad_p3 PARTITION OF prt2_ad FOR VALUES FROM (350) TO (500);
+CREATE INDEX prt2_ad_b_idx on prt2_ad (b);
+INSERT INTO prt2_ad_p1 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 149) i;
+INSERT INTO prt2_ad_p2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(200, 299) i;
+INSERT INTO prt2_ad_p3 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_ad;
+
+-- inner join
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+-- Test cases where one side has an extra partition
+
+-- Add an extra partition to prt2_ad
+CREATE TABLE prt2_ad_extra PARTITION OF prt2_ad FOR VALUES FROM (500) TO (MAXVALUE);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(500, 599) i;
+ANALYZE prt2_ad;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c FROM prt2_ad t1 LEFT JOIN prt1_ad t2 ON (t1.b = t2.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt1_ad t2 WHERE t1.b = t2.a) AND t1.a = 0;
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+-- 3-way join where not every pair of relations can do partitioned join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1_ad t1 RIGHT JOIN prt2_ad t2 ON (t1.a = t2.b) INNER JOIN prt1_ad t3 ON (t2.b = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1_ad t1 RIGHT JOIN prt2_ad t2 ON (t1.a = t2.b) INNER JOIN prt1_ad t3 ON (t2.b = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.a, t3.c;
+
+DROP TABLE prt2_ad_extra;
+
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in those cases
+
+-- Split prt2_ad_p3 into two partitions so that prt1_ad_p3 matches both
+-- partitions
+ALTER TABLE prt2_ad DETACH PARTITION prt2_ad_p3;
+CREATE TABLE prt2_ad_p3_350_375 PARTITION OF prt2_ad FOR VALUES FROM (350) TO (375);
+CREATE TABLE prt2_ad_p3_375_500 PARTITION OF prt2_ad FOR VALUES FROM (375) TO (500);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_ad;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+DROP TABLE prt2_ad_p3_350_375;
+DROP TABLE prt2_ad_p3_375_500;
+
+-- Test default partitions
+
+-- Change prt1_ad_p1 to the default partition
+ALTER TABLE prt1_ad DETACH PARTITION prt1_ad_p1;
+ALTER TABLE prt1_ad ATTACH PARTITION prt1_ad_p1 DEFAULT;
+ANALYZE prt1_ad;
+
+CREATE TABLE prt2_ad_p3_300_400 PARTITION OF prt2_ad FOR VALUES FROM (300) TO (400);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(300, 399) i;
+ANALYZE prt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt2_ad_p3_300_400;
+
+-- Restore prt2_ad_p3
+ALTER TABLE prt2_ad ATTACH PARTITION prt2_ad_p3 FOR VALUES FROM (350) TO (500);
+ANALYZE prt2_ad;
+
+-- Partitioned join can't be applied because the default partition of prt1_ad
+-- prt1_ad_p1 matches prt2_ad_p1 and prt2_ad_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- Change prt2_ad_p3 to the default partition
+ALTER TABLE prt2_ad DETACH PARTITION prt2_ad_p3;
+ALTER TABLE prt2_ad ATTACH PARTITION prt2_ad_p3 DEFAULT;
+ANALYZE prt2_ad;
+
+-- Partitioned join can't be applied because the default partition of prt1_ad
+-- prt1_ad_p1 matches prt2_ad_p1 and prt2_ad_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt1_ad_p3;
+ANALYZE prt1_ad;
+
+DROP TABLE prt2_ad_p3;
+ANALYZE prt2_ad;
+
+CREATE TABLE prt3_ad (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt3_ad_p1 PARTITION OF prt3_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt3_ad_p2 PARTITION OF prt3_ad FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt3_ad_a_idx on prt3_ad (a);
+INSERT INTO prt3_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(200, 399) i;
+ANALYZE prt3_ad;
+
+-- 3-way join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) LEFT JOIN prt3_ad t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) LEFT JOIN prt3_ad t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+
+
+-- Tests for list-partitioned tables
+CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0001', '0003');
+CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt1_ad_p3 PARTITION OF plt1_ad FOR VALUES IN ('0008', '0009');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 NOT IN (0, 2, 5, 7);
+ANALYZE plt1_ad;
+
+CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt2_ad_p3 PARTITION OF plt2_ad FOR VALUES IN ('0007', '0009');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 NOT IN (0, 1, 5, 8);
+ANALYZE plt2_ad;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+-- Test cases where one side has an extra partition
+
+-- Add an extra partition to plt2_ad
+CREATE TABLE plt2_ad_extra PARTITION OF plt2_ad FOR VALUES IN ('0000');
+INSERT INTO plt2_ad_extra VALUES (0, 0, '0000');
+ANALYZE plt2_ad;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2_ad t1 LEFT JOIN plt1_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt2_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt1_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt2_ad_extra;
+
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in those cases
+
+-- Split plt2_ad_p2 into two partitions so that prt1_ad_p2 matches both
+-- partitions
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2;
+CREATE TABLE plt2_ad_p2_0004 PARTITION OF plt2_ad FOR VALUES IN ('0004');
+CREATE TABLE plt2_ad_p2_0006 PARTITION OF plt2_ad FOR VALUES IN ('0006');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6);
+ANALYZE plt2_ad;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt2_ad_p2_0004;
+DROP TABLE plt2_ad_p2_0006;
+
+-- Restore plt2_ad_p2
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_ad;
+
+-- Test NULL partitions
+
+-- Change plt1_ad_p1 to the NULL partition
+ALTER TABLE plt1_ad DETACH PARTITION plt1_ad_p1;
+CREATE TABLE plt1_ad_p1_null PARTITION OF plt1_ad FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt1_ad VALUES (-1, -1, NULL);
+ANALYZE plt1_ad;
+
+-- Change plt2_ad_p3 to the NULL partition
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p3;
+CREATE TABLE plt2_ad_p3_null PARTITION OF plt2_ad FOR VALUES IN (NULL, '0007', '0009');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (7, 9);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt1_ad_p1_null;
+DROP TABLE plt2_ad_p3_null;
+
+-- Restore plt1_ad_p1
+ALTER TABLE plt1_ad ATTACH PARTITION plt1_ad_p1 FOR VALUES IN ('0001', '0003');
+
+-- Add to plt1_ad the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt1_ad_extra PARTITION OF plt1_ad FOR VALUES IN (NULL);
+INSERT INTO plt1_ad VALUES (-1, -1, NULL);
+ANALYZE plt1_ad;
+
+-- Restore plt2_ad_p3
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p3 FOR VALUES IN ('0007', '0009');
+ANALYZE plt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Partitioned join can't be applied because there isn't any partition on the
+-- plt2_ad side that matches the NULL partition of plt1_ad plt1_ad_extra
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+-- Add to plt2_ad the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt2_ad_extra PARTITION OF plt2_ad FOR VALUES IN (NULL);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt1_ad_extra;
+DROP TABLE plt2_ad_extra;
+
+-- Test default partitions
+
+-- Change plt1_ad_p1 to the default partition
+ALTER TABLE plt1_ad DETACH PARTITION plt1_ad_p1;
+ALTER TABLE plt1_ad ATTACH PARTITION plt1_ad_p1 DEFAULT;
+DROP TABLE plt1_ad_p3;
+ANALYZE plt1_ad;
+
+DROP TABLE plt2_ad_p3;
+ANALYZE plt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Change plt2_ad_p2 to contain '0005' in addition to '0004' and '0006' as
+-- the key values
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2;
+CREATE TABLE plt2_ad_p2_ext PARTITION OF plt2_ad FOR VALUES IN ('0004', '0005', '0006');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 5, 6);
+ANALYZE plt2_ad;
+
+-- Partitioned join can't be applied because the default partition of plt1_ad
+-- plt1_ad_p1 matches plt2_ad_p1 and plt2_ad_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Change plt2_ad_p2_ext to the default partition
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2_ext;
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2_ext DEFAULT;
+ANALYZE plt2_ad;
+
+-- Partitioned join can't be applied because the default partition of plt1_ad
+-- plt1_ad_p1 matches plt2_ad_p1 and plt2_ad_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt2_ad_p2_ext;
+
+-- Restore plt2_ad_p2
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_ad;
+
+CREATE TABLE plt3_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_ad_p1 PARTITION OF plt3_ad FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt3_ad_p2 PARTITION OF plt3_ad FOR VALUES IN ('0007', '0009');
+INSERT INTO plt3_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6, 7, 9);
+ANALYZE plt3_ad;
+
+-- 3-way join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_ad t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_ad t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Test cases where one side has the default partition while the other side
+-- has the NULL partition
+
+-- Change plt2_ad_p1 to the NULL partition
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p1;
+CREATE TABLE plt2_ad_p1_null PARTITION OF plt2_ad FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Change the NULL partition to contain only NULL values as the key values
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p1_null;
+CREATE TABLE plt2_ad_p1_nullonly PARTITION OF plt2_ad FOR VALUES IN (NULL);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt2_ad_p1_nullonly;
+DROP TABLE plt2_ad_p1_null;
+DROP TABLE plt2_ad_p1;
+
+DROP TABLE plt1_ad;
+DROP TABLE plt2_ad;
+DROP TABLE plt3_ad;
+
+
+CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1_ad;
+
+CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0002');
+CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2_ad;
+
+CREATE TABLE plt3_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_ad_p1 PARTITION OF plt3_ad FOR VALUES IN ('0001');
+CREATE TABLE plt3_ad_p2 PARTITION OF plt3_ad FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.c = t2.c)) FULL JOIN plt3_ad t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.c = t2.c)) FULL JOIN plt3_ad t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
v30-0002-Fix-handling-of-NULL-partitions.patchapplication/octet-stream; name=v30-0002-Fix-handling-of-NULL-partitions.patchDownload
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index d619ab4486..0c1e4ab7f5 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -4528,8 +4528,8 @@ merge_null_partitions(PartitionMap *outer_map,
 					  int *next_index,
 					  int *null_index)
 {
-	bool 		merge_outer_null = false;
-	bool 		merge_inner_null = false;
+	bool 		outer_null_unmerged = false;
+	bool 		inner_null_unmerged = false;
 
 	Assert(outer_has_null || inner_has_null);
 	Assert(*null_index == -1);
@@ -4554,7 +4554,7 @@ merge_null_partitions(PartitionMap *outer_map,
 		 */
 		Assert(outer_null >= 0 && outer_null < outer_map->nparts);
 		if (outer_map->merged_indexes[outer_null] == -1)
-			merge_outer_null = true;
+		 	outer_null_unmerged = true;
 	}
 	if (inner_has_null)
 	{
@@ -4565,15 +4565,14 @@ merge_null_partitions(PartitionMap *outer_map,
 		 * partition below.
 		 */
 		Assert(inner_null >= 0 && inner_null < inner_map->nparts);
-		if (jointype == JOIN_FULL &&
-			inner_map->merged_indexes[inner_null] == -1)
-			merge_inner_null = true;
+		if (inner_map->merged_indexes[inner_null] == -1)
+			inner_null_unmerged = true;
 	}
 
-	if (!merge_outer_null && !merge_inner_null)
+	if (!outer_null_unmerged && !inner_null_unmerged)
 		return;
 
-	if (merge_outer_null && !merge_inner_null)
+	if (outer_null_unmerged && !inner_null_unmerged)
 	{
 		Assert(outer_has_null);
 
@@ -4584,24 +4583,23 @@ merge_null_partitions(PartitionMap *outer_map,
 		*null_index = merge_partition_with_dummy(outer_map, outer_null,
 												 next_index);
 	}
-	else if (!merge_outer_null && merge_inner_null)
+	else if (!outer_null_unmerged && inner_null_unmerged)
 	{
 		Assert(inner_has_null);
-		Assert(jointype == JOIN_FULL);
 
 		/*
 		 * Assign a new merged partition.  The resulting partition should be
 		 * the NULL partition of the joinrel.
 		 */
-		*null_index = merge_partition_with_dummy(inner_map, inner_null,
-												 next_index);
+		if (jointype == JOIN_FULL)
+			*null_index = merge_partition_with_dummy(inner_map, inner_null,
+													 next_index);
 	}
 	else
 	{
-		Assert(merge_outer_null && merge_inner_null);
+		Assert(outer_null_unmerged && inner_null_unmerged);
 		Assert(outer_has_null);
 		Assert(inner_has_null);
-		Assert(jointype == JOIN_FULL);
 
 		/*
 		 * Merge the NULL partitions.  Both partitions aren't merged yet, so
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index dc78bc2503..924cb99863 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -3673,6 +3673,32 @@ SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t
  9 | 0009 | 9 | 0009
 (4 rows)
 
+-- Partitioned join can't be applied because there isn't any partition on the
+-- plt2_ad side that matches the NULL partition of plt1_ad plt1_ad_extra
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Seq Scan on plt2_ad_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_extra t1_4
+                           Filter: (b < 10)
+(18 rows)
+
 -- Partitioned join can't be applied because there isn't any partition on the
 -- plt2_ad side that matches the NULL partition of plt1_ad plt1_ad_extra
 EXPLAIN (COSTS OFF)
@@ -3737,6 +3763,50 @@ SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t
  9 | 0009 | 9 | 0009
 (4 rows)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+         ->  Nested Loop Left Join
+               Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+               ->  Seq Scan on plt1_ad_extra t1_4
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_extra t2_4
+(26 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a  |  c   | a |  c   
+----+------+---+------
+ -1 |      |   | 
+  1 | 0001 |   | 
+  3 | 0003 | 3 | 0003
+  4 | 0004 | 4 | 0004
+  6 | 0006 | 6 | 0006
+  8 | 0008 |   | 
+  9 | 0009 | 9 | 0009
+(7 rows)
+
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
                                     QUERY PLAN                                     
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index c3e47ee549..418ffa117f 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -815,6 +815,11 @@ EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
 SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
 
+-- Partitioned join can't be applied because there isn't any partition on the
+-- plt2_ad side that matches the NULL partition of plt1_ad plt1_ad_extra
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
 -- Partitioned join can't be applied because there isn't any partition on the
 -- plt2_ad side that matches the NULL partition of plt1_ad plt1_ad_extra
 EXPLAIN (COSTS OFF)
@@ -830,6 +835,10 @@ EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
 SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
 
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
 SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
#113Mark Dilger
mark.dilger@enterprisedb.com
In reply to: Etsuro Fujita (#112)
3 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Tue, Dec 10, 2019 at 7:30 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

I don't see any issues in the latest version, but I think we
need to polish the patch, so I'll do that.

I noticed some issues. :-( I think we should address it before
polishing the patch. One thing I noticed is: the patch heavily
modifies the existing test cases in partition_join.sql to test the new
partition-matching algorithm, but I think we should leave those test
cases alone because we would handle the exiting test cases (except one
negative test case) as before (see the try_partitionwise_join()
change in the patch), so those test cases would be still needed to
test that. Attached is a proposed patch for that
(v30-0001-Improve-partition-matching-for-partitionwise-join.patch)
that 1) avoids modifying the existing test cases and 2) adds a
slightly modified version of the test cases proposed in the previous
patch to test the new algorithm. Though I omitted some test cases
that seem redundant to me and added a bit more test cases involving
NULL partitions and/or default partitions. The elapsed time to run
the partition_join.sql regression test increased from 741 ms (HEAD) to
1086 ms in my environment, but I think that would be acceptable. I
fixed one white space issue, but other than that, no code/comment
changes.

Another thing I noticed while working on the above is: the patch fails
to apply PWJ to this case:

CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0001', '0003');
CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0004', '0006');
CREATE TABLE plt1_ad_p3 PARTITION OF plt1_ad FOR VALUES IN ('0008', '0009');
CREATE TABLE plt1_ad_extra PARTITION OF plt1_ad FOR VALUES IN (NULL);
INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM
generate_series(1, 299) i WHERE i % 10 NOT IN (0, 2, 5, 7);
INSERT INTO plt1_ad VALUES (-1, -1, NULL);
ANALYZE plt1_ad;
CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c);
CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0002', '0003');
CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0004', '0006');
CREATE TABLE plt2_ad_p3 PARTITION OF plt2_ad FOR VALUES IN ('0007', '0009');
CREATE TABLE plt2_ad_extra PARTITION OF plt2_ad FOR VALUES IN (NULL);
INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM
generate_series(1, 299) i WHERE i % 10 NOT IN (0, 1, 5, 8);
INSERT INTO plt2_ad VALUES (-1, -1, NULL);
ANALYZE plt2_ad;

EXPLAIN (COSTS OFF)
SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON
(t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
QUERY PLAN
--------------------------------------------------------
Sort
Sort Key: t1.a
-> Hash Right Join
Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
-> Append
-> Seq Scan on plt2_ad_p1 t2_1
-> Seq Scan on plt2_ad_p2 t2_2
-> Seq Scan on plt2_ad_p3 t2_3
-> Seq Scan on plt2_ad_extra t2_4
-> Hash
-> Append
-> Seq Scan on plt1_ad_p1 t1_1
Filter: (b < 10)
-> Seq Scan on plt1_ad_p2 t1_2
Filter: (b < 10)
-> Seq Scan on plt1_ad_p3 t1_3
Filter: (b < 10)
-> Seq Scan on plt1_ad_extra t1_4
Filter: (b < 10)
(19 rows)

because merge_null_partitions() does not consider matching the NULL
partitions from both sides, but matches the NULL partition on the
plt1_ad side and a dummy partition, resulting in a non-PWJ plan (see
[1]). I overlooked this case when modifying that function. :-(
Another patch attached to fix this issue
(v30-0002-Fix-handling-of-NULL-partitions.patch). (We would not need
to fix this, if we could handle the case where a dummy partition is on
the nullable side of an outer join [1], but we can't, so I think it
would be a good idea at least for now to match the NULL partitions
from both sides to do PWJ.)

Best regards,
Etsuro Fujita

[1] https://git.postgresql.org/gitweb/?p=postgresql.git;a=commit;h=7ad6498fd5a654de6e743814c36cf619a3b5ddb6

<v30-0001-Improve-partition-matching-for-partitionwise-join.patch><v30-0002-Fix-handling-of-NULL-partitions.patch>

Fujita-san,

With respect to these two patches: They apply, compile, and pass all the regression tests. The code looks reasonable.

There is stray whitespace in v30-0002:

src/backend/partitioning/partbounds.c:4557: space before tab in indent.
+ outer_null_unmerged = true;

I have added tests checking correctness and showing some partition pruning limitations. Find three patches, attached.

The v31-0001-… patch merely applies your patches as a starting point for the next two patches. It is your work, not mine.

The v31-0002-… patch adds more regression tests for range partitioning. The commit message contains my comments about that.

The v31-0003-… patch adds more regression tests for list partitioning, and again, the commit message contains my comments about that.

I hope this review is helpful.


Mark Dilger
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

Attachments:

v31-0001-Applying-Etsuro-Fujita-s-patches.patchapplication/octet-stream; name=v31-0001-Applying-Etsuro-Fujita-s-patches.patch; x-unix-mode=0644Download
From 672dfb708a3a0a4f4c9f4158245ec130dcbeab15 Mon Sep 17 00:00:00 2001
From: Mark Dilger <mark.dilger@enterprisedb.com>
Date: Mon, 27 Jan 2020 18:06:37 -0800
Subject: [PATCH 1/3] Applying Etsuro Fujita's patches

This applies both these patches:

  v30-0001-Improve-partition-matching-for-partitionwise-join.patch
  v30-0002-Fix-handling-of-NULL-partitions.patch

Those patches are byh Etsuro Fujita, though I fixed one whitespace
issue after applying them.

Author: Etsuro Fujita
---
 src/backend/nodes/outfuncs.c                 |    2 +
 src/backend/optimizer/path/joinrels.c        |  245 +-
 src/backend/optimizer/util/inherit.c         |    2 +
 src/backend/optimizer/util/relnode.c         |   45 +-
 src/backend/partitioning/partbounds.c        | 1613 +++++++++++++
 src/include/nodes/pathnodes.h                |    5 +
 src/include/partitioning/partbounds.h        |    7 +
 src/test/regress/expected/partition_join.out | 2120 +++++++++++++++++-
 src/test/regress/sql/partition_join.sql      |  513 ++++-
 9 files changed, 4477 insertions(+), 75 deletions(-)

diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index d76fae44b8..e42a52fb9b 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -2285,6 +2285,8 @@ _outRelOptInfo(StringInfo str, const RelOptInfo *node)
 	WRITE_BOOL_FIELD(has_eclass_joins);
 	WRITE_BOOL_FIELD(consider_partitionwise_join);
 	WRITE_BITMAPSET_FIELD(top_parent_relids);
+	WRITE_BOOL_FIELD(merged);
+	WRITE_BITMAPSET_FIELD(all_partrels);
 	WRITE_NODE_FIELD(partitioned_child_rels);
 }
 
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index a21c295b99..77e6ff5376 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -48,6 +48,9 @@ static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 												Relids left_relids, Relids right_relids);
 static int	match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel,
 										 bool strict_op);
+static void get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2);
 
 
 /*
@@ -1357,25 +1360,30 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	bool		merged = false;
+	List	   *parts1 = NIL;
+	List	   *parts2 = NIL;
+	ListCell   *lcr1 = NULL;
+	ListCell   *lcr2 = NULL;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1390,34 +1398,107 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		   joinrel->part_scheme == rel2->part_scheme);
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * If we don't have the partition bounds for the join rel yet, try to
+	 * create it along with pairs of partitions to be joined; else generate
+	 * those using the partitioning info for the join rel we already have.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	if (joinrel->nparts == -1)
+	{
+		PartitionScheme part_scheme = joinrel->part_scheme;
+		PartitionBoundInfo boundinfo = NULL;
+		int			nparts = 0;
+
+		Assert(joinrel->boundinfo == NULL);
+		Assert(joinrel->part_rels == NULL);
+
+		/*
+		 * See if the partition bounds for inputs are exactly the same, in
+		 * which case we don't need to work hard: partitions with the same
+		 * partition indexes will form join pairs, and the join rel will have
+		 * the same partition bounds as inputs; otherwise try to merge the
+		 * partition bounds along with generating join pairs.
+		 *
+		 * Even if one or both inputs have merged partition bounds, it'd be
+		 * possible for the partition bounds to be exactly the same, but it
+		 * seems unlikely to be worth the cycles to check; do this check only
+		 * if both inputs have non-merged partition bounds.
+		 */
+		if (!rel1->merged &&
+			!rel2->merged &&
+			rel1->nparts == rel2->nparts &&
+			partition_bounds_equal(part_scheme->partnatts,
+								   part_scheme->parttyplen,
+								   part_scheme->parttypbyval,
+								   rel1->boundinfo, rel2->boundinfo))
+		{
+			boundinfo = rel1->boundinfo;
+			nparts = rel1->nparts;
+		}
+		else
+		{
+			boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											   part_scheme->parttyplen,
+											   part_scheme->parttypbyval,
+											   part_scheme->partsupfunc,
+											   part_scheme->partcollation,
+											   rel1, rel2,
+											   parent_sjinfo->jointype,
+											   &parts1, &parts2);
+			if (boundinfo == NULL)
+			{
+				joinrel->nparts = 0;
+				return;
+			}
+			nparts = list_length(parts1);
+			merged = true;
+		}
+
+		Assert(nparts > 0);
+		joinrel->boundinfo = boundinfo;
+		joinrel->merged = merged;
+		joinrel->nparts = nparts;
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+	}
+	else
+	{
+		Assert(joinrel->nparts > 0);
+		Assert(joinrel->boundinfo);
+		Assert(joinrel->part_rels);
+
+		/*
+		 * If the partition bounds for the join rel are not merged ones,
+		 * inputs are guaranteed to have the same partition bounds, so
+		 * partitions with the same partition indexes will form join pairs;
+		 * else let get_matching_part_pairs() do the work.
+		 */
+		if (joinrel->merged)
+		{
+			get_matching_part_pairs(root, joinrel, rel1, rel2,
+									&parts1, &parts2);
+			Assert(list_length(parts1) == joinrel->nparts);
+			Assert(list_length(parts2) == joinrel->nparts);
+			merged = true;
+		}
+	}
 
-	nparts = joinrel->nparts;
+	if (merged)
+	{
+		lcr1 = list_head(parts1);
+		lcr2 = list_head(parts2);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
-		bool		rel1_empty = (child_rel1 == NULL ||
-								  IS_DUMMY_REL(child_rel1));
-		bool		rel2_empty = (child_rel2 == NULL ||
-								  IS_DUMMY_REL(child_rel2));
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		bool		rel1_empty;
+		bool		rel2_empty;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1425,6 +1506,22 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		if (merged)
+		{
+			child_rel1 = lfirst_node(RelOptInfo, lcr1);
+			child_rel2 = lfirst_node(RelOptInfo, lcr2);
+			lcr1 = lnext(parts1, lcr1);
+			lcr2 = lnext(parts2, lcr2);
+		}
+		else
+		{
+			child_rel1 = rel1->part_rels[cnt_parts];
+			child_rel2 = rel2->part_rels[cnt_parts];
+		}
+
+		rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1));
+		rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2));
+
 		/*
 		 * Check for cases where we can prove that this segment of the join
 		 * returns no rows, due to one or both inputs being empty (including
@@ -1522,6 +1619,8 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 												 child_sjinfo,
 												 child_sjinfo->jointype);
 			joinrel->part_rels[cnt_parts] = child_joinrel;
+			joinrel->all_partrels = bms_add_members(joinrel->all_partrels,
+													child_joinrel->relids);
 		}
 
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
@@ -1738,3 +1837,99 @@ match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op)
 
 	return -1;
 }
+
+/*
+ * get_matching_part_pairs
+ *		Generate join pairs of partitions for the two inputs
+ */
+static void
+get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2)
+{
+	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
+	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
+	int 		cnt_parts;
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
+	{
+		RelOptInfo *child_joinrel = joinrel->part_rels[cnt_parts];
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		Relids		child_relids1;
+		Relids		child_relids2;
+
+		/*
+		 * If this segment of the join is empty, it means that this segment
+		 * was ignored when previously creating child-join paths for it in
+		 * try_partitionwise_join() as it would not contribute to the join
+		 * result, due to one or both inputs being empty; add NULL to each of
+		 * the given lists so that this segment will be ignored again in that
+		 * function.
+		 */
+		if (!child_joinrel)
+		{
+			*parts1 = lappend(*parts1, NULL);
+			*parts2 = lappend(*parts2, NULL);
+			continue;
+		}
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel1 side.
+		 */
+		child_relids1 = bms_intersect(child_joinrel->relids,
+									  rel1->all_partrels);
+		Assert(bms_num_members(child_relids1) == bms_num_members(rel1->relids));
+
+		/*
+		 * Get a child rel for rel1 with the relids.  Note that we should have
+		 * the child rel even if rel1 is a join rel, because in that case the
+		 * partitions specified in the relids would have matching/overlapping
+		 * boundaries, so those partitions should be considered as ones to be
+		 * joined even when planning partitionwise joins of rel1, meaning that
+		 * the child rel would have been built by the time we get here.
+		 */
+		if (rel1_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids1);
+
+			child_rel1 = find_base_rel(root, varno);
+		}
+		else
+			child_rel1 = find_join_rel(root, child_relids1);
+		Assert(child_rel1);
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel2 side.
+		 */
+		child_relids2 = bms_intersect(child_joinrel->relids,
+									  rel2->all_partrels);
+		Assert(bms_num_members(child_relids2) == bms_num_members(rel2->relids));
+
+		/*
+		 * Get a child rel for rel2 with the relids.  See above comments.
+		 */
+		if (rel2_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids2);
+
+			child_rel2 = find_base_rel(root, varno);
+		}
+		else
+			child_rel2 = find_join_rel(root, child_relids2);
+		Assert(child_rel2);
+
+		/*
+		 * The join of rel1 and rel2 is legal, so is the join of the child
+		 * rels obtained above; add them to the given lists as a join pair
+		 * producing this join segment.
+		 */
+		*parts1 = lappend(*parts1, child_rel1);
+		*parts2 = lappend(*parts2, child_rel2);
+	}
+}
diff --git a/src/backend/optimizer/util/inherit.c b/src/backend/optimizer/util/inherit.c
index 7db67fdf34..3132fd35a5 100644
--- a/src/backend/optimizer/util/inherit.c
+++ b/src/backend/optimizer/util/inherit.c
@@ -376,6 +376,8 @@ expand_partitioned_rtentry(PlannerInfo *root, RelOptInfo *relinfo,
 		/* Create the otherrel RelOptInfo too. */
 		childrelinfo = build_simple_rel(root, childRTindex, relinfo);
 		relinfo->part_rels[i] = childrelinfo;
+		relinfo->all_partrels = bms_add_members(relinfo->all_partrels,
+												childrelinfo->relids);
 
 		/* If this child is itself partitioned, recurse */
 		if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 374f93890b..1df6da64ed 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -240,10 +240,12 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->has_eclass_joins = false;
 	rel->consider_partitionwise_join = false;	/* might get changed later */
 	rel->part_scheme = NULL;
-	rel->nparts = 0;
+	rel->nparts = -1;
 	rel->boundinfo = NULL;
+	rel->merged = false;
 	rel->partition_qual = NIL;
 	rel->part_rels = NULL;
+	rel->all_partrels = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->partitioned_child_rels = NIL;
@@ -653,10 +655,12 @@ build_join_rel(PlannerInfo *root,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -829,10 +833,12 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -1639,7 +1645,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1652,24 +1658,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1681,17 +1669,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 95af37b9c7..43c4e03dde 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -25,6 +25,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/pathnodes.h"
 #include "parser/parse_coerce.h"
 #include "partitioning/partbounds.h"
 #include "partitioning/partdesc.h"
@@ -68,6 +69,23 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+typedef struct PartitionMap
+{
+	int			nparts;			/* number of partitions */
+	int		   *merged_indexes;	/* indexes of merged partitions */
+	bool	   *merged;			/* flags to indicate whether partitions are
+								 * merged with non-dummy partitions */
+	bool		did_remapping;	/* did we remap partitions? */
+	int		   *old_indexes;	/* old indexes of merged partitions if
+								 * did_remapping */
+} PartitionMap;
+
+#define compare_range_bounds(partnatts, partsupfunc, partcollations, \
+							 bound1, bound2) \
+	(partition_rbound_cmp(partnatts, partsupfunc, partcollations, \
+						  (bound1)->datums, (bound1)->kind, (bound1)->lower, \
+						  bound2))
+
 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);
@@ -107,6 +125,96 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 									 Expr **keyCol,
 									 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static PartitionBoundInfo partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts);
+static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							JoinType jointype,
+							List **outer_parts, List **inner_parts);
+static void init_partition_map(RelOptInfo *rel, PartitionMap *map);
+static void free_partition_map(PartitionMap *map);
+static int map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_part, int inner_part, int *next_index);
+static int merge_partition_with_dummy(PartitionMap *map, int index,
+						   int *next_index);
+static int process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index);
+static int process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index);
+static void fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes);
+static void generate_matching_part_pairs(RelOptInfo *outer_rel,
+							 RelOptInfo *inner_rel,
+							 PartitionMap *outer_map,
+							 PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_part_list,
+							 List **inner_part_list);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int get_range_partition(PartitionBoundInfo bi, int *lb_pos,
+					PartitionRangeBound *lb, PartitionRangeBound *ub);
+static bool compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+						 Oid *partcollations,
+						 PartitionRangeBound *outer_lb,
+						 PartitionRangeBound *outer_ub,
+						 PartitionRangeBound *inner_lb,
+						 PartitionRangeBound *inner_ub,
+						 int *lb_cmpval, int *ub_cmpval);
+static void get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations, JoinType jointype,
+						PartitionRangeBound *outer_lb,
+						PartitionRangeBound *outer_ub,
+						PartitionRangeBound *inner_lb,
+						PartitionRangeBound *inner_ub,
+						int	lb_cmpval, int ub_cmpval,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub);
+static void add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub,
+						int merged_index,
+						List **merged_datums,
+						List **merged_kinds,
+						List **merged_indexes);
+static void merge_default_partitions(PartitionMap *outer_map,
+						 PartitionMap *inner_map,
+						 bool outer_has_default,
+						 bool inner_has_default,
+						 int outer_default,
+						 int inner_default,
+						 JoinType jointype,
+						 int *next_index,
+						 int *default_index);
+static void merge_null_partitions(PartitionMap *outer_map,
+					  PartitionMap *inner_map,
+					  bool outer_has_null,
+					  bool inner_has_null,
+					  int outer_null,
+					  int inner_null,
+					  JoinType jointype,
+					  int *next_index,
+					  int *null_index);
 
 /*
  * get_qual_from_partbound
@@ -2999,3 +3107,1508 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * This function builds and returns the partition bounds for a join relation
+ * between input relations, creating two lists of partitions, which are
+ * returned to *outer_parts and *inner_parts respectively.  The lists contain
+ * the same number of partitions, and the partitions at the same positions in
+ * the lists indicate join pairs used for partitioned join.
+ *
+ * This function returns NULL, setting *outer_parts and *inner_parts to NIL,
+ * if a partition on one side matches multiple partitions on the other side,
+ * in which case we currently don't support partitioned join.
+ */
+PartitionBoundInfo
+partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
+{
+	PartitionBoundInfo 	merged_bounds;
+	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
+						inner_binfo = inner_rel->boundinfo;
+	char				strategy = outer_binfo->strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = NULL;
+
+			break;
+
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partsupfunc,
+														partcollation,
+														outer_rel,
+														inner_rel,
+														jointype,
+														outer_parts,
+														inner_parts);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(partnatts,
+														 partsupfunc,
+														 partcollation,
+														 outer_rel,
+														 inner_rel,
+														 jointype,
+														 outer_parts,
+														 inner_parts);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	return merged_bounds;
+}
+
+/*
+ * get_range_partition
+ *		Returns the index of the range partition with the given lower bound
+ *
+ * *lb and *ub are set to the lower and upper bounds of the range partition
+ * respectively, and *lb_index is advanced to the next lower bound, if any.
+ */
+static int
+get_range_partition(PartitionBoundInfo bi, int *lb_index,
+					PartitionRangeBound *lb, PartitionRangeBound *ub)
+{
+	/* Return the index as -1 if we've exhausted all the lower bounds. */
+	if (*lb_index >= bi->ndatums)
+		return -1;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(*lb_index + 1 < bi->ndatums);
+
+	lb->index = bi->indexes[*lb_index];
+	lb->kind = bi->kind[*lb_index];
+	lb->datums = bi->datums[*lb_index];
+	lb->lower = true;
+	ub->index = bi->indexes[*lb_index + 1];
+	ub->kind = bi->kind[*lb_index + 1];
+	ub->datums = bi->datums[*lb_index + 1];
+	ub->lower = false;
+
+	/* The partition index of an upper bound should be valid. */
+	Assert(ub->index >= 0);
+
+	/*
+	 * Advance the lower bound; if there are no bounds left beyond the upper
+	 * bound, we have reached the last lower bound.
+	 */
+	if (*lb_index + 2 >= bi->ndatums)
+		*lb_index = bi->ndatums;
+	else
+	{
+		/*
+		 * If the index assigned to the bound next to the upper bound isn't
+		 * valid, that is the lower bound of the next range partition; else,
+		 * the upper bound of the current range partition is also the lower
+		 * bound of the next range partition.
+		 */
+		if (bi->indexes[*lb_index + 2] < 0)
+			*lb_index = *lb_index + 2;
+		else
+			*lb_index = *lb_index + 1;
+	}
+
+	return ub->index;
+}
+
+/*
+ * compare_range_partitions
+ *		Compares the bounds of two range partitions, and returns true if the
+ *		ranges of the partitions overlap, false otherwise
+ *
+ * *lb_cmpval is set to -1, 0 or 1 if the outer partition's lower bound is
+ * lower than, equal to or higher than the inner partition's lower bound
+ * respectively.  Likewise, *ub_cmpval is set to -1, 0 or 1 if the outer
+ * partition's upper bound is lower than, equal to or higher than the inner
+ * partition's upper bound respectively.
+ */
+static bool
+compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+						 Oid *partcollations,
+						 PartitionRangeBound *outer_lb,
+						 PartitionRangeBound *outer_ub,
+						 PartitionRangeBound *inner_lb,
+						 PartitionRangeBound *inner_ub,
+						 int *lb_cmpval, int *ub_cmpval)
+{
+	/*
+	 * Check to see if the upper bound of the outer partition is lower than
+	 * the lower bound of the inner partition; in which case the partitions
+	 * aren't overlapping.
+	 */
+	if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+							 outer_ub, inner_lb) < 0)
+	{
+		*lb_cmpval = -1;
+		*ub_cmpval = -1;
+		return false;
+	}
+
+	/*
+	 * Check to see if the lower bound of the outer partition is higher than
+	 * the upper bound of the inner partition; in which case the partitions
+	 * aren't overlapping.
+	 */
+	if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+							 outer_lb, inner_ub) > 0)
+	{
+		*lb_cmpval = 1;
+		*ub_cmpval = 1;
+		return false;
+	}
+
+	/* All other cases indicate overlapping partitions. */
+	*lb_cmpval = compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									  outer_lb, inner_lb);
+	*ub_cmpval = compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									  outer_ub, inner_ub);
+	return true;
+}
+
+/*
+ * get_merged_range_bounds
+ *		Given the bounds of range partitions to be join, determine the range
+ *		bounds of the merged partition produced from the range partitions
+ *
+ * *merged_lb and *merged_ub are set to the lower and upper bounds of the
+ * merged partition.
+ */
+static void
+get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations, JoinType jointype,
+						PartitionRangeBound *outer_lb,
+						PartitionRangeBound *outer_ub,
+						PartitionRangeBound *inner_lb,
+						PartitionRangeBound *inner_ub,
+						int	lb_cmpval, int ub_cmpval,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub)
+{
+	Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								outer_lb, inner_lb) == lb_cmpval);
+	Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								outer_ub, inner_ub) == ub_cmpval);
+
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			*merged_lb = (lb_cmpval > 0) ? *outer_lb : *inner_lb;
+			*merged_ub = (ub_cmpval < 0) ? *outer_ub : *inner_ub;
+			break;
+
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = *outer_ub;
+			*merged_lb = *outer_lb;
+			break;
+
+		case JOIN_FULL:
+			*merged_lb = (lb_cmpval < 0) ? *outer_lb : *inner_lb;
+			*merged_ub = (ub_cmpval > 0) ? *outer_ub : *inner_ub;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+}
+
+/*
+ * add_merged_range_bounds
+ *		Add the range bounds of a merged partition to the lists of range
+ *		bounds
+ */
+static void
+add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub,
+						int merged_index,
+						List **merged_datums,
+						List **merged_kinds,
+						List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		/* First merged partition */
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		Assert(*merged_kinds && *merged_indexes);
+
+		/* Get the last upper bound. */
+		prev_ub.index = llast_int(*merged_indexes);
+		prev_ub.datums = (Datum *) llast(*merged_datums);
+		prev_ub.kind = (PartitionRangeDatumKind *) llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		/*
+		 * We pass to partition_rbound_cmp() lower1 as false to prevent it
+		 * from considering the last upper bound to be smaller than the lower
+		 * bound of the merged partition when the values of the two range
+		 * bounds compare equal.
+		 */
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  merged_lb->datums, merged_lb->kind,
+									  false, &prev_ub);
+		Assert(cmpval >= 0);
+	}
+
+	/*
+	 * If the lower bound is higher than the last upper bound, add the lower
+	 * bound with the index as -1 indicating that that is a lower bound; else,
+	 * the last upper bound will be reused as the lower bound of the merged
+	 * partition, so skip this.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, merged_lb->datums);
+		*merged_kinds = lappend(*merged_kinds, merged_lb->kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	/* Add the upper bound and index of the merged partition. */
+	*merged_datums = lappend(*merged_datums, merged_ub->datums);
+	*merged_kinds = lappend(*merged_kinds, merged_ub->kind);
+	*merged_indexes = lappend_int(*merged_indexes, merged_index);
+}
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			outer_part;
+	int			inner_part;
+	int			outer_lb_index;
+	int			inner_lb_index;
+	PartitionRangeBound outer_lb;
+	PartitionRangeBound outer_ub;
+	PartitionRangeBound inner_lb;
+	PartitionRangeBound inner_ub;
+	int			next_index = 0;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_kinds = NIL;
+	List	   *merged_indexes = NIL;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = inner_lb_index = 0;
+	outer_part = get_range_partition(outer_bi, &outer_lb_index,
+									 &outer_lb, &outer_ub);
+	inner_part = get_range_partition(inner_bi, &inner_lb_index,
+									 &inner_lb, &inner_ub);
+	while (outer_part >= 0 || inner_part >= 0)
+	{
+		PartitionRangeBound merged_lb;
+		PartitionRangeBound merged_ub;
+		int			merged_index = -1;
+		bool		overlap;
+		int			ub_cmpval;
+		int			lb_cmpval;
+
+		if (outer_part >= 0)
+			Assert(outer_map.merged_indexes[outer_part] == -1 &&
+				   outer_map.merged[outer_part] == false);
+		if (inner_part >= 0)
+			Assert(inner_map.merged_indexes[inner_part] == -1 &&
+				   inner_map.merged[inner_part] == false);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (outer_part == -1)
+		{
+			overlap = false;
+			lb_cmpval = 1;
+			ub_cmpval = 1;
+		}
+		else if (inner_part == -1)
+		{
+			overlap = false;
+			lb_cmpval = -1;
+			ub_cmpval = -1;
+		}
+		else
+			overlap = compare_range_partitions(partnatts, partsupfuncs,
+											   partcollations,
+											   &outer_lb, &outer_ub,
+											   &inner_lb, &inner_ub,
+											   &lb_cmpval, &ub_cmpval);
+
+		if (overlap)
+		{
+			/* The ranges of partitions overlap; form a join pair of them. */
+
+			PartitionRangeBound save_outer_ub;
+			PartitionRangeBound save_inner_ub;
+
+			/* Get the bounds of the merged partition. */
+			get_merged_range_bounds(partnatts, partsupfuncs,
+									partcollations, jointype,
+									&outer_lb, &outer_ub,
+									&inner_lb, &inner_ub,
+									lb_cmpval, ub_cmpval,
+									&merged_lb, &merged_ub);
+
+			/*
+			 * Get the index of the merged partition.  Both partitions aren't
+			 * merged yet, so the partitions should be merged successfully.
+			 */
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
+													outer_part, inner_part,
+													&next_index);
+			Assert(merged_index >= 0);
+
+			/* Save the upper bounds of both partitions for use below. */
+			save_outer_ub = outer_ub;
+			save_inner_ub = inner_ub;
+
+			/* Move to the next pair of partitions. */
+			outer_part = get_range_partition(outer_bi, &outer_lb_index,
+											 &outer_lb, &outer_ub);
+			inner_part = get_range_partition(inner_bi, &inner_lb_index,
+											 &inner_lb, &inner_ub);
+
+			/*
+			 * If the range of a partition on one side overlaps the range of
+			 * the next partition on the other side, that will cause the
+			 * partition on one side to match at least two partitions on the
+			 * other side, which is the case that we currently don't support
+			 * partitioned join for; give up.
+			 */
+			if (ub_cmpval > 0 && inner_part >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &save_outer_ub, &inner_lb) > 0)
+				goto cleanup;
+			if (ub_cmpval < 0 && outer_part >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &outer_lb, &save_inner_ub) < 0)
+				goto cleanup;
+
+			/*
+			 * A row from a non-overlapping portion (if any) of a partition
+			 * on one side might find its join partner in the default
+			 * partition (if any) on the other side, causing the same
+			 * situation as above; if so, give up.
+			 */
+			if ((outer_has_default && (lb_cmpval > 0 || ub_cmpval < 0)) ||
+				(inner_has_default && (lb_cmpval < 0 || ub_cmpval > 0)))
+				goto cleanup;
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			merged_lb = outer_lb;
+			merged_ub = outer_ub;
+
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				merged_index = process_outer_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   outer_part,
+													   inner_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+			}
+
+			/* Move to the next partition on the outer side. */
+			outer_part = get_range_partition(outer_bi, &outer_lb_index,
+											 &outer_lb, &outer_ub);
+		}
+		else
+		{
+			/* Upper bound of outer range higher than that of the inner. */
+			Assert(ub_cmpval > 0);
+
+			merged_lb = inner_lb;
+			merged_ub = inner_ub;
+
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				merged_index = process_inner_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   inner_part,
+													   outer_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+			}
+
+			/* Move to the next partition on the inner side. */
+			inner_part = get_range_partition(inner_bi, &inner_lb_index,
+											 &inner_lb, &inner_ub);
+		}
+
+		if (merged_index >= 0 && merged_index != default_index)
+		{
+			/* Add the range bounds of the merged partition. */
+			add_merged_range_bounds(partnatts, partsupfuncs, partcollations,
+									&merged_lb, &merged_ub, merged_index,
+									&merged_datums, &merged_kinds,
+									&merged_indexes);
+		}
+	}
+
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+		merge_default_partitions(&outer_map, &inner_map,
+								 outer_has_default, inner_has_default,
+								 outer_default, inner_default,
+								 jointype, &next_index, &default_index);
+	else
+		Assert(default_index == -1);
+
+	/*
+	 * Unlike the case for list partitioning, we wouldn't have re-merged
+	 * partitions, so did_remapping should be left alone.
+	 */
+	Assert(!outer_map.did_remapping && !inner_map.did_remapping);
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 &outer_map, &inner_map,
+								 next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes,
+													  merged_kinds,
+													  -1, default_index);
+	}
+
+cleanup:
+	/* Free local memory before returning. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							JoinType jointype,
+							List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			next_index = 0;
+	int			null_index = -1;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			cnto;
+	int			cnti;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *merged_datum = NULL;
+		int			merged_index = -1;
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			cmpval;
+
+		/* Get the list datums of the next pair of partitions. */
+		odatums = cnto < outer_bi->ndatums ? outer_bi->datums[cnto] : NULL;
+		idatums = cnti < inner_bi->ndatums ? inner_bi->datums[cnti] : NULL;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+			cmpval = 1;
+		else if (cnti >= inner_bi->ndatums)
+			cmpval = -1;
+		else
+		{
+			Assert(odatums != NULL && idatums != NULL);
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0],
+													 idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			int			o_index = outer_bi->indexes[cnto];
+			int			i_index = inner_bi->indexes[cnti];
+
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			Assert(o_index >= 0 && i_index >= 0);
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
+													o_index, i_index,
+													&next_index);
+
+			if (merged_index == -1)
+				goto cleanup;
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			Assert(cnto < outer_bi->ndatums);
+
+			/* A datum missing from the inner side. */
+			merged_datum = odatums;
+
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				int			o_index = outer_bi->indexes[cnto];
+
+				Assert(o_index >= 0);
+				merged_index = process_outer_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   o_index,
+													   inner_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+			}
+
+			/* Move to the next datum on the outer side. */
+			cnto++;
+		}
+		else
+		{
+			Assert(cmpval > 0);
+			Assert(cnti < inner_bi->ndatums);
+
+			/* A datum missing from the outer side. */
+			merged_datum = idatums;
+
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				int			i_index = inner_bi->indexes[cnti];
+
+				Assert(i_index >= 0);
+				merged_index = process_inner_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   i_index,
+													   outer_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+			}
+
+			/* Move to the next datum on the inner side. */
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0 && merged_index != default_index)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	/* Merge null partitions if any. */
+	if (outer_has_null || inner_has_null)
+		merge_null_partitions(&outer_map, &inner_map,
+							  outer_has_null, inner_has_null,
+							  outer_bi->null_index, inner_bi->null_index,
+							  jointype, &next_index, &null_index);
+	else
+		Assert(null_index == -1);
+
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+		merge_default_partitions(&outer_map, &inner_map,
+								 outer_has_default, inner_has_default,
+								 outer_default, inner_default,
+								 jointype, &next_index, &default_index);
+	else
+		Assert(default_index == -1);
+
+	/* Fix the merged_indexes list if necessary. */
+	if (outer_map.did_remapping || inner_map.did_remapping)
+	{
+		Assert(jointype == JOIN_FULL);
+		fix_merged_indexes(&outer_map, &inner_map,
+						   next_index, merged_indexes);
+	}
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 &outer_map, &inner_map,
+								 next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) <= next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes, NIL,
+													  null_index, default_index);
+	}
+
+cleanup:
+	/* Free local memory before returning. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+	list_free(merged_datums);
+	list_free(merged_indexes);
+
+	return merged_bounds;
+}
+
+/*
+ * init_partition_map
+ *
+ * Initialize a PartitionMap struct for given relation.
+ */
+static void
+init_partition_map(RelOptInfo *rel, PartitionMap *map)
+{
+	int			nparts = rel->nparts;
+	int			i;
+
+	map->nparts = nparts;
+	map->merged_indexes = (int *) palloc(sizeof(int) * nparts);
+	map->merged = (bool *) palloc(sizeof(bool) * nparts);
+	map->did_remapping = false;
+	map->old_indexes = (int *) palloc(sizeof(int) * nparts);
+	for (i = 0; i < nparts; i++)
+	{
+		map->merged_indexes[i] = map->old_indexes[i] = -1;
+		map->merged[i] = false;
+	}
+}
+
+/*
+ * free_partition_map
+ */
+static void
+free_partition_map(PartitionMap *map)
+{
+	pfree(map->merged_indexes);
+	pfree(map->merged);
+	pfree(map->old_indexes);
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given partitions.
+ */
+static int
+map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_index, int inner_index, int *next_index)
+{
+	int 		outer_merged_index;
+	bool 		outer_merged;
+	int 		inner_merged_index;
+	bool 		inner_merged;
+
+	Assert(outer_index >= 0 && outer_index < outer_map->nparts);
+	outer_merged_index = outer_map->merged_indexes[outer_index];
+	outer_merged = outer_map->merged[outer_index];
+	Assert(inner_index >= 0 && inner_index < inner_map->nparts);
+	inner_merged_index = inner_map->merged_indexes[inner_index];
+	inner_merged = inner_map->merged[inner_index];
+
+	/*
+	 * Handle cases where both partitions are mapped to merged partitions.
+	 */
+	if (outer_merged_index >= 0 && inner_merged_index >= 0)
+	{
+		/*
+		 * If the mereged partitions are the same, no need to do anything;
+		 * return the index of the merged partition.  Otherwise, if both
+		 * partitions are merged with dummy partitions, re-merge them; map
+		 * them to the merged partition with the smaller of the two merged
+		 * indexes and return the smaller index.  Otherwise they can't be
+		 * merged, so return -1.
+		 */
+		if (outer_merged_index == inner_merged_index)
+		{
+			Assert(outer_merged);
+			Assert(inner_merged);
+			return outer_merged_index;
+		}
+		if (!outer_merged && !inner_merged)
+		{
+			/*
+			 * Note that we will fix the larger index that have been added to
+			 * the merged_indexes list so far in fix_merged_indexes().
+			 */
+			if (outer_merged_index < inner_merged_index)
+			{
+				outer_map->merged[outer_index] = true;
+				inner_map->merged_indexes[inner_index] = outer_merged_index;
+				inner_map->merged[inner_index] = true;
+				inner_map->did_remapping = true;
+				inner_map->old_indexes[inner_index] = inner_merged_index;
+				return outer_merged_index;
+			}
+			else
+			{
+				inner_map->merged[inner_index] = true;
+				outer_map->merged_indexes[outer_index] = inner_merged_index;
+				outer_map->merged[outer_index] = true;
+				outer_map->did_remapping = true;
+				outer_map->old_indexes[outer_index] = outer_merged_index;
+				return inner_merged_index;
+			}
+		}
+		return -1;
+	}
+
+	/* At least one partition isn't mapped to a merged partition. */
+	Assert(outer_merged_index == -1 || inner_merged_index == -1);
+
+	/*
+	 * If neither of partitions isn't mapped, assign them a new merged
+	 * partition and return the index of the merged partition.  Otherwise, if
+	 * one of partitions is merged with a dummy relation (and the other isn't
+	 * merged), re-merge it with the other, with the same index, and return
+	 * the index.  Otherwise they can't be merged, so return -1.
+	 */
+	if (outer_merged_index == -1 && inner_merged_index == -1)
+	{
+		int 		merged_index = *next_index;
+
+		Assert(!outer_merged);
+		Assert(!inner_merged);
+		outer_map->merged_indexes[outer_index] = merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged_indexes[inner_index] = merged_index;
+		inner_map->merged[inner_index] = true;
+		*next_index = *next_index + 1;
+		return merged_index;
+	}
+	if (outer_merged_index >= 0 && !outer_map->merged[outer_index])
+	{
+		Assert(inner_merged_index == -1);
+		Assert(!inner_merged);
+		inner_map->merged_indexes[inner_index] = outer_merged_index;
+		inner_map->merged[inner_index] = true;
+		outer_map->merged[outer_index] = true;
+		return outer_merged_index;
+	}
+	if (inner_merged_index >= 0 && !inner_map->merged[inner_index])
+	{
+		Assert(outer_merged_index == -1);
+		Assert(!outer_merged);
+		outer_map->merged_indexes[outer_index] = inner_merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged[inner_index] = true;
+		return inner_merged_index;
+	}
+	return -1;
+}
+
+/*
+ * merge_partition_with_dummy
+ *
+ * *next_index is incremented.
+ */
+static int
+merge_partition_with_dummy(PartitionMap *map, int index, int *next_index)
+{
+	int 		merged_index = *next_index;
+
+	Assert(index >= 0 && index < map->nparts);
+	Assert(map->merged_indexes[index] == -1);
+	Assert(!map->merged[index]);
+	map->merged_indexes[index] = merged_index;
+	/* Leave the merged flag alone! */
+	*next_index = *next_index + 1;
+	return merged_index;
+}
+
+/*
+ * process_outer_partition
+ *
+ * Determine the merged partition associated with the given outer partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given outer partition.
+ */
+static int
+process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index)
+{
+	int 		merged_index = -1;
+
+	Assert(outer_index >= 0);
+
+	/*
+	 * If the inner side has the default partition, the outer partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an outer join, in which case the outer partition has to be
+	 * scanned all the way anyway; if the outer partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the outer partition with a dummy partition.
+	 */
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0);
+
+		/*
+		 * If the outer side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (outer_has_default)
+			return -1;
+
+		merged_index = map_and_merge_partitions(outer_map, inner_map,
+												outer_index, inner_default,
+												next_index);
+		if (merged_index == -1)
+			return -1;
+
+		/*
+		 * If this is a FULL join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (*default_index == -1)
+				*default_index = merged_index;
+			else
+				Assert(*default_index == merged_index);
+		}
+	}
+	else
+	{
+		Assert(IS_OUTER_JOIN(jointype));
+		Assert(jointype != JOIN_RIGHT);
+
+		merged_index = outer_map->merged_indexes[outer_index];
+		if (merged_index == -1)
+			merged_index = merge_partition_with_dummy(outer_map, outer_index,
+													  next_index);
+	}
+	return merged_index;
+}
+
+/*
+ * process_inner_partition
+ *
+ * Determine the merged partition associated with the given inner partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given inner partition.
+ */
+static int
+process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index)
+{
+	int 		merged_index = -1;
+
+	Assert(inner_index >= 0);
+
+	/*
+	 * If the outer side has the default partition, the inner partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an FULL join, in which case the inner partition has to be
+	 * scanned all the way anyway; if the inner partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the inner partition with a dummy partition.
+	 */
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0);
+
+		/*
+		 * If the inner side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (inner_has_default)
+			return -1;
+
+		merged_index = map_and_merge_partitions(outer_map, inner_map,
+												outer_default, inner_index,
+												next_index);
+		if (merged_index == -1)
+			return -1;
+
+		/*
+		 * If this is an outer join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			if (*default_index == -1)
+				*default_index = merged_index;
+			else
+				Assert(*default_index == merged_index);
+		}
+	}
+	else
+	{
+		Assert(jointype == JOIN_FULL);
+
+		merged_index = inner_map->merged_indexes[inner_index];
+		if (merged_index == -1)
+			merged_index = merge_partition_with_dummy(inner_map, inner_index,
+													  next_index);
+	}
+	return merged_index;
+}
+
+/*
+ * fix_merged_indexes
+ */
+static void
+fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes)
+{
+	int		   *new_indexes;
+	int			merged_index;
+	int			i;
+	ListCell   *lc;
+
+	new_indexes = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		new_indexes[i] = -1;
+
+	/* Build the mapping of old merged indexes to new merged indexes. */
+	if (outer_map->did_remapping)
+	{
+		for (i = 0; i < outer_map->nparts; i++)
+		{
+			merged_index = outer_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = outer_map->merged_indexes[i];
+		}
+	}
+	if (inner_map->did_remapping)
+	{
+		for (i = 0; i < inner_map->nparts; i++)
+		{
+			merged_index = inner_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = inner_map->merged_indexes[i];
+		}
+	}
+
+	/* Fix the merged_indexes list using the mapping. */
+	foreach(lc, merged_indexes)
+	{
+		merged_index = lfirst_int(lc);
+		Assert(merged_index >= 0);
+		if (new_indexes[merged_index] >= 0)
+			lfirst_int(lc) = new_indexes[merged_index];
+	}
+
+	pfree(new_indexes);
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * This function produces the list pairs of partitions which produce merged
+ * partitions in the order of merged partition indexes.
+ *
+ * nmerged is the number of merged partitions.
+ *
+ * If successful, the list pairs are returned as two separate lists,
+ * outer_part_list and inner_part_list, one for each side.  Otherwise, those
+ * lists will be set to NIL.
+ */
+static void
+generate_matching_part_pairs(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 PartitionMap *outer_map, PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_part_list, List **inner_part_list)
+{
+	int			outer_nparts = outer_map->nparts;
+	int			inner_nparts = inner_map->nparts;
+	int		   *outer_part_array;
+	int		   *inner_part_array;
+	int			max_nparts;
+	int 		i;
+
+	Assert(outer_nparts == outer_rel->nparts);
+	Assert(inner_nparts == inner_rel->nparts);
+
+	outer_part_array = (int *) palloc(sizeof(int) * nmerged);
+	inner_part_array = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		outer_part_array[i] = inner_part_array[i] = -1;
+
+	/* Set pairs of matching partitions. */
+	max_nparts = Max(outer_nparts, inner_nparts);
+	for (i = 0; i < max_nparts; i++)
+	{
+		if (i < outer_nparts)
+		{
+			int 		merged_index = outer_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				outer_part_array[merged_index] = i;
+			}
+		}
+		if (i < inner_nparts)
+		{
+			int 		merged_index = inner_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				inner_part_array[merged_index] = i;
+			}
+		}
+	}
+
+	*outer_part_list = NIL;
+	*inner_part_list = NIL;
+
+	/* Build the list pairs. */
+	for (i = 0; i < nmerged; i++)
+	{
+		int			outer_part = outer_part_array[i];
+		int			inner_part = inner_part_array[i];
+
+		/*
+		 * If both partitions are dummy, it means the merged partition was
+		 * removed by partition re-merging in map_and_merge_partitions();
+		 * ignore the merged partition
+		 */
+		if (outer_part == -1 && inner_part == -1)
+			continue;
+
+		*outer_part_list = lappend(*outer_part_list, outer_part >= 0 ?
+								   outer_rel->part_rels[outer_part] : NULL);
+		*inner_part_list = lappend(*inner_part_list, inner_part >= 0 ?
+								   inner_rel->part_rels[inner_part] : NULL);
+	}
+
+	pfree(outer_part_array);
+	pfree(inner_part_array);
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * merge_default_partitions
+ *		Merge the default partitions from both sides, if any
+ *
+ * *default_index is set to the index of the default partition of the joinrel,
+ * if appropriate.
+ */
+static void
+merge_default_partitions(PartitionMap *outer_map,
+						 PartitionMap *inner_map,
+						 bool outer_has_default,
+						 bool inner_has_default,
+						 int outer_default,
+						 int inner_default,
+						 JoinType jointype,
+						 int *next_index,
+						 int *default_index)
+{
+	int 		outer_merged_index = -1;
+	int 		inner_merged_index = -1;
+
+	Assert(outer_has_default || inner_has_default);
+
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0 && outer_default < outer_map->nparts);
+		outer_merged_index = outer_map->merged_indexes[outer_default];
+	}
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0 && inner_default < inner_map->nparts);
+		inner_merged_index = inner_map->merged_indexes[inner_default];
+	}
+
+	if (outer_has_default && !inner_has_default)
+	{
+		/*
+		 * If this is an outer join, assign a new merged partition if not
+		 * already done.  The resulting partition should be the default
+		 * partition of the joinrel.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			if (outer_merged_index == -1)
+				*default_index = merge_partition_with_dummy(outer_map,
+															outer_default,
+															next_index);
+			else
+				Assert(*default_index == outer_merged_index);
+		}
+		else
+			Assert(*default_index == -1);
+	}
+	else if (!outer_has_default && inner_has_default)
+	{
+		/*
+		 * If this is a FULL join, assign a new merged partition if not
+		 * already done.  The resulting partition should be the default
+		 * partition of the joinrel.
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (inner_merged_index == -1)
+				*default_index = merge_partition_with_dummy(inner_map,
+															inner_default,
+															next_index);
+			else
+				Assert(*default_index == inner_merged_index);
+		}
+		else
+			Assert(*default_index == -1);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+		Assert(outer_merged_index == -1);
+		Assert(inner_merged_index == -1);
+		Assert(*default_index == -1);
+
+		/*
+		 * Merge the default partitions.  Both partitions aren't merged yet,
+		 * so the partitions should be merged successfully.  The resulting
+		 * partition should be the default partition of the joinrel.
+		 */
+		*default_index = map_and_merge_partitions(outer_map,
+												  inner_map,
+												  outer_default,
+												  inner_default,
+												  next_index);
+		Assert(*default_index >= 0);
+	}
+}
+
+/*
+ * merge_null_partitions
+ *		Merge the NULL partitions from both sides, if any
+ *
+ * *null_index is set to the index of the NULL partition of the joinrel, if
+ * appropriate.
+ */
+static void
+merge_null_partitions(PartitionMap *outer_map,
+					  PartitionMap *inner_map,
+					  bool outer_has_null,
+					  bool inner_has_null,
+					  int outer_null,
+					  int inner_null,
+					  JoinType jointype,
+					  int *next_index,
+					  int *null_index)
+{
+	bool 		outer_null_unmerged = false;
+	bool 		inner_null_unmerged = false;
+
+	Assert(outer_has_null || inner_has_null);
+	Assert(*null_index == -1);
+
+	/*
+	 * If this is an INNER or SEMI join, two NULL values won't be considered
+	 * to be equal, in which case we don't need to do anything.
+	 */
+	if (jointype == JOIN_INNER || jointype == JOIN_SEMI)
+		return;
+
+	Assert(IS_OUTER_JOIN(jointype));
+	Assert(jointype != JOIN_RIGHT);
+
+	if (outer_has_null)
+	{
+		/*
+		 * Since this is an outer join, if the NULL partition of the outer
+		 * side isn't merged yet, it means that the partition contains only
+		 * NULL values as the key values, in which case we need to create a
+		 * merged partition below.
+		 */
+		Assert(outer_null >= 0 && outer_null < outer_map->nparts);
+		if (outer_map->merged_indexes[outer_null] == -1)
+			outer_null_unmerged = true;
+	}
+	if (inner_has_null)
+	{
+		/*
+		 * If this is a FULL join, and the NULL partition of the inner side
+		 * isn't merged yet, it means that the partition contains only NULL
+		 * values as the key values, in which case we need to create a merged
+		 * partition below.
+		 */
+		Assert(inner_null >= 0 && inner_null < inner_map->nparts);
+		if (inner_map->merged_indexes[inner_null] == -1)
+			inner_null_unmerged = true;
+	}
+
+	if (!outer_null_unmerged && !inner_null_unmerged)
+		return;
+
+	if (outer_null_unmerged && !inner_null_unmerged)
+	{
+		Assert(outer_has_null);
+
+		/*
+		 * Assign a new merged partition.  The resulting partition should be
+		 * the NULL partition of the joinrel.
+		 */
+		*null_index = merge_partition_with_dummy(outer_map, outer_null,
+												 next_index);
+	}
+	else if (!outer_null_unmerged && inner_null_unmerged)
+	{
+		Assert(inner_has_null);
+
+		/*
+		 * Assign a new merged partition.  The resulting partition should be
+		 * the NULL partition of the joinrel.
+		 */
+		if (jointype == JOIN_FULL)
+			*null_index = merge_partition_with_dummy(inner_map, inner_null,
+													 next_index);
+	}
+	else
+	{
+		Assert(outer_null_unmerged && inner_null_unmerged);
+		Assert(outer_has_null);
+		Assert(inner_has_null);
+
+		/*
+		 * Merge the NULL partitions.  Both partitions aren't merged yet, so
+		 * the partitions should be merged successfully.  The resulting
+		 * partition should be the NULL partition of the joinrel.
+		 */
+		*null_index = map_and_merge_partitions(outer_map, inner_map,
+											   outer_null, inner_null,
+											   next_index);
+		Assert(*null_index >= 0);
+	}
+}
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index 3d3be197e0..1545877d8c 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -579,8 +579,10 @@ typedef struct PartitionSchemeData *PartitionScheme;
  *		part_scheme - Partitioning scheme of the relation
  *		nparts - Number of partitions
  *		boundinfo - Partition bounds
+ *		merged - true if partition bounds are merged ones
  *		partition_qual - Partition constraint if not the root
  *		part_rels - RelOptInfos for each partition
+ *		all_partrels - Relids set of all partition relids
  *		partexprs, nullable_partexprs - Partition key expressions
  *		partitioned_child_rels - RT indexes of unpruned partitions of
  *								 this relation that are partitioned tables
@@ -720,9 +722,12 @@ typedef struct RelOptInfo
 	PartitionScheme part_scheme;	/* Partitioning scheme. */
 	int			nparts;			/* number of partitions */
 	struct PartitionBoundInfoData *boundinfo;	/* Partition bounds */
+	bool		merged;			/* true if partition bounds were created by
+								 * partition_bounds_merge() */
 	List	   *partition_qual; /* partition constraint */
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of partitions,
 									 * stored in the same order of bounds */
+	Relids		all_partrels;	/* Relids set of all partition relids */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 	List	   *partitioned_child_rels; /* List of RT indexes. */
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index be940b5192..5442e843af 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -16,6 +16,7 @@
 #include "nodes/pg_list.h"
 #include "partitioning/partdefs.h"
 #include "utils/relcache.h"
+struct RelOptInfo;				/* avoid including pathnodes.h here */
 
 
 /*
@@ -108,5 +109,11 @@ extern int	partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 										  int nvalues, Datum *values, bool *is_equal);
 extern int	partition_hash_bsearch(PartitionBoundInfo boundinfo,
 								   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   struct RelOptInfo *outer_rel, struct RelOptInfo *inner_rel,
+					   JoinType jointype,
+					   List **outer_parts, List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index b3fbe47bde..924cb99863 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -2055,30 +2055,2116 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
                ->  Seq Scan on prt1_n_p2 t1_2
 (10 rows)
 
--- partitionwise join can not be applied if only one of joining tables has
--- default partition
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
-ANALYZE prt2;
+--
+-- Test advanced partition-matching algorithm for partitioned join
+--
+-- Tests for range-partitioned tables
+CREATE TABLE prt1_ad (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_ad_p1 PARTITION OF prt1_ad FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_ad_p2 PARTITION OF prt1_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_ad_p3 PARTITION OF prt1_ad FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_ad_a_idx on prt1_ad (a);
+INSERT INTO prt1_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_ad;
+CREATE TABLE prt2_ad (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_ad_p1 PARTITION OF prt2_ad FOR VALUES FROM (100) TO (150);
+CREATE TABLE prt2_ad_p2 PARTITION OF prt2_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt2_ad_p3 PARTITION OF prt2_ad FOR VALUES FROM (350) TO (500);
+CREATE INDEX prt2_ad_b_idx on prt2_ad (b);
+INSERT INTO prt2_ad_p1 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 149) i;
+INSERT INTO prt2_ad_p2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(200, 299) i;
+INSERT INTO prt2_ad_p3 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_ad;
+-- inner join
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                     QUERY PLAN                      
+-----------------------------------------------------
  Sort
    Sort Key: t1.a
-   ->  Hash Join
-         Hash Cond: (t2.b = t1.a)
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_ad_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_ad_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p2 t2_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_ad_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 100 | 0 | 0100
+ 125 | 0 | 0125
+ 200 | 0 | 0200
+ 225 | 0 | 0225
+ 250 | 0 | 0250
+ 275 | 0 | 0275
+ 350 | 0 | 0350
+ 375 | 0 | 0375
+(8 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 |     | 
+ 175 | 0175 |     | 
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 300 | 0300 |     | 
+ 325 | 0325 |     | 
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(12 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+                  QUERY PLAN                   
+-----------------------------------------------
+ Append
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.a = t2_1.b)
+         ->  Seq Scan on prt1_ad_p1 t1_1
+               Filter: (b = 0)
+         ->  Hash
+               ->  Seq Scan on prt2_ad_p1 t2_1
+   ->  Hash Anti Join
+         Hash Cond: (t1_2.a = t2_2.b)
+         ->  Seq Scan on prt1_ad_p2 t1_2
+               Filter: (b = 0)
+         ->  Hash
+               ->  Seq Scan on prt2_ad_p2 t2_2
+   ->  Hash Anti Join
+         Hash Cond: (t1_3.a = t2_3.b)
+         ->  Seq Scan on prt1_ad_p3 t1_3
+               Filter: (b = 0)
+         ->  Hash
+               ->  Seq Scan on prt2_ad_p3 t2_3
+(19 rows)
+
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+  a  | b |  c   
+-----+---+------
+ 150 | 0 | 0150
+ 175 | 0 | 0175
+ 300 | 0 | 0300
+ 325 | 0 | 0325
+(4 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Sort
+   Sort Key: prt1_ad.a, prt2_ad.b
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_ad_1.a = prt2_ad_1.b)
+               Filter: (((175) = prt1_ad_1.a) OR ((425) = prt2_ad_1.b))
+               ->  Seq Scan on prt1_ad_p1 prt1_ad_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p1 prt2_ad_1
+                           Filter: (a = 0)
+         ->  Hash Full Join
+               Hash Cond: (prt1_ad_2.a = prt2_ad_2.b)
+               Filter: (((175) = prt1_ad_2.a) OR ((425) = prt2_ad_2.b))
+               ->  Seq Scan on prt1_ad_p2 prt1_ad_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p2 prt2_ad_2
+                           Filter: (a = 0)
+         ->  Hash Full Join
+               Hash Cond: (prt2_ad_3.b = prt1_ad_3.a)
+               Filter: (((175) = prt1_ad_3.a) OR ((425) = prt2_ad_3.b))
+               ->  Seq Scan on prt2_ad_p3 prt2_ad_3
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p3 prt1_ad_3
+                           Filter: (b = 0)
+(27 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 175 | 0175 |     | 
+     |      | 425 | 0425
+(2 rows)
+
+-- Test cases where one side has an extra partition
+-- Add an extra partition to prt2_ad
+CREATE TABLE prt2_ad_extra PARTITION OF prt2_ad FOR VALUES FROM (500) TO (MAXVALUE);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(500, 599) i;
+ANALYZE prt2_ad;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_ad_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_ad_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p2 t2_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_ad_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 100 | 0 | 0100
+ 125 | 0 | 0125
+ 200 | 0 | 0200
+ 225 | 0 | 0225
+ 250 | 0 | 0250
+ 275 | 0 | 0275
+ 350 | 0 | 0350
+ 375 | 0 | 0375
+(8 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 |     | 
+ 175 | 0175 |     | 
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 300 | 0300 |     | 
+ 325 | 0325 |     | 
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(12 rows)
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c FROM prt2_ad t1 LEFT JOIN prt1_ad t2 ON (t1.b = t2.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t2.a
+   ->  Hash Right Join
+         Hash Cond: (t2.a = t1.b)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2_1
-               ->  Seq Scan on prt2_p2 t2_2
-               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt1_ad_p1 t2_1
+               ->  Seq Scan on prt1_ad_p2 t2_2
+               ->  Seq Scan on prt1_ad_p3 t2_3
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt2_ad_p1 t1_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_ad_p2 t1_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_ad_p3 t1_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_ad_extra t1_4
+                           Filter: (a = 0)
+(18 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+                  QUERY PLAN                   
+-----------------------------------------------
+ Append
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.a = t2_1.b)
+         ->  Seq Scan on prt1_ad_p1 t1_1
+               Filter: (b = 0)
+         ->  Hash
+               ->  Seq Scan on prt2_ad_p1 t2_1
+   ->  Hash Anti Join
+         Hash Cond: (t1_2.a = t2_2.b)
+         ->  Seq Scan on prt1_ad_p2 t1_2
+               Filter: (b = 0)
+         ->  Hash
+               ->  Seq Scan on prt2_ad_p2 t2_2
+   ->  Hash Anti Join
+         Hash Cond: (t1_3.a = t2_3.b)
+         ->  Seq Scan on prt1_ad_p3 t1_3
+               Filter: (b = 0)
+         ->  Hash
+               ->  Seq Scan on prt2_ad_p3 t2_3
+(19 rows)
+
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+  a  | b |  c   
+-----+---+------
+ 150 | 0 | 0150
+ 175 | 0 | 0175
+ 300 | 0 | 0300
+ 325 | 0 | 0325
+(4 rows)
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt1_ad t2 WHERE t1.b = t2.a) AND t1.a = 0;
+                  QUERY PLAN                   
+-----------------------------------------------
+ Hash Anti Join
+   Hash Cond: (t1.b = t2.a)
+   ->  Append
+         ->  Seq Scan on prt2_ad_p1 t1_1
+               Filter: (a = 0)
+         ->  Seq Scan on prt2_ad_p2 t1_2
+               Filter: (a = 0)
+         ->  Seq Scan on prt2_ad_p3 t1_3
+               Filter: (a = 0)
+         ->  Seq Scan on prt2_ad_extra t1_4
+               Filter: (a = 0)
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_ad_p1 t2_1
+               ->  Seq Scan on prt1_ad_p2 t2_2
+               ->  Seq Scan on prt1_ad_p3 t2_3
+(16 rows)
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: prt1_ad.a, prt2_ad.b
+   ->  Hash Full Join
+         Hash Cond: (prt2_ad.b = prt1_ad.a)
+         Filter: (((175) = prt1_ad.a) OR ((425) = prt2_ad.b))
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 prt2_ad_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_ad_p2 prt2_ad_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_ad_p3 prt2_ad_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_ad_extra prt2_ad_4
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_ad_p1 prt1_ad_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_ad_p2 prt1_ad_2
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_ad_p3 prt1_ad_3
                            Filter: (b = 0)
-(16 rows)
+(22 rows)
+
+-- 3-way join where not every pair of relations can do partitioned join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1_ad t1 RIGHT JOIN prt2_ad t2 ON (t1.a = t2.b) INNER JOIN prt1_ad t3 ON (t2.b = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.a, t3.c;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a, t3.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: (t1_1.a = t3_1.a)
+               ->  Hash Join
+                     Hash Cond: (t2_1.b = t1_1.a)
+                     ->  Seq Scan on prt2_ad_p1 t2_1
+                     ->  Hash
+                           ->  Seq Scan on prt1_ad_p1 t1_1
+                                 Filter: (b = 0)
+               ->  Index Scan using prt1_ad_p1_a_idx on prt1_ad_p1 t3_1
+                     Index Cond: (a = t2_1.b)
+         ->  Nested Loop
+               Join Filter: (t1_2.a = t3_2.a)
+               ->  Hash Join
+                     Hash Cond: (t2_2.b = t1_2.a)
+                     ->  Seq Scan on prt2_ad_p2 t2_2
+                     ->  Hash
+                           ->  Seq Scan on prt1_ad_p2 t1_2
+                                 Filter: (b = 0)
+               ->  Index Scan using prt1_ad_p2_a_idx on prt1_ad_p2 t3_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t2_3.b)
+               ->  Hash Join
+                     Hash Cond: (t3_3.a = t1_3.a)
+                     ->  Seq Scan on prt1_ad_p3 t3_3
+                     ->  Hash
+                           ->  Seq Scan on prt1_ad_p3 t1_3
+                                 Filter: (b = 0)
+               ->  Index Scan using prt2_ad_p3_b_idx on prt2_ad_p3 t2_3
+                     Index Cond: (b = t3_3.a)
+(33 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1_ad t1 RIGHT JOIN prt2_ad t2 ON (t1.a = t2.b) INNER JOIN prt1_ad t3 ON (t2.b = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.a, t3.c;
+  a  | a |  c   
+-----+---+------
+ 100 | 0 | 0100
+ 125 | 0 | 0125
+ 200 | 0 | 0200
+ 225 | 0 | 0225
+ 250 | 0 | 0250
+ 275 | 0 | 0275
+ 350 | 0 | 0350
+ 375 | 0 | 0375
+(8 rows)
+
+DROP TABLE prt2_ad_extra;
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in those cases
+-- Split prt2_ad_p3 into two partitions so that prt1_ad_p3 matches both
+-- partitions
+ALTER TABLE prt2_ad DETACH PARTITION prt2_ad_p3;
+CREATE TABLE prt2_ad_p3_350_375 PARTITION OF prt2_ad FOR VALUES FROM (350) TO (375);
+CREATE TABLE prt2_ad_p3_375_500 PARTITION OF prt2_ad FOR VALUES FROM (375) TO (500);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_ad;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Seq Scan on prt2_ad_p3_350_375 t2_3
+               ->  Seq Scan on prt2_ad_p3_375_500 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p3 t1_3
+                           Filter: (b = 0)
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                          
+-------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_ad_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_ad_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_ad_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_ad_p1 t2_1
+                     ->  Seq Scan on prt2_ad_p2 t2_2
+                     ->  Seq Scan on prt2_ad_p3_350_375 t2_3
+                     ->  Seq Scan on prt2_ad_p3_375_500 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Seq Scan on prt2_ad_p3_350_375 t2_3
+               ->  Seq Scan on prt2_ad_p3_375_500 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p3 t1_3
+                           Filter: (b = 0)
+(17 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Hash Anti Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_ad_p1 t1_1
+               Filter: (b = 0)
+         ->  Seq Scan on prt1_ad_p2 t1_2
+               Filter: (b = 0)
+         ->  Seq Scan on prt1_ad_p3 t1_3
+               Filter: (b = 0)
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Seq Scan on prt2_ad_p3_350_375 t2_3
+               ->  Seq Scan on prt2_ad_p3_375_500 t2_4
+(15 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: prt1_ad.a, prt2_ad.b
+   ->  Hash Full Join
+         Hash Cond: (prt2_ad.b = prt1_ad.a)
+         Filter: (((175) = prt1_ad.a) OR ((425) = prt2_ad.b))
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 prt2_ad_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_ad_p2 prt2_ad_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_ad_p3_350_375 prt2_ad_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_ad_p3_375_500 prt2_ad_4
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_ad_p1 prt1_ad_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p2 prt1_ad_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p3 prt1_ad_3
+                           Filter: (b = 0)
+(22 rows)
+
+DROP TABLE prt2_ad_p3_350_375;
+DROP TABLE prt2_ad_p3_375_500;
+-- Test default partitions
+-- Change prt1_ad_p1 to the default partition
+ALTER TABLE prt1_ad DETACH PARTITION prt1_ad_p1;
+ALTER TABLE prt1_ad ATTACH PARTITION prt1_ad_p1 DEFAULT;
+ANALYZE prt1_ad;
+CREATE TABLE prt2_ad_p3_300_400 PARTITION OF prt2_ad FOR VALUES FROM (300) TO (400);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(300, 399) i;
+ANALYZE prt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_3.a)
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p1 t1_3
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p2 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_3.b = t1_2.a)
+               ->  Seq Scan on prt2_ad_p3_300_400 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p3 t1_2
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 300 | 0300 | 300 | 0300
+ 325 | 0325 | 325 | 0325
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(10 rows)
+
+DROP TABLE prt2_ad_p3_300_400;
+-- Restore prt2_ad_p3
+ALTER TABLE prt2_ad ATTACH PARTITION prt2_ad_p3 FOR VALUES FROM (350) TO (500);
+ANALYZE prt2_ad;
+-- Partitioned join can't be applied because the default partition of prt1_ad
+-- prt1_ad_p1 matches prt2_ad_p1 and prt2_ad_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Seq Scan on prt2_ad_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_ad_p2 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p3 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p1 t1_3
+                           Filter: (b = 0)
+(16 rows)
+
+-- Change prt2_ad_p3 to the default partition
+ALTER TABLE prt2_ad DETACH PARTITION prt2_ad_p3;
+ALTER TABLE prt2_ad ATTACH PARTITION prt2_ad_p3 DEFAULT;
+ANALYZE prt2_ad;
+-- Partitioned join can't be applied because the default partition of prt1_ad
+-- prt1_ad_p1 matches prt2_ad_p1 and prt2_ad_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Seq Scan on prt2_ad_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_ad_p2 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p3 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p1 t1_3
+                           Filter: (b = 0)
+(16 rows)
+
+DROP TABLE prt1_ad_p3;
+ANALYZE prt1_ad;
+DROP TABLE prt2_ad_p3;
+ANALYZE prt2_ad;
+CREATE TABLE prt3_ad (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt3_ad_p1 PARTITION OF prt3_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt3_ad_p2 PARTITION OF prt3_ad FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt3_ad_a_idx on prt3_ad (a);
+INSERT INTO prt3_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(200, 399) i;
+ANALYZE prt3_ad;
+-- 3-way join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) LEFT JOIN prt3_ad t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+                           QUERY PLAN                            
+-----------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b, t3.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t3_1.a = t1_1.a)
+               ->  Seq Scan on prt3_ad_p1 t3_1
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_2.b = t1_1.a)
+                           ->  Seq Scan on prt2_ad_p2 t2_2
+                           ->  Hash
+                                 ->  Seq Scan on prt1_ad_p2 t1_1
+                                       Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t3_2.a = t1_2.a)
+               ->  Seq Scan on prt3_ad_p2 t3_2
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_1.b = t1_2.a)
+                           ->  Seq Scan on prt2_ad_p1 t2_1
+                           ->  Hash
+                                 ->  Seq Scan on prt1_ad_p1 t1_2
+                                       Filter: (b = 0)
+(23 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) LEFT JOIN prt3_ad t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+  a  |  c   |  b  |  c   |  a  |  c   
+-----+------+-----+------+-----+------
+ 100 | 0100 | 100 | 0100 |     | 
+ 125 | 0125 | 125 | 0125 |     | 
+ 150 | 0150 |     |      |     | 
+ 175 | 0175 |     |      |     | 
+ 200 | 0200 | 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275 | 275 | 0275
+(8 rows)
+
+-- Tests for list-partitioned tables
+CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0001', '0003');
+CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt1_ad_p3 PARTITION OF plt1_ad FOR VALUES IN ('0008', '0009');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 NOT IN (0, 2, 5, 7);
+ANALYZE plt1_ad;
+CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt2_ad_p3 PARTITION OF plt2_ad FOR VALUES IN ('0007', '0009');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 NOT IN (0, 1, 5, 8);
+ANALYZE plt2_ad;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_ad_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+(6 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_ad_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 1 | 1 | 0001
+ 8 | 8 | 0008
+(2 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p3 t2_3
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+   |      | 2 | 0002
+   |      | 7 | 0007
+(8 rows)
+
+-- Test cases where one side has an extra partition
+-- Add an extra partition to plt2_ad
+CREATE TABLE plt2_ad_extra PARTITION OF plt2_ad FOR VALUES IN ('0000');
+INSERT INTO plt2_ad_extra VALUES (0, 0, '0000');
+ANALYZE plt2_ad;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_ad_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+(6 rows)
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2_ad t1 LEFT JOIN plt1_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt1_ad_p1 t2_1
+               ->  Seq Scan on plt1_ad_p2 t2_2
+               ->  Seq Scan on plt1_ad_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_ad_extra t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_ad_p1 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_ad_p2 t1_3
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_ad_p3 t1_4
+                           Filter: (b < 10)
+(18 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_ad_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 1 | 1 | 0001
+ 8 | 8 | 0008
+(2 rows)
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt2_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt1_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Anti Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt2_ad_extra t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p1 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3 t1_4
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p1 t2_1
+                     ->  Seq Scan on plt1_ad_p2 t2_2
+                     ->  Seq Scan on plt1_ad_p3 t2_3
+(18 rows)
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt2_ad_extra t2_1
+               ->  Seq Scan on plt2_ad_p1 t2_2
+               ->  Seq Scan on plt2_ad_p2 t2_3
+               ->  Seq Scan on plt2_ad_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+(15 rows)
+
+DROP TABLE plt2_ad_extra;
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in those cases
+-- Split plt2_ad_p2 into two partitions so that prt1_ad_p2 matches both
+-- partitions
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2;
+CREATE TABLE plt2_ad_p2_0004 PARTITION OF plt2_ad FOR VALUES IN ('0004');
+CREATE TABLE plt2_ad_p2_0006 PARTITION OF plt2_ad FOR VALUES IN ('0006');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6);
+ANALYZE plt2_ad;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Seq Scan on plt2_ad_p2_0004 t2_2
+               ->  Seq Scan on plt2_ad_p2_0006 t2_3
+               ->  Seq Scan on plt2_ad_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                        QUERY PLAN                        
+----------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Semi Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt1_ad_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+                     ->  Seq Scan on plt2_ad_p2_0004 t2_2
+                     ->  Seq Scan on plt2_ad_p2_0006 t2_3
+                     ->  Seq Scan on plt2_ad_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Seq Scan on plt2_ad_p2_0004 t2_2
+               ->  Seq Scan on plt2_ad_p2_0006 t2_3
+               ->  Seq Scan on plt2_ad_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(17 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                        QUERY PLAN                        
+----------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Anti Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt1_ad_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+                     ->  Seq Scan on plt2_ad_p2_0004 t2_2
+                     ->  Seq Scan on plt2_ad_p2_0006 t2_3
+                     ->  Seq Scan on plt2_ad_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Seq Scan on plt2_ad_p2_0004 t2_2
+               ->  Seq Scan on plt2_ad_p2_0006 t2_3
+               ->  Seq Scan on plt2_ad_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+(15 rows)
+
+DROP TABLE plt2_ad_p2_0004;
+DROP TABLE plt2_ad_p2_0006;
+-- Restore plt2_ad_p2
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_ad;
+-- Test NULL partitions
+-- Change plt1_ad_p1 to the NULL partition
+ALTER TABLE plt1_ad DETACH PARTITION plt1_ad_p1;
+CREATE TABLE plt1_ad_p1_null PARTITION OF plt1_ad FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt1_ad VALUES (-1, -1, NULL);
+ANALYZE plt1_ad;
+-- Change plt2_ad_p3 to the NULL partition
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p3;
+CREATE TABLE plt2_ad_p3_null PARTITION OF plt2_ad FOR VALUES IN (NULL, '0007', '0009');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (7, 9);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1_null t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_ad_p1_null t1_1
+                     Filter: (b < 10)
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3_null t2_3
+(19 rows)
+
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1_null t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a  |  c   | a |  c   
+----+------+---+------
+ -1 |      |   | 
+  1 | 0001 |   | 
+  3 | 0003 | 3 | 0003
+  4 | 0004 | 4 | 0004
+  6 | 0006 | 6 | 0006
+  8 | 0008 |   | 
+  9 | 0009 | 9 | 0009
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_ad_p1_null t1_1
+                     Filter: (b < 10)
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3_null t2_3
+(19 rows)
+
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a  | b  |  c   
+----+----+------
+ -1 | -1 | 
+  1 |  1 | 0001
+  8 |  8 | 0008
+(3 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p1_null t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt2_ad_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+ -1 |      |    | 
+  1 | 0001 |    | 
+  3 | 0003 |  3 | 0003
+  4 | 0004 |  4 | 0004
+  6 | 0006 |  6 | 0006
+  8 | 0008 |    | 
+  9 | 0009 |  9 | 0009
+    |      | -1 | 
+    |      |  2 | 0002
+    |      |  7 | 0007
+(10 rows)
+
+DROP TABLE plt1_ad_p1_null;
+DROP TABLE plt2_ad_p3_null;
+-- Restore plt1_ad_p1
+ALTER TABLE plt1_ad ATTACH PARTITION plt1_ad_p1 FOR VALUES IN ('0001', '0003');
+-- Add to plt1_ad the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt1_ad_extra PARTITION OF plt1_ad FOR VALUES IN (NULL);
+INSERT INTO plt1_ad VALUES (-1, -1, NULL);
+ANALYZE plt1_ad;
+-- Restore plt2_ad_p3
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p3 FOR VALUES IN ('0007', '0009');
+ANALYZE plt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- Partitioned join can't be applied because there isn't any partition on the
+-- plt2_ad side that matches the NULL partition of plt1_ad plt1_ad_extra
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Seq Scan on plt2_ad_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_extra t1_4
+                           Filter: (b < 10)
+(18 rows)
+
+-- Partitioned join can't be applied because there isn't any partition on the
+-- plt2_ad side that matches the NULL partition of plt1_ad plt1_ad_extra
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt1_ad_p1 t1_1
+               ->  Seq Scan on plt1_ad_p2 t1_2
+               ->  Seq Scan on plt1_ad_p3 t1_3
+               ->  Seq Scan on plt1_ad_extra t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+                     ->  Seq Scan on plt2_ad_p2 t2_2
+                     ->  Seq Scan on plt2_ad_p3 t2_3
+(15 rows)
+
+-- Add to plt2_ad the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt2_ad_extra PARTITION OF plt2_ad FOR VALUES IN (NULL);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+         ->  Nested Loop Left Join
+               Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+               ->  Seq Scan on plt1_ad_extra t1_4
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_extra t2_4
+(26 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a  |  c   | a |  c   
+----+------+---+------
+ -1 |      |   | 
+  1 | 0001 |   | 
+  3 | 0003 | 3 | 0003
+  4 | 0004 | 4 | 0004
+  6 | 0006 | 6 | 0006
+  8 | 0008 |   | 
+  9 | 0009 | 9 | 0009
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p3 t2_3
+         ->  Hash Full Join
+               Hash Cond: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+               Filter: ((COALESCE(t1_4.b, 0) < 10) AND (COALESCE(t2_4.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_extra t1_4
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_extra t2_4
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+ -1 |      |    | 
+  1 | 0001 |    | 
+  3 | 0003 |  3 | 0003
+  4 | 0004 |  4 | 0004
+  6 | 0006 |  6 | 0006
+  8 | 0008 |    | 
+  9 | 0009 |  9 | 0009
+    |      | -1 | 
+    |      |  2 | 0002
+    |      |  7 | 0007
+(10 rows)
+
+DROP TABLE plt1_ad_extra;
+DROP TABLE plt2_ad_extra;
+-- Test default partitions
+-- Change plt1_ad_p1 to the default partition
+ALTER TABLE plt1_ad DETACH PARTITION plt1_ad_p1;
+ALTER TABLE plt1_ad ATTACH PARTITION plt1_ad_p1 DEFAULT;
+DROP TABLE plt1_ad_p3;
+ANALYZE plt1_ad;
+DROP TABLE plt2_ad_p3;
+ANALYZE plt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_1
+                           Filter: (b < 10)
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(3 rows)
+
+-- Change plt2_ad_p2 to contain '0005' in addition to '0004' and '0006' as
+-- the key values
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2;
+CREATE TABLE plt2_ad_p2_ext PARTITION OF plt2_ad FOR VALUES IN ('0004', '0005', '0006');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 5, 6);
+ANALYZE plt2_ad;
+-- Partitioned join can't be applied because the default partition of plt1_ad
+-- plt1_ad_p1 matches plt2_ad_p1 and plt2_ad_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Seq Scan on plt2_ad_p2_ext t2_2
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p2 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p1 t1_2
+                           Filter: (b < 10)
+(13 rows)
+
+-- Change plt2_ad_p2_ext to the default partition
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2_ext;
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2_ext DEFAULT;
+ANALYZE plt2_ad;
+-- Partitioned join can't be applied because the default partition of plt1_ad
+-- plt1_ad_p1 matches plt2_ad_p1 and plt2_ad_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Seq Scan on plt2_ad_p2_ext t2_2
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p2 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p1 t1_2
+                           Filter: (b < 10)
+(13 rows)
+
+DROP TABLE plt2_ad_p2_ext;
+-- Restore plt2_ad_p2
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_ad;
+CREATE TABLE plt3_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_ad_p1 PARTITION OF plt3_ad FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt3_ad_p2 PARTITION OF plt3_ad FOR VALUES IN ('0007', '0009');
+INSERT INTO plt3_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6, 7, 9);
+ANALYZE plt3_ad;
+-- 3-way join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_ad t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t3_1.a = t1_1.a) AND (t3_1.c = t1_1.c))
+               ->  Seq Scan on plt3_ad_p1 t3_1
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+                           ->  Seq Scan on plt2_ad_p2 t2_2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_ad_p2 t1_1
+                                       Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t3_2.a = t1_2.a) AND (t3_2.c = t1_2.c))
+               ->  Seq Scan on plt3_ad_p2 t3_2
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+                           ->  Seq Scan on plt2_ad_p1 t2_1
+                           ->  Hash
+                                 ->  Seq Scan on plt1_ad_p1 t1_2
+                                       Filter: (b < 10)
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_ad t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   | a |  c   
+---+------+---+------+---+------
+ 1 | 0001 |   |      |   | 
+ 3 | 0003 | 3 | 0003 |   | 
+ 4 | 0004 | 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006 | 6 | 0006
+(4 rows)
+
+-- Test cases where one side has the default partition while the other side
+-- has the NULL partition
+-- Change plt2_ad_p1 to the NULL partition
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p1;
+CREATE TABLE plt2_ad_p1_null PARTITION OF plt2_ad FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p1_null t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_1
+                           Filter: (b < 10)
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 | 1 | 0001
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(4 rows)
+
+-- Change the NULL partition to contain only NULL values as the key values
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p1_null;
+CREATE TABLE plt2_ad_p1_nullonly PARTITION OF plt2_ad FOR VALUES IN (NULL);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Seq Scan on plt2_ad_p2 t2
+         ->  Hash
+               ->  Seq Scan on plt1_ad_p2 t1
+                     Filter: (b < 10)
+(8 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(2 rows)
+
+DROP TABLE plt2_ad_p1_nullonly;
+DROP TABLE plt2_ad_p1_null;
+DROP TABLE plt2_ad_p1;
+DROP TABLE plt1_ad;
+DROP TABLE plt2_ad;
+DROP TABLE plt3_ad;
+CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1_ad;
+CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0002');
+CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2_ad;
+CREATE TABLE plt3_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_ad_p1 PARTITION OF plt3_ad FOR VALUES IN ('0001');
+CREATE TABLE plt3_ad_p2 PARTITION OF plt3_ad FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.c = t2.c)) FULL JOIN plt3_ad t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2.a, t3.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t3_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_1.c = t2_1.c)
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                     ->  Hash
+                           ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt3_ad_p1 t3_1
+         ->  Hash Full Join
+               Hash Cond: (t1_2.c = t3_2.c)
+               Filter: (((COALESCE(t1_2.a, 0) % 5) <> 3) AND ((COALESCE(t1_2.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_2.c = t2_2.c)
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                     ->  Hash
+                           ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt3_ad_p2 t3_2
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.c = t2.c)) FULL JOIN plt3_ad t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+ a  |  c   | a  |  c   | a  |  c   
+----+------+----+------+----+------
+  0 | 0000 |    |      |    | 
+  5 | 0000 |    |      |    | 
+ 10 | 0000 |    |      |    | 
+ 15 | 0000 |    |      |    | 
+ 20 | 0000 |    |      |    | 
+  1 | 0001 |    |      |  1 | 0001
+  1 | 0001 |    |      |  6 | 0001
+  1 | 0001 |    |      | 11 | 0001
+  1 | 0001 |    |      | 16 | 0001
+  1 | 0001 |    |      | 21 | 0001
+  6 | 0001 |    |      |  1 | 0001
+  6 | 0001 |    |      |  6 | 0001
+  6 | 0001 |    |      | 11 | 0001
+  6 | 0001 |    |      | 16 | 0001
+  6 | 0001 |    |      | 21 | 0001
+ 11 | 0001 |    |      |  1 | 0001
+ 11 | 0001 |    |      |  6 | 0001
+ 11 | 0001 |    |      | 11 | 0001
+ 11 | 0001 |    |      | 16 | 0001
+ 11 | 0001 |    |      | 21 | 0001
+ 16 | 0001 |    |      |  1 | 0001
+ 16 | 0001 |    |      |  6 | 0001
+ 16 | 0001 |    |      | 11 | 0001
+ 16 | 0001 |    |      | 16 | 0001
+ 16 | 0001 |    |      | 21 | 0001
+ 21 | 0001 |    |      |  1 | 0001
+ 21 | 0001 |    |      |  6 | 0001
+ 21 | 0001 |    |      | 11 | 0001
+ 21 | 0001 |    |      | 16 | 0001
+ 21 | 0001 |    |      | 21 | 0001
+  2 | 0002 |  2 | 0002 |    | 
+  2 | 0002 |  7 | 0002 |    | 
+  2 | 0002 | 12 | 0002 |    | 
+  2 | 0002 | 17 | 0002 |    | 
+  2 | 0002 | 22 | 0002 |    | 
+  7 | 0002 |  2 | 0002 |    | 
+  7 | 0002 |  7 | 0002 |    | 
+  7 | 0002 | 12 | 0002 |    | 
+  7 | 0002 | 17 | 0002 |    | 
+  7 | 0002 | 22 | 0002 |    | 
+ 12 | 0002 |  2 | 0002 |    | 
+ 12 | 0002 |  7 | 0002 |    | 
+ 12 | 0002 | 12 | 0002 |    | 
+ 12 | 0002 | 17 | 0002 |    | 
+ 12 | 0002 | 22 | 0002 |    | 
+ 17 | 0002 |  2 | 0002 |    | 
+ 17 | 0002 |  7 | 0002 |    | 
+ 17 | 0002 | 12 | 0002 |    | 
+ 17 | 0002 | 17 | 0002 |    | 
+ 17 | 0002 | 22 | 0002 |    | 
+ 22 | 0002 |  2 | 0002 |    | 
+ 22 | 0002 |  7 | 0002 |    | 
+ 22 | 0002 | 12 | 0002 |    | 
+ 22 | 0002 | 17 | 0002 |    | 
+ 22 | 0002 | 22 | 0002 |    | 
+(55 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 575ba7b8d4..418ffa117f 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -445,11 +445,512 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
 
--- partitionwise join can not be applied if only one of joining tables has
--- default partition
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
-ANALYZE prt2;
 
+--
+-- Test advanced partition-matching algorithm for partitioned join
+--
+
+-- Tests for range-partitioned tables
+CREATE TABLE prt1_ad (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_ad_p1 PARTITION OF prt1_ad FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_ad_p2 PARTITION OF prt1_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_ad_p3 PARTITION OF prt1_ad FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_ad_a_idx on prt1_ad (a);
+INSERT INTO prt1_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_ad;
+
+CREATE TABLE prt2_ad (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_ad_p1 PARTITION OF prt2_ad FOR VALUES FROM (100) TO (150);
+CREATE TABLE prt2_ad_p2 PARTITION OF prt2_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt2_ad_p3 PARTITION OF prt2_ad FOR VALUES FROM (350) TO (500);
+CREATE INDEX prt2_ad_b_idx on prt2_ad (b);
+INSERT INTO prt2_ad_p1 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 149) i;
+INSERT INTO prt2_ad_p2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(200, 299) i;
+INSERT INTO prt2_ad_p3 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_ad;
+
+-- inner join
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+-- Test cases where one side has an extra partition
+
+-- Add an extra partition to prt2_ad
+CREATE TABLE prt2_ad_extra PARTITION OF prt2_ad FOR VALUES FROM (500) TO (MAXVALUE);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(500, 599) i;
+ANALYZE prt2_ad;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c FROM prt2_ad t1 LEFT JOIN prt1_ad t2 ON (t1.b = t2.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt1_ad t2 WHERE t1.b = t2.a) AND t1.a = 0;
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+-- 3-way join where not every pair of relations can do partitioned join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1_ad t1 RIGHT JOIN prt2_ad t2 ON (t1.a = t2.b) INNER JOIN prt1_ad t3 ON (t2.b = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1_ad t1 RIGHT JOIN prt2_ad t2 ON (t1.a = t2.b) INNER JOIN prt1_ad t3 ON (t2.b = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.a, t3.c;
+
+DROP TABLE prt2_ad_extra;
+
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in those cases
+
+-- Split prt2_ad_p3 into two partitions so that prt1_ad_p3 matches both
+-- partitions
+ALTER TABLE prt2_ad DETACH PARTITION prt2_ad_p3;
+CREATE TABLE prt2_ad_p3_350_375 PARTITION OF prt2_ad FOR VALUES FROM (350) TO (375);
+CREATE TABLE prt2_ad_p3_375_500 PARTITION OF prt2_ad FOR VALUES FROM (375) TO (500);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_ad;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+DROP TABLE prt2_ad_p3_350_375;
+DROP TABLE prt2_ad_p3_375_500;
+
+-- Test default partitions
+
+-- Change prt1_ad_p1 to the default partition
+ALTER TABLE prt1_ad DETACH PARTITION prt1_ad_p1;
+ALTER TABLE prt1_ad ATTACH PARTITION prt1_ad_p1 DEFAULT;
+ANALYZE prt1_ad;
+
+CREATE TABLE prt2_ad_p3_300_400 PARTITION OF prt2_ad FOR VALUES FROM (300) TO (400);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(300, 399) i;
+ANALYZE prt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt2_ad_p3_300_400;
+
+-- Restore prt2_ad_p3
+ALTER TABLE prt2_ad ATTACH PARTITION prt2_ad_p3 FOR VALUES FROM (350) TO (500);
+ANALYZE prt2_ad;
+
+-- Partitioned join can't be applied because the default partition of prt1_ad
+-- prt1_ad_p1 matches prt2_ad_p1 and prt2_ad_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- Change prt2_ad_p3 to the default partition
+ALTER TABLE prt2_ad DETACH PARTITION prt2_ad_p3;
+ALTER TABLE prt2_ad ATTACH PARTITION prt2_ad_p3 DEFAULT;
+ANALYZE prt2_ad;
+
+-- Partitioned join can't be applied because the default partition of prt1_ad
+-- prt1_ad_p1 matches prt2_ad_p1 and prt2_ad_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt1_ad_p3;
+ANALYZE prt1_ad;
+
+DROP TABLE prt2_ad_p3;
+ANALYZE prt2_ad;
+
+CREATE TABLE prt3_ad (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt3_ad_p1 PARTITION OF prt3_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt3_ad_p2 PARTITION OF prt3_ad FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt3_ad_a_idx on prt3_ad (a);
+INSERT INTO prt3_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(200, 399) i;
+ANALYZE prt3_ad;
+
+-- 3-way join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) LEFT JOIN prt3_ad t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) LEFT JOIN prt3_ad t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+
+
+-- Tests for list-partitioned tables
+CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0001', '0003');
+CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt1_ad_p3 PARTITION OF plt1_ad FOR VALUES IN ('0008', '0009');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 NOT IN (0, 2, 5, 7);
+ANALYZE plt1_ad;
+
+CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt2_ad_p3 PARTITION OF plt2_ad FOR VALUES IN ('0007', '0009');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 NOT IN (0, 1, 5, 8);
+ANALYZE plt2_ad;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+-- Test cases where one side has an extra partition
+
+-- Add an extra partition to plt2_ad
+CREATE TABLE plt2_ad_extra PARTITION OF plt2_ad FOR VALUES IN ('0000');
+INSERT INTO plt2_ad_extra VALUES (0, 0, '0000');
+ANALYZE plt2_ad;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2_ad t1 LEFT JOIN plt1_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt2_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt1_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt2_ad_extra;
+
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in those cases
+
+-- Split plt2_ad_p2 into two partitions so that prt1_ad_p2 matches both
+-- partitions
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2;
+CREATE TABLE plt2_ad_p2_0004 PARTITION OF plt2_ad FOR VALUES IN ('0004');
+CREATE TABLE plt2_ad_p2_0006 PARTITION OF plt2_ad FOR VALUES IN ('0006');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6);
+ANALYZE plt2_ad;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt2_ad_p2_0004;
+DROP TABLE plt2_ad_p2_0006;
+
+-- Restore plt2_ad_p2
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_ad;
+
+-- Test NULL partitions
+
+-- Change plt1_ad_p1 to the NULL partition
+ALTER TABLE plt1_ad DETACH PARTITION plt1_ad_p1;
+CREATE TABLE plt1_ad_p1_null PARTITION OF plt1_ad FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt1_ad VALUES (-1, -1, NULL);
+ANALYZE plt1_ad;
+
+-- Change plt2_ad_p3 to the NULL partition
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p3;
+CREATE TABLE plt2_ad_p3_null PARTITION OF plt2_ad FOR VALUES IN (NULL, '0007', '0009');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (7, 9);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt1_ad_p1_null;
+DROP TABLE plt2_ad_p3_null;
+
+-- Restore plt1_ad_p1
+ALTER TABLE plt1_ad ATTACH PARTITION plt1_ad_p1 FOR VALUES IN ('0001', '0003');
+
+-- Add to plt1_ad the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt1_ad_extra PARTITION OF plt1_ad FOR VALUES IN (NULL);
+INSERT INTO plt1_ad VALUES (-1, -1, NULL);
+ANALYZE plt1_ad;
+
+-- Restore plt2_ad_p3
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p3 FOR VALUES IN ('0007', '0009');
+ANALYZE plt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Partitioned join can't be applied because there isn't any partition on the
+-- plt2_ad side that matches the NULL partition of plt1_ad plt1_ad_extra
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Partitioned join can't be applied because there isn't any partition on the
+-- plt2_ad side that matches the NULL partition of plt1_ad plt1_ad_extra
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+-- Add to plt2_ad the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt2_ad_extra PARTITION OF plt2_ad FOR VALUES IN (NULL);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt1_ad_extra;
+DROP TABLE plt2_ad_extra;
+
+-- Test default partitions
+
+-- Change plt1_ad_p1 to the default partition
+ALTER TABLE plt1_ad DETACH PARTITION plt1_ad_p1;
+ALTER TABLE plt1_ad ATTACH PARTITION plt1_ad_p1 DEFAULT;
+DROP TABLE plt1_ad_p3;
+ANALYZE plt1_ad;
+
+DROP TABLE plt2_ad_p3;
+ANALYZE plt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Change plt2_ad_p2 to contain '0005' in addition to '0004' and '0006' as
+-- the key values
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2;
+CREATE TABLE plt2_ad_p2_ext PARTITION OF plt2_ad FOR VALUES IN ('0004', '0005', '0006');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 5, 6);
+ANALYZE plt2_ad;
+
+-- Partitioned join can't be applied because the default partition of plt1_ad
+-- plt1_ad_p1 matches plt2_ad_p1 and plt2_ad_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Change plt2_ad_p2_ext to the default partition
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2_ext;
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2_ext DEFAULT;
+ANALYZE plt2_ad;
+
+-- Partitioned join can't be applied because the default partition of plt1_ad
+-- plt1_ad_p1 matches plt2_ad_p1 and plt2_ad_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt2_ad_p2_ext;
+
+-- Restore plt2_ad_p2
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_ad;
+
+CREATE TABLE plt3_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_ad_p1 PARTITION OF plt3_ad FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt3_ad_p2 PARTITION OF plt3_ad FOR VALUES IN ('0007', '0009');
+INSERT INTO plt3_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6, 7, 9);
+ANALYZE plt3_ad;
+
+-- 3-way join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_ad t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_ad t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Test cases where one side has the default partition while the other side
+-- has the NULL partition
+
+-- Change plt2_ad_p1 to the NULL partition
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p1;
+CREATE TABLE plt2_ad_p1_null PARTITION OF plt2_ad FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Change the NULL partition to contain only NULL values as the key values
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p1_null;
+CREATE TABLE plt2_ad_p1_nullonly PARTITION OF plt2_ad FOR VALUES IN (NULL);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt2_ad_p1_nullonly;
+DROP TABLE plt2_ad_p1_null;
+DROP TABLE plt2_ad_p1;
+
+DROP TABLE plt1_ad;
+DROP TABLE plt2_ad;
+DROP TABLE plt3_ad;
+
+
+CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1_ad;
+
+CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0002');
+CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2_ad;
+
+CREATE TABLE plt3_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_ad_p1 PARTITION OF plt3_ad FOR VALUES IN ('0001');
+CREATE TABLE plt3_ad_p2 PARTITION OF plt3_ad FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.c = t2.c)) FULL JOIN plt3_ad t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.c = t2.c)) FULL JOIN plt3_ad t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
-- 
2.21.1 (Apple Git-122.3)

v31-0002-Extending-partition-join-tests.patchapplication/octet-stream; name=v31-0002-Extending-partition-join-tests.patch; x-unix-mode=0644Download
From 6201bbcfe36b1adcfa991b855fc6ab69e3fc7cb4 Mon Sep 17 00:00:00 2001
From: Mark Dilger <mark.dilger@enterprisedb.com>
Date: Mon, 27 Jan 2020 18:14:58 -0800
Subject: [PATCH 2/3] Extending partition join tests.
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit

Adding more tests of the partition-wise join logic.  The expected
output for the new tests is based on current behavior rather than
what I would necessarily like to see.  In particular, the EXPLAIN
output for the new tests shows more partitions being accessed than
seem strictly necessary to satisfy the queries being explained.

For the curious:

      query condition      =>   partition with matching data
      ---------------           ----------------------------
WHERE beta.a = 'äbç'       =>           beta_f
WHERE beta.a = 'ὀδυσσεύς'  =>           beta_default
WHERE alpha.a = 'äbç'      =>           alpha_e
WHERE alpha.a = 'ὀδυσσεύς' =>           alpha_default

Both tables alpha and beta have other partitions which contain
neither of these two strings.

When joining alpha with beta with the restriction

     alpha.a = beta.a and alpha.a = 'äbç'

the planner accesses all four of beta_f, beta_default, alpha_e,
and alpha_default, despite not needing anything from the default
partitions.  Interestingly, when querying with the restriction

     alpha.a = beta.a and alpha.a = 'ὀδυσσεύς'

the planner does the right thing and only checks the default
partitions.

When joining with

    alpha.a = beta.a and alpha.a IN ('äbç', 'ὀδυσσεύς')

the planner does the right thing for one side of the query, but
hits all partitions for the other side, which it doesn't need to
do.

The queries return the right results, so these critiques are
restricted to performance and not correctness.
---
 src/test/regress/expected/partition_join.out | 407 +++++++++++++++++++
 src/test/regress/sql/partition_join.sql      | 110 +++++
 2 files changed, 517 insertions(+)

diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 924cb99863..f20a85689f 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -4168,3 +4168,410 @@ SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2
  22 | 0002 | 22 | 0002 |    | 
 (55 rows)
 
+CREATE TABLE raw_data (a text);
+INSERT INTO raw_data (a) VALUES ('Türkiye'),
+								('TÜRKIYE'),
+								('bıt'),
+								('BIT'),
+								('äbç'),
+								('ÄBÇ'),
+								('aaá'),
+								('coté'),
+								('Götz'),
+								('ὀδυσσεύς'),
+								('ὈΔΥΣΣΕΎΣ'),
+								('を読み取り用'),
+								('にオープンできませんでした');
+CREATE TABLE alpha
+	(a TEXT, b TEXT)
+	PARTITION BY RANGE(a, b);
+CREATE TABLE alpha_a PARTITION OF alpha FOR VALUES FROM ('a','v') TO ('c','q');
+CREATE TABLE alpha_b PARTITION OF alpha FOR VALUES FROM ('c','q') TO ('d','f');
+CREATE TABLE alpha_c PARTITION OF alpha FOR VALUES FROM ('d','f') TO ('p','m');
+CREATE TABLE alpha_d PARTITION OF alpha FOR VALUES FROM ('p','m') TO ('z','z');
+CREATE TABLE alpha_e PARTITION OF alpha FOR VALUES FROM ('z','z') TO ('√','Σ');
+CREATE TABLE alpha_f PARTITION OF alpha FOR VALUES FROM ('√','Σ') TO ('き','ま');
+CREATE TABLE alpha_default PARTITION OF alpha DEFAULT;
+CREATE TABLE beta
+	(a TEXT, b TEXT)
+	PARTITION BY RANGE(a, b);
+CREATE TABLE beta_a PARTITION OF beta FOR VALUES FROM ('a','z') TO ('d','z');
+CREATE TABLE beta_b PARTITION OF beta FOR VALUES FROM ('d','z') TO ('g','z');
+CREATE TABLE beta_c PARTITION OF beta FOR VALUES FROM ('g','z') TO ('k','z');
+CREATE TABLE beta_d PARTITION OF beta FOR VALUES FROM ('k','z') TO ('o','z');
+CREATE TABLE beta_e PARTITION OF beta FOR VALUES FROM ('o','z') TO ('t','z');
+CREATE TABLE beta_f PARTITION OF beta FOR VALUES FROM ('t','z') TO ('Δ','υ');
+CREATE TABLE beta_g PARTITION OF beta FOR VALUES FROM ('Δ','υ') TO ('ὀ','√');
+CREATE TABLE beta_h PARTITION OF beta FOR VALUES FROM ('ὀ','√') TO ('ん', '用');
+CREATE TABLE beta_default PARTITION OF beta DEFAULT;
+INSERT INTO alpha (SELECT a, a FROM raw_data);
+INSERT INTO beta (SELECT a, a FROM raw_data);
+ANALYZE alpha;
+ANALYZE beta;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Hash Join
+   Hash Cond: (t2.a = t1.a)
+   ->  Append
+         ->  Seq Scan on beta_a t2_1
+         ->  Seq Scan on beta_b t2_2
+         ->  Seq Scan on beta_c t2_3
+         ->  Seq Scan on beta_d t2_4
+         ->  Seq Scan on beta_e t2_5
+         ->  Seq Scan on beta_f t2_6
+         ->  Seq Scan on beta_g t2_7
+         ->  Seq Scan on beta_h t2_8
+         ->  Seq Scan on beta_default t2_9
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on alpha_e t1_1
+                     Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+               ->  Seq Scan on alpha_default t1_2
+                     Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+(18 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+    a     |    a     
+----------+----------
+ äbç      | äbç
+ ὀδυσσεύς | ὀδυσσεύς
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'äbç';
+                   QUERY PLAN                    
+-------------------------------------------------
+ Nested Loop
+   ->  Append
+         ->  Seq Scan on alpha_e t1_1
+               Filter: (a = 'äbç'::text)
+         ->  Seq Scan on alpha_default t1_2
+               Filter: (a = 'äbç'::text)
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on beta_f t2_1
+                     Filter: (a = 'äbç'::text)
+               ->  Seq Scan on beta_default t2_2
+                     Filter: (a = 'äbç'::text)
+(12 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'äbç';
+  a  |  a  
+-----+-----
+ äbç | äbç
+(1 row)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
+               QUERY PLAN               
+----------------------------------------
+ Nested Loop
+   ->  Seq Scan on alpha_default t1
+         Filter: (a = 'ὀδυσσεύς'::text)
+   ->  Seq Scan on beta_default t2
+         Filter: (a = 'ὀδυσσεύς'::text)
+(5 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
+    a     |    a     
+----------+----------
+ ὀδυσσεύς | ὀδυσσεύς
+(1 row)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a IN ('äbç', 'ὀδυσσεύς');
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Hash Semi Join
+   Hash Cond: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on alpha_e t1_1
+               Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+         ->  Seq Scan on alpha_default t1_2
+               Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on beta_a t2_1
+               ->  Seq Scan on beta_b t2_2
+               ->  Seq Scan on beta_c t2_3
+               ->  Seq Scan on beta_d t2_4
+               ->  Seq Scan on beta_e t2_5
+               ->  Seq Scan on beta_f t2_6
+               ->  Seq Scan on beta_g t2_7
+               ->  Seq Scan on beta_h t2_8
+               ->  Seq Scan on beta_default t2_9
+(18 rows)
+
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a IN ('äbç', 'ὀδυσσεύς');
+    a     
+----------
+ äbç
+ ὀδυσσεύς
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'äbç';
+                   QUERY PLAN                    
+-------------------------------------------------
+ Nested Loop Semi Join
+   ->  Append
+         ->  Seq Scan on alpha_e t1_1
+               Filter: (a = 'äbç'::text)
+         ->  Seq Scan on alpha_default t1_2
+               Filter: (a = 'äbç'::text)
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on beta_f t2_1
+                     Filter: (a = 'äbç'::text)
+               ->  Seq Scan on beta_default t2_2
+                     Filter: (a = 'äbç'::text)
+(12 rows)
+
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'äbç';
+  a  
+-----
+ äbç
+(1 row)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'ὀδυσσεύς';
+               QUERY PLAN               
+----------------------------------------
+ Nested Loop Semi Join
+   ->  Seq Scan on alpha_default t1
+         Filter: (a = 'ὀδυσσεύς'::text)
+   ->  Seq Scan on beta_default t2
+         Filter: (a = 'ὀδυσσεύς'::text)
+(5 rows)
+
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'ὀδυσσεύς';
+    a     
+----------
+ ὀδυσσεύς
+(1 row)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.a = t1.a)
+   ->  Append
+         ->  Seq Scan on beta_a t2_1
+         ->  Seq Scan on beta_b t2_2
+         ->  Seq Scan on beta_c t2_3
+         ->  Seq Scan on beta_d t2_4
+         ->  Seq Scan on beta_e t2_5
+         ->  Seq Scan on beta_f t2_6
+         ->  Seq Scan on beta_g t2_7
+         ->  Seq Scan on beta_h t2_8
+         ->  Seq Scan on beta_default t2_9
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on alpha_e t1_1
+                     Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+               ->  Seq Scan on alpha_default t1_2
+                     Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+(18 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+    a     |    a     
+----------+----------
+ äbç      | äbç
+ ὀδυσσεύς | ὀδυσσεύς
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'äbç';
+                   QUERY PLAN                    
+-------------------------------------------------
+ Nested Loop Left Join
+   Join Filter: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on alpha_e t1_1
+               Filter: (a = 'äbç'::text)
+         ->  Seq Scan on alpha_default t1_2
+               Filter: (a = 'äbç'::text)
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on beta_f t2_1
+                     Filter: (a = 'äbç'::text)
+               ->  Seq Scan on beta_default t2_2
+                     Filter: (a = 'äbç'::text)
+(13 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'äbç';
+  a  |  a  
+-----+-----
+ äbç | äbç
+(1 row)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
+               QUERY PLAN               
+----------------------------------------
+ Nested Loop Left Join
+   Join Filter: (t1.a = t2.a)
+   ->  Seq Scan on alpha_default t1
+         Filter: (a = 'ὀδυσσεύς'::text)
+   ->  Seq Scan on beta_default t2
+         Filter: (a = 'ὀδυσσεύς'::text)
+(6 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
+    a     |    a     
+----------+----------
+ ὀδυσσεύς | ὀδυσσεύς
+(1 row)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a IN ('äbç', 'ὀδυσσεύς');
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Hash Anti Join
+   Hash Cond: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on alpha_e t1_1
+               Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+         ->  Seq Scan on alpha_default t1_2
+               Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on beta_a t2_1
+               ->  Seq Scan on beta_b t2_2
+               ->  Seq Scan on beta_c t2_3
+               ->  Seq Scan on beta_d t2_4
+               ->  Seq Scan on beta_e t2_5
+               ->  Seq Scan on beta_f t2_6
+               ->  Seq Scan on beta_g t2_7
+               ->  Seq Scan on beta_h t2_8
+               ->  Seq Scan on beta_default t2_9
+(18 rows)
+
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a IN ('äbç', 'ὀδυσσεύς');
+ a 
+---
+(0 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'äbç';
+                   QUERY PLAN                    
+-------------------------------------------------
+ Nested Loop Anti Join
+   Join Filter: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on alpha_e t1_1
+               Filter: (a = 'äbç'::text)
+         ->  Seq Scan on alpha_default t1_2
+               Filter: (a = 'äbç'::text)
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on beta_f t2_1
+                     Filter: (a = 'äbç'::text)
+               ->  Seq Scan on beta_default t2_2
+                     Filter: (a = 'äbç'::text)
+(13 rows)
+
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'äbç';
+ a 
+---
+(0 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'ὀδυσσεύς';
+               QUERY PLAN               
+----------------------------------------
+ Nested Loop Anti Join
+   Join Filter: (t1.a = t2.a)
+   ->  Seq Scan on alpha_default t1
+         Filter: (a = 'ὀδυσσεύς'::text)
+   ->  Seq Scan on beta_default t2
+         Filter: (a = 'ὀδυσσεύς'::text)
+(6 rows)
+
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'ὀδυσσεύς';
+ a 
+---
+(0 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.a = t1.a)
+   ->  Append
+         ->  Seq Scan on beta_a t2_1
+         ->  Seq Scan on beta_b t2_2
+         ->  Seq Scan on beta_c t2_3
+         ->  Seq Scan on beta_d t2_4
+         ->  Seq Scan on beta_e t2_5
+         ->  Seq Scan on beta_f t2_6
+         ->  Seq Scan on beta_g t2_7
+         ->  Seq Scan on beta_h t2_8
+         ->  Seq Scan on beta_default t2_9
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on alpha_e t1_1
+                     Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+               ->  Seq Scan on alpha_default t1_2
+                     Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+(18 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+    a     |    a     
+----------+----------
+ äbç      | äbç
+ ὀδυσσεύς | ὀδυσσεύς
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'äbç';
+                   QUERY PLAN                    
+-------------------------------------------------
+ Nested Loop Left Join
+   Join Filter: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on alpha_e t1_1
+               Filter: (a = 'äbç'::text)
+         ->  Seq Scan on alpha_default t1_2
+               Filter: (a = 'äbç'::text)
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on beta_f t2_1
+                     Filter: (a = 'äbç'::text)
+               ->  Seq Scan on beta_default t2_2
+                     Filter: (a = 'äbç'::text)
+(13 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'äbç';
+  a  |  a  
+-----+-----
+ äbç | äbç
+(1 row)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
+               QUERY PLAN               
+----------------------------------------
+ Nested Loop Left Join
+   Join Filter: (t1.a = t2.a)
+   ->  Seq Scan on alpha_default t1
+         Filter: (a = 'ὀδυσσεύς'::text)
+   ->  Seq Scan on beta_default t2
+         Filter: (a = 'ὀδυσσεύς'::text)
+(6 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
+    a     |    a     
+----------+----------
+ ὀδυσσεύς | ὀδυσσεύς
+(1 row)
+
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 418ffa117f..159d459831 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -954,3 +954,113 @@ ANALYZE plt3_ad;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.c = t2.c)) FULL JOIN plt3_ad t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
 SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.c = t2.c)) FULL JOIN plt3_ad t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+
+CREATE TABLE raw_data (a text);
+INSERT INTO raw_data (a) VALUES ('Türkiye'),
+								('TÜRKIYE'),
+								('bıt'),
+								('BIT'),
+								('äbç'),
+								('ÄBÇ'),
+								('aaá'),
+								('coté'),
+								('Götz'),
+								('ὀδυσσεύς'),
+								('ὈΔΥΣΣΕΎΣ'),
+								('を読み取り用'),
+								('にオープンできませんでした');
+
+CREATE TABLE alpha
+	(a TEXT, b TEXT)
+	PARTITION BY RANGE(a, b);
+CREATE TABLE alpha_a PARTITION OF alpha FOR VALUES FROM ('a','v') TO ('c','q');
+CREATE TABLE alpha_b PARTITION OF alpha FOR VALUES FROM ('c','q') TO ('d','f');
+CREATE TABLE alpha_c PARTITION OF alpha FOR VALUES FROM ('d','f') TO ('p','m');
+CREATE TABLE alpha_d PARTITION OF alpha FOR VALUES FROM ('p','m') TO ('z','z');
+CREATE TABLE alpha_e PARTITION OF alpha FOR VALUES FROM ('z','z') TO ('√','Σ');
+CREATE TABLE alpha_f PARTITION OF alpha FOR VALUES FROM ('√','Σ') TO ('き','ま');
+CREATE TABLE alpha_default PARTITION OF alpha DEFAULT;
+
+CREATE TABLE beta
+	(a TEXT, b TEXT)
+	PARTITION BY RANGE(a, b);
+CREATE TABLE beta_a PARTITION OF beta FOR VALUES FROM ('a','z') TO ('d','z');
+CREATE TABLE beta_b PARTITION OF beta FOR VALUES FROM ('d','z') TO ('g','z');
+CREATE TABLE beta_c PARTITION OF beta FOR VALUES FROM ('g','z') TO ('k','z');
+CREATE TABLE beta_d PARTITION OF beta FOR VALUES FROM ('k','z') TO ('o','z');
+CREATE TABLE beta_e PARTITION OF beta FOR VALUES FROM ('o','z') TO ('t','z');
+CREATE TABLE beta_f PARTITION OF beta FOR VALUES FROM ('t','z') TO ('Δ','υ');
+CREATE TABLE beta_g PARTITION OF beta FOR VALUES FROM ('Δ','υ') TO ('ὀ','√');
+CREATE TABLE beta_h PARTITION OF beta FOR VALUES FROM ('ὀ','√') TO ('ん', '用');
+CREATE TABLE beta_default PARTITION OF beta DEFAULT;
+
+INSERT INTO alpha (SELECT a, a FROM raw_data);
+INSERT INTO beta (SELECT a, a FROM raw_data);
+
+ANALYZE alpha;
+ANALYZE beta;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'äbç';
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'äbç';
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a IN ('äbç', 'ὀδυσσεύς');
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a IN ('äbç', 'ὀδυσσεύς');
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'äbç';
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'äbç';
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'ὀδυσσεύς';
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'ὀδυσσεύς';
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'äbç';
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'äbç';
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a IN ('äbç', 'ὀδυσσεύς');
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a IN ('äbç', 'ὀδυσσεύς');
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'äbç';
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'äbç';
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'ὀδυσσεύς';
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'ὀδυσσεύς';
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'äbç';
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'äbç';
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
-- 
2.21.1 (Apple Git-122.3)

v31-0003-Further-extending-partition-join-tests.patchapplication/octet-stream; name=v31-0003-Further-extending-partition-join-tests.patch; x-unix-mode=0644Download
From 3f16e328c1b334089a8521d2c7bb9a4f7ad4a50e Mon Sep 17 00:00:00 2001
From: Mark Dilger <mark.dilger@enterprisedb.com>
Date: Mon, 27 Jan 2020 20:09:49 -0800
Subject: [PATCH 3/3] Further extending partition join tests.

Adding more tests of the partition-wise join logic, this time
testing list partitioning rather than range partitioning.  Like
in the previous commit, the expected output for the new tests
is based on current behavior rather than what I would necessarily
like to see.  In particular, the EXPLAIN output for the new tests
shows more partitions being accessed than seem strictly necessary
to satisfy the queries being explained.

For example:

	EXPLAIN (COSTS OFF)
	SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IS NULL;
	                QUERY PLAN
	-------------------------------------------
	 Hash Join
	   Hash Cond: (t2.a = t1.a)
	   ->  Append
	         ->  Seq Scan on beta_d t2_1
	         ->  Seq Scan on beta_b t2_2
	         ->  Seq Scan on beta_a t2_3
	         ->  Seq Scan on beta_c t2_4
	         ->  Seq Scan on beta_default t2_5
	   ->  Hash
	         ->  Seq Scan on alpha_f t1
	               Filter: (a IS NULL)
	(11 rows)

It seems that if t1.a is null, and we join on t1.a = t2.a, then
the system should be able to work out that no rows will match, but
all partitions on t2 are scanned.

Similarly, given that t1 and t2 each only have one partition
for NULL values:

	EXPLAIN (COSTS OFF)
	SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a IS NOT DISTINCT FROM t2.a) WHERE t1.a IS NULL;
	                    QUERY PLAN
	---------------------------------------------------
	 Nested Loop
	   Join Filter: (NOT (t1.a IS DISTINCT FROM t2.a))
	   ->  Seq Scan on alpha_f t1
	         Filter: (a IS NULL)
	   ->  Append
	         ->  Seq Scan on beta_d t2_1
	         ->  Seq Scan on beta_b t2_2
	         ->  Seq Scan on beta_a t2_3
	         ->  Seq Scan on beta_c t2_4
	         ->  Seq Scan on beta_default t2_5
	(10 rows)

If t1.a is null, and t2.a is not distinct from t1.a, then t2.a must
be null, so scanning all partitions of t2 is unnecessary.
---
 src/test/regress/expected/partition_join.out | 631 +++++++++++++++++++
 src/test/regress/sql/partition_join.sql      | 143 +++++
 2 files changed, 774 insertions(+)

diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index f20a85689f..d7bbca98a1 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -4575,3 +4575,634 @@ SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a =
  ὀδυσσεύς | ὀδυσσεύς
 (1 row)
 
+-- Try again, this time with list partitioning
+DROP TABLE alpha CASCADE;
+DROP TABLE beta CASCADE;
+CREATE TABLE alpha (a TEXT) PARTITION BY LIST(a);
+CREATE TABLE alpha_a PARTITION OF alpha FOR VALUES IN ('Türkiye', 'TÜRKIYE');
+CREATE TABLE alpha_b PARTITION OF alpha FOR VALUES IN ('bıt', 'BIT');
+CREATE TABLE alpha_c PARTITION OF alpha FOR VALUES IN ('äbç', 'ÄBÇ');
+CREATE TABLE alpha_d PARTITION OF alpha FOR VALUES IN ('aaá', 'coté', 'Götz');
+CREATE TABLE alpha_e PARTITION OF alpha FOR VALUES IN ('ὀδυσσεύς', 'ὈΔΥΣΣΕΎΣ');
+CREATE TABLE alpha_f PARTITION OF alpha FOR VALUES IN ('を読み取り用', 'にオープンできませんでした', NULL);
+CREATE TABLE alpha_default PARTITION OF alpha DEFAULT;
+CREATE TABLE beta (a TEXT) PARTITION BY LIST(a);
+CREATE TABLE beta_a PARTITION OF beta FOR VALUES IN ('Türkiye', 'coté', 'ὈΔΥΣΣΕΎΣ');
+CREATE TABLE beta_b PARTITION OF beta FOR VALUES IN ('bıt', 'TÜRKIYE');
+CREATE TABLE beta_c PARTITION OF beta FOR VALUES IN ('äbç', 'を読み取り用', 'にオープンできませんでした');
+CREATE TABLE beta_d PARTITION OF beta FOR VALUES IN ('aaá', 'Götz', 'BIT', 'ὀδυσσεύς', 'ÄBÇ', NULL);
+CREATE TABLE beta_default PARTITION OF beta DEFAULT;
+INSERT INTO alpha (SELECT a FROM raw_data);
+INSERT INTO beta (SELECT a FROM raw_data);
+INSERT INTO alpha VALUES (null);
+INSERT INTO beta VALUES (null);
+INSERT INTO alpha VALUES ('grumble');
+INSERT INTO beta VALUES ('grumble');
+ANALYZE alpha;
+ANALYZE beta;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Hash Join
+   Hash Cond: (t2.a = t1.a)
+   ->  Append
+         ->  Seq Scan on beta_d t2_1
+         ->  Seq Scan on beta_b t2_2
+         ->  Seq Scan on beta_a t2_3
+         ->  Seq Scan on beta_c t2_4
+         ->  Seq Scan on beta_default t2_5
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on alpha_c t1_1
+                     Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+               ->  Seq Scan on alpha_e t1_2
+                     Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+(14 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+    a     |    a     
+----------+----------
+ ὀδυσσεύς | ὀδυσσεύς
+ äbç      | äbç
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'äbç';
+            QUERY PLAN             
+-----------------------------------
+ Nested Loop
+   ->  Seq Scan on alpha_c t1
+         Filter: (a = 'äbç'::text)
+   ->  Seq Scan on beta_c t2
+         Filter: (a = 'äbç'::text)
+(5 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'äbç';
+  a  |  a  
+-----+-----
+ äbç | äbç
+(1 row)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
+               QUERY PLAN               
+----------------------------------------
+ Nested Loop
+   ->  Seq Scan on alpha_e t1
+         Filter: (a = 'ὀδυσσεύς'::text)
+   ->  Seq Scan on beta_d t2
+         Filter: (a = 'ὀδυσσεύς'::text)
+(5 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
+    a     |    a     
+----------+----------
+ ὀδυσσεύς | ὀδυσσεύς
+(1 row)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IS NULL;
+                QUERY PLAN                 
+-------------------------------------------
+ Hash Join
+   Hash Cond: (t2.a = t1.a)
+   ->  Append
+         ->  Seq Scan on beta_d t2_1
+         ->  Seq Scan on beta_b t2_2
+         ->  Seq Scan on beta_a t2_3
+         ->  Seq Scan on beta_c t2_4
+         ->  Seq Scan on beta_default t2_5
+   ->  Hash
+         ->  Seq Scan on alpha_f t1
+               Filter: (a IS NULL)
+(11 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IS NULL;
+ a | a 
+---+---
+(0 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a IS NOT DISTINCT FROM t2.a) WHERE t1.a IS NULL;
+                    QUERY PLAN                     
+---------------------------------------------------
+ Nested Loop
+   Join Filter: (NOT (t1.a IS DISTINCT FROM t2.a))
+   ->  Seq Scan on alpha_f t1
+         Filter: (a IS NULL)
+   ->  Append
+         ->  Seq Scan on beta_d t2_1
+         ->  Seq Scan on beta_b t2_2
+         ->  Seq Scan on beta_a t2_3
+         ->  Seq Scan on beta_c t2_4
+         ->  Seq Scan on beta_default t2_5
+(10 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a IS NOT DISTINCT FROM t2.a) WHERE t1.a IS NULL;
+ a | a 
+---+---
+   | 
+(1 row)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IS NULL;
+                QUERY PLAN                 
+-------------------------------------------
+ Hash Join
+   Hash Cond: (t2.a = t1.a)
+   ->  Append
+         ->  Seq Scan on beta_d t2_1
+         ->  Seq Scan on beta_b t2_2
+         ->  Seq Scan on beta_a t2_3
+         ->  Seq Scan on beta_c t2_4
+         ->  Seq Scan on beta_default t2_5
+   ->  Hash
+         ->  Seq Scan on alpha_f t1
+               Filter: (a IS NULL)
+(11 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IS NULL;
+ a | a 
+---+---
+(0 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a IS NOT DISTINCT FROM t2.a) WHERE t1.a IS NULL;
+                    QUERY PLAN                     
+---------------------------------------------------
+ Nested Loop
+   Join Filter: (NOT (t1.a IS DISTINCT FROM t2.a))
+   ->  Seq Scan on alpha_f t1
+         Filter: (a IS NULL)
+   ->  Append
+         ->  Seq Scan on beta_d t2_1
+         ->  Seq Scan on beta_b t2_2
+         ->  Seq Scan on beta_a t2_3
+         ->  Seq Scan on beta_c t2_4
+         ->  Seq Scan on beta_default t2_5
+(10 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a IS NOT DISTINCT FROM t2.a) WHERE t1.a IS NULL;
+ a | a 
+---+---
+   | 
+(1 row)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a IN ('äbç', 'ὀδυσσεύς');
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Hash Semi Join
+   Hash Cond: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on alpha_c t1_1
+               Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+         ->  Seq Scan on alpha_e t1_2
+               Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on beta_d t2_1
+               ->  Seq Scan on beta_b t2_2
+               ->  Seq Scan on beta_a t2_3
+               ->  Seq Scan on beta_c t2_4
+               ->  Seq Scan on beta_default t2_5
+(14 rows)
+
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a IN ('äbç', 'ὀδυσσεύς');
+    a     
+----------
+ äbç
+ ὀδυσσεύς
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'äbç';
+            QUERY PLAN             
+-----------------------------------
+ Nested Loop Semi Join
+   ->  Seq Scan on alpha_c t1
+         Filter: (a = 'äbç'::text)
+   ->  Seq Scan on beta_c t2
+         Filter: (a = 'äbç'::text)
+(5 rows)
+
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'äbç';
+  a  
+-----
+ äbç
+(1 row)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'ὀδυσσεύς';
+               QUERY PLAN               
+----------------------------------------
+ Nested Loop Semi Join
+   ->  Seq Scan on alpha_e t1
+         Filter: (a = 'ὀδυσσεύς'::text)
+   ->  Seq Scan on beta_d t2
+         Filter: (a = 'ὀδυσσεύς'::text)
+(5 rows)
+
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'ὀδυσσεύς';
+    a     
+----------
+ ὀδυσσεύς
+(1 row)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a IS NOT DISTINCT FROM t2.a) AND t1.a IS NULL;
+                    QUERY PLAN                     
+---------------------------------------------------
+ Nested Loop Semi Join
+   Join Filter: (NOT (t1.a IS DISTINCT FROM t2.a))
+   ->  Seq Scan on alpha_f t1
+         Filter: (a IS NULL)
+   ->  Append
+         ->  Seq Scan on beta_d t2_1
+         ->  Seq Scan on beta_b t2_2
+         ->  Seq Scan on beta_a t2_3
+         ->  Seq Scan on beta_c t2_4
+         ->  Seq Scan on beta_default t2_5
+(10 rows)
+
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a IS NOT DISTINCT FROM t2.a) AND t1.a IS NULL;
+ a 
+---
+ 
+(1 row)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a IS NULL;
+                QUERY PLAN                 
+-------------------------------------------
+ Nested Loop Semi Join
+   Join Filter: (t1.a = t2.a)
+   ->  Seq Scan on alpha_f t1
+         Filter: (a IS NULL)
+   ->  Append
+         ->  Seq Scan on beta_d t2_1
+         ->  Seq Scan on beta_b t2_2
+         ->  Seq Scan on beta_a t2_3
+         ->  Seq Scan on beta_c t2_4
+         ->  Seq Scan on beta_default t2_5
+(10 rows)
+
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a IS NULL;
+ a 
+---
+(0 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.a = t1.a)
+   ->  Append
+         ->  Seq Scan on beta_d t2_1
+         ->  Seq Scan on beta_b t2_2
+         ->  Seq Scan on beta_a t2_3
+         ->  Seq Scan on beta_c t2_4
+         ->  Seq Scan on beta_default t2_5
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on alpha_c t1_1
+                     Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+               ->  Seq Scan on alpha_e t1_2
+                     Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+(14 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+    a     |    a     
+----------+----------
+ ὀδυσσεύς | ὀδυσσεύς
+ äbç      | äbç
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'äbç';
+            QUERY PLAN             
+-----------------------------------
+ Nested Loop Left Join
+   Join Filter: (t1.a = t2.a)
+   ->  Seq Scan on alpha_c t1
+         Filter: (a = 'äbç'::text)
+   ->  Seq Scan on beta_c t2
+         Filter: (a = 'äbç'::text)
+(6 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'äbç';
+  a  |  a  
+-----+-----
+ äbç | äbç
+(1 row)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
+               QUERY PLAN               
+----------------------------------------
+ Nested Loop Left Join
+   Join Filter: (t1.a = t2.a)
+   ->  Seq Scan on alpha_e t1
+         Filter: (a = 'ὀδυσσεύς'::text)
+   ->  Seq Scan on beta_d t2
+         Filter: (a = 'ὀδυσσεύς'::text)
+(6 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
+    a     |    a     
+----------+----------
+ ὀδυσσεύς | ὀδυσσεύς
+(1 row)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a IS NOT DISTINCT FROM t2.a) WHERE t1.a IS NULL;
+                    QUERY PLAN                     
+---------------------------------------------------
+ Nested Loop Left Join
+   Join Filter: (NOT (t1.a IS DISTINCT FROM t2.a))
+   ->  Seq Scan on alpha_f t1
+         Filter: (a IS NULL)
+   ->  Append
+         ->  Seq Scan on beta_d t2_1
+         ->  Seq Scan on beta_b t2_2
+         ->  Seq Scan on beta_a t2_3
+         ->  Seq Scan on beta_c t2_4
+         ->  Seq Scan on beta_default t2_5
+(10 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a IS NOT DISTINCT FROM t2.a) WHERE t1.a IS NULL;
+ a | a 
+---+---
+   | 
+(1 row)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IS NULL;
+                QUERY PLAN                 
+-------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.a = t1.a)
+   ->  Append
+         ->  Seq Scan on beta_d t2_1
+         ->  Seq Scan on beta_b t2_2
+         ->  Seq Scan on beta_a t2_3
+         ->  Seq Scan on beta_c t2_4
+         ->  Seq Scan on beta_default t2_5
+   ->  Hash
+         ->  Seq Scan on alpha_f t1
+               Filter: (a IS NULL)
+(11 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IS NULL;
+ a | a 
+---+---
+   | 
+(1 row)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a IS NOT DISTINCT FROM t2.a) WHERE t1.a IS NOT DISTINCT FROM t2.a;
+                    QUERY PLAN                     
+---------------------------------------------------
+ Nested Loop Left Join
+   Join Filter: (NOT (t1.a IS DISTINCT FROM t2.a))
+   Filter: (NOT (t1.a IS DISTINCT FROM t2.a))
+   ->  Append
+         ->  Seq Scan on alpha_b t1_1
+         ->  Seq Scan on alpha_d t1_2
+         ->  Seq Scan on alpha_a t1_3
+         ->  Seq Scan on alpha_c t1_4
+         ->  Seq Scan on alpha_f t1_5
+         ->  Seq Scan on alpha_e t1_6
+         ->  Seq Scan on alpha_default t1_7
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on beta_d t2_1
+               ->  Seq Scan on beta_b t2_2
+               ->  Seq Scan on beta_a t2_3
+               ->  Seq Scan on beta_c t2_4
+               ->  Seq Scan on beta_default t2_5
+(18 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a IS NOT DISTINCT FROM t2.a) WHERE t1.a IS NOT DISTINCT FROM t2.a;
+             a              |             a              
+----------------------------+----------------------------
+ bıt                        | bıt
+ BIT                        | BIT
+ aaá                        | aaá
+ coté                       | coté
+ Götz                       | Götz
+ Türkiye                    | Türkiye
+ TÜRKIYE                    | TÜRKIYE
+ äbç                        | äbç
+ ÄBÇ                        | ÄBÇ
+ を読み取り用               | を読み取り用
+ にオープンできませんでした | にオープンできませんでした
+                            | 
+ ὀδυσσεύς                   | ὀδυσσεύς
+ ὈΔΥΣΣΕΎΣ                   | ὈΔΥΣΣΕΎΣ
+ grumble                    | grumble
+(15 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a IN ('äbç', 'ὀδυσσεύς');
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Hash Anti Join
+   Hash Cond: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on alpha_c t1_1
+               Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+         ->  Seq Scan on alpha_e t1_2
+               Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on beta_d t2_1
+               ->  Seq Scan on beta_b t2_2
+               ->  Seq Scan on beta_a t2_3
+               ->  Seq Scan on beta_c t2_4
+               ->  Seq Scan on beta_default t2_5
+(14 rows)
+
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a IN ('äbç', 'ὀδυσσεύς');
+ a 
+---
+(0 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'äbç';
+            QUERY PLAN             
+-----------------------------------
+ Nested Loop Anti Join
+   Join Filter: (t1.a = t2.a)
+   ->  Seq Scan on alpha_c t1
+         Filter: (a = 'äbç'::text)
+   ->  Seq Scan on beta_c t2
+         Filter: (a = 'äbç'::text)
+(6 rows)
+
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'äbç';
+ a 
+---
+(0 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'ὀδυσσεύς';
+               QUERY PLAN               
+----------------------------------------
+ Nested Loop Anti Join
+   Join Filter: (t1.a = t2.a)
+   ->  Seq Scan on alpha_e t1
+         Filter: (a = 'ὀδυσσεύς'::text)
+   ->  Seq Scan on beta_d t2
+         Filter: (a = 'ὀδυσσεύς'::text)
+(6 rows)
+
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'ὀδυσσεύς';
+ a 
+---
+(0 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a IS NOT DISTINCT FROM t2.a) AND t1.a IS NULL;
+                    QUERY PLAN                     
+---------------------------------------------------
+ Nested Loop Anti Join
+   Join Filter: (NOT (t1.a IS DISTINCT FROM t2.a))
+   ->  Seq Scan on alpha_f t1
+         Filter: (a IS NULL)
+   ->  Append
+         ->  Seq Scan on beta_d t2_1
+         ->  Seq Scan on beta_b t2_2
+         ->  Seq Scan on beta_a t2_3
+         ->  Seq Scan on beta_c t2_4
+         ->  Seq Scan on beta_default t2_5
+(10 rows)
+
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a IS NOT DISTINCT FROM t2.a) AND t1.a IS NULL;
+ a 
+---
+(0 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a IS NULL;
+                QUERY PLAN                 
+-------------------------------------------
+ Nested Loop Anti Join
+   Join Filter: (t1.a = t2.a)
+   ->  Seq Scan on alpha_f t1
+         Filter: (a IS NULL)
+   ->  Append
+         ->  Seq Scan on beta_d t2_1
+         ->  Seq Scan on beta_b t2_2
+         ->  Seq Scan on beta_a t2_3
+         ->  Seq Scan on beta_c t2_4
+         ->  Seq Scan on beta_default t2_5
+(10 rows)
+
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a IS NULL;
+ a 
+---
+ 
+(1 row)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.a = t1.a)
+   ->  Append
+         ->  Seq Scan on beta_d t2_1
+         ->  Seq Scan on beta_b t2_2
+         ->  Seq Scan on beta_a t2_3
+         ->  Seq Scan on beta_c t2_4
+         ->  Seq Scan on beta_default t2_5
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on alpha_c t1_1
+                     Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+               ->  Seq Scan on alpha_e t1_2
+                     Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+(14 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+    a     |    a     
+----------+----------
+ ὀδυσσεύς | ὀδυσσεύς
+ äbç      | äbç
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'äbç';
+            QUERY PLAN             
+-----------------------------------
+ Nested Loop Left Join
+   Join Filter: (t1.a = t2.a)
+   ->  Seq Scan on alpha_c t1
+         Filter: (a = 'äbç'::text)
+   ->  Seq Scan on beta_c t2
+         Filter: (a = 'äbç'::text)
+(6 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'äbç';
+  a  |  a  
+-----+-----
+ äbç | äbç
+(1 row)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
+               QUERY PLAN               
+----------------------------------------
+ Nested Loop Left Join
+   Join Filter: (t1.a = t2.a)
+   ->  Seq Scan on alpha_e t1
+         Filter: (a = 'ὀδυσσεύς'::text)
+   ->  Seq Scan on beta_d t2
+         Filter: (a = 'ὀδυσσεύς'::text)
+(6 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
+    a     |    a     
+----------+----------
+ ὀδυσσεύς | ὀδυσσεύς
+(1 row)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IS NULL;
+                   QUERY PLAN                    
+-------------------------------------------------
+ Hash Full Join
+   Hash Cond: (t1.a = t2.a)
+   Filter: (t1.a IS NULL)
+   ->  Append
+         ->  Seq Scan on alpha_b t1_1
+         ->  Seq Scan on alpha_d t1_2
+         ->  Seq Scan on alpha_a t1_3
+         ->  Seq Scan on alpha_c t1_4
+         ->  Seq Scan on alpha_f t1_5
+         ->  Seq Scan on alpha_e t1_6
+         ->  Seq Scan on alpha_default t1_7
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on beta_d t2_1
+               ->  Seq Scan on beta_b t2_2
+               ->  Seq Scan on beta_a t2_3
+               ->  Seq Scan on beta_c t2_4
+               ->  Seq Scan on beta_default t2_5
+(18 rows)
+
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IS NULL;
+ a | a 
+---+---
+   | 
+   | 
+(2 rows)
+
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 159d459831..9bc2188ca8 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -1064,3 +1064,146 @@ SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a =
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
 SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
+
+-- Try again, this time with list partitioning
+DROP TABLE alpha CASCADE;
+DROP TABLE beta CASCADE;
+
+CREATE TABLE alpha (a TEXT) PARTITION BY LIST(a);
+CREATE TABLE alpha_a PARTITION OF alpha FOR VALUES IN ('Türkiye', 'TÜRKIYE');
+CREATE TABLE alpha_b PARTITION OF alpha FOR VALUES IN ('bıt', 'BIT');
+CREATE TABLE alpha_c PARTITION OF alpha FOR VALUES IN ('äbç', 'ÄBÇ');
+CREATE TABLE alpha_d PARTITION OF alpha FOR VALUES IN ('aaá', 'coté', 'Götz');
+CREATE TABLE alpha_e PARTITION OF alpha FOR VALUES IN ('ὀδυσσεύς', 'ὈΔΥΣΣΕΎΣ');
+CREATE TABLE alpha_f PARTITION OF alpha FOR VALUES IN ('を読み取り用', 'にオープンできませんでした', NULL);
+CREATE TABLE alpha_default PARTITION OF alpha DEFAULT;
+
+CREATE TABLE beta (a TEXT) PARTITION BY LIST(a);
+CREATE TABLE beta_a PARTITION OF beta FOR VALUES IN ('Türkiye', 'coté', 'ὈΔΥΣΣΕΎΣ');
+CREATE TABLE beta_b PARTITION OF beta FOR VALUES IN ('bıt', 'TÜRKIYE');
+CREATE TABLE beta_c PARTITION OF beta FOR VALUES IN ('äbç', 'を読み取り用', 'にオープンできませんでした');
+CREATE TABLE beta_d PARTITION OF beta FOR VALUES IN ('aaá', 'Götz', 'BIT', 'ὀδυσσεύς', 'ÄBÇ', NULL);
+CREATE TABLE beta_default PARTITION OF beta DEFAULT;
+
+INSERT INTO alpha (SELECT a FROM raw_data);
+INSERT INTO beta (SELECT a FROM raw_data);
+INSERT INTO alpha VALUES (null);
+INSERT INTO beta VALUES (null);
+INSERT INTO alpha VALUES ('grumble');
+INSERT INTO beta VALUES ('grumble');
+
+ANALYZE alpha;
+ANALYZE beta;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'äbç';
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'äbç';
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IS NULL;
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IS NULL;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a IS NOT DISTINCT FROM t2.a) WHERE t1.a IS NULL;
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a IS NOT DISTINCT FROM t2.a) WHERE t1.a IS NULL;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IS NULL;
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IS NULL;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a IS NOT DISTINCT FROM t2.a) WHERE t1.a IS NULL;
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a IS NOT DISTINCT FROM t2.a) WHERE t1.a IS NULL;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a IN ('äbç', 'ὀδυσσεύς');
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a IN ('äbç', 'ὀδυσσεύς');
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'äbç';
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'äbç';
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'ὀδυσσεύς';
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'ὀδυσσεύς';
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a IS NOT DISTINCT FROM t2.a) AND t1.a IS NULL;
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a IS NOT DISTINCT FROM t2.a) AND t1.a IS NULL;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a IS NULL;
+SELECT t1.a FROM alpha t1 WHERE EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a IS NULL;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'äbç';
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'äbç';
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a IS NOT DISTINCT FROM t2.a) WHERE t1.a IS NULL;
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a IS NOT DISTINCT FROM t2.a) WHERE t1.a IS NULL;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IS NULL;
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IS NULL;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a IS NOT DISTINCT FROM t2.a) WHERE t1.a IS NOT DISTINCT FROM t2.a;
+SELECT t1.a, t2.a FROM alpha t1 LEFT JOIN beta t2 ON (t1.a IS NOT DISTINCT FROM t2.a) WHERE t1.a IS NOT DISTINCT FROM t2.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a IN ('äbç', 'ὀδυσσεύς');
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a IN ('äbç', 'ὀδυσσεύς');
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'äbç';
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'äbç';
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'ὀδυσσεύς';
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a = 'ὀδυσσεύς';
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a IS NOT DISTINCT FROM t2.a) AND t1.a IS NULL;
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a IS NOT DISTINCT FROM t2.a) AND t1.a IS NULL;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a IS NULL;
+SELECT t1.a FROM alpha t1 WHERE NOT EXISTS (SELECT 1 FROM beta t2 WHERE t1.a = t2.a) AND t1.a IS NULL;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'äbç';
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'äbç';
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a = 'ὀδυσσεύς';
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IS NULL;
+SELECT t1.a, t2.a FROM alpha t1 FULL JOIN beta t2 ON (t1.a = t2.a) WHERE t1.a IS NULL;
-- 
2.21.1 (Apple Git-122.3)

#114Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Mark Dilger (#113)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Hi Mark,

On Tue, Jan 28, 2020 at 1:39 PM Mark Dilger
<mark.dilger@enterprisedb.com> wrote:

There is stray whitespace in v30-0002:

src/backend/partitioning/partbounds.c:4557: space before tab in indent.
+ outer_null_unmerged = true;

Good catch!

I have added tests checking correctness and showing some partition pruning limitations. Find three patches, attached.

The v31-0001-… patch merely applies your patches as a starting point for the next two patches. It is your work, not mine.

The v31-0002-… patch adds more regression tests for range partitioning. The commit message contains my comments about that.

The v31-0003-… patch adds more regression tests for list partitioning, and again, the commit message contains my comments about that.

The PWJ behavior shown by the tests you added looks interesting! I'll
dig into it more closely. Thanks for the patches and review!

Best regards,
Etsuro Fujita

#115Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Etsuro Fujita (#114)
2 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Wed, Jan 29, 2020 at 9:15 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

On Tue, Jan 28, 2020 at 1:39 PM Mark Dilger
<mark.dilger@enterprisedb.com> wrote:

I have added tests checking correctness and showing some partition pruning limitations. Find three patches, attached.

The v31-0001-… patch merely applies your patches as a starting point for the next two patches. It is your work, not mine.

The v31-0002-… patch adds more regression tests for range partitioning. The commit message contains my comments about that.

The v31-0003-… patch adds more regression tests for list partitioning, and again, the commit message contains my comments about that.

I'll dig into it more closely.

I tested the new test patches. The patches are applied to the
partition_join.sql regression test cleanly, but the new tests failed
in my environment (even with make check LANG=C). I think I should set
the right locale for the testing. Which one did you use? Maybe I'm
missing something, but let me comment on the new tests. This is the
one proposed in the v31-0002 patch:

+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a)
WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+                            QUERY PLAN
+------------------------------------------------------------------
+ Hash Join
+   Hash Cond: (t2.a = t1.a)
+   ->  Append
+         ->  Seq Scan on beta_a t2_1
+         ->  Seq Scan on beta_b t2_2
+         ->  Seq Scan on beta_c t2_3
+         ->  Seq Scan on beta_d t2_4
+         ->  Seq Scan on beta_e t2_5
+         ->  Seq Scan on beta_f t2_6
+         ->  Seq Scan on beta_g t2_7
+         ->  Seq Scan on beta_h t2_8
+         ->  Seq Scan on beta_default t2_9
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on alpha_e t1_1
+                     Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+               ->  Seq Scan on alpha_default t1_2
+                     Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+(18 rows)

The commit message says:

When joining with

alpha.a = beta.a and alpha.a IN ('äbç', 'ὀδυσσεύς')

the planner does the right thing for one side of the query, but
hits all partitions for the other side, which it doesn't need to
do.

Yeah, I agree that the resulting plan is not efficient. The reason
for that would be that the planner doesn't generate a qual on the
outer side matching the ScalarArrayOpExpr qual "a = ANY
('{äbç,ὀδυσσεύς}'::text[])" on the inner side, which I think would be
a restriction caused by the equivalence machinery not by the
partitionwise join logic IIUC. I think the critique would be useful,
so I don't object to adding this test case, but the critique would be
more about query planning that is actually not related to the
partitionwise join logic, so I'm not sure that the partition_join.sql
regression test is the best place to add it. I guess that there would
be much better places than partition_join.sql. (This is nitpicking;
but another thing I noticed about this test case is that the join
query contains only a single join condition "t1.a = t2.a", but the
queried tables alpha and beta are range-partitioned by multiple
columns a and b, so the query should have a join condition for each of
the columns like "t1.a = t2.a AND t1.b = t2.b" if adding this as a
test case for partitionwise join.) I feel almost the same to other
test cases in the patch (and the v31-0003 patch), except this one
proposed in the v31-0003 patch:

+CREATE TABLE alpha (a TEXT) PARTITION BY LIST(a);
+CREATE TABLE alpha_a PARTITION OF alpha FOR VALUES IN ('Turkiye', 'TURKIYE');
+CREATE TABLE alpha_b PARTITION OF alpha FOR VALUES IN ('b?t', 'BIT');
+CREATE TABLE alpha_c PARTITION OF alpha FOR VALUES IN ('abc', 'ABC');
+CREATE TABLE alpha_d PARTITION OF alpha FOR VALUES IN ('aaa', 'cote', 'Gotz');
+CREATE TABLE alpha_e PARTITION OF alpha FOR VALUES IN ('?δυσσε??', '?ΔΥΣΣΕ?Σ');
+CREATE TABLE alpha_f PARTITION OF alpha FOR VALUES IN ('を読み取り用',
'にオープンできませんでした', NULL);
+CREATE TABLE alpha_default PARTITION OF alpha DEFAULT;
+CREATE TABLE beta (a TEXT) PARTITION BY LIST(a);
+CREATE TABLE beta_a PARTITION OF beta FOR VALUES IN ('Turkiye',
'cote', '?ΔΥΣΣΕ?Σ');
+CREATE TABLE beta_b PARTITION OF beta FOR VALUES IN ('b?t', 'TURKIYE');
+CREATE TABLE beta_c PARTITION OF beta FOR VALUES IN ('abc', 'を読み取り用',
'にオープンできませんでした');
+CREATE TABLE beta_d PARTITION OF beta FOR VALUES IN ('aaa', 'Gotz',
'BIT', '?δυσσε??', 'ABC', NULL);
+CREATE TABLE beta_default PARTITION OF beta DEFAULT;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a)
WHERE t1.a IS NULL;
+                QUERY PLAN
+-------------------------------------------
+ Hash Join
+   Hash Cond: (t2.a = t1.a)
+   ->  Append
+         ->  Seq Scan on beta_d t2_1
+         ->  Seq Scan on beta_b t2_2
+         ->  Seq Scan on beta_a t2_3
+         ->  Seq Scan on beta_c t2_4
+         ->  Seq Scan on beta_default t2_5
+   ->  Hash
+         ->  Seq Scan on alpha_f t1
+               Filter: (a IS NULL)
+(11 rows)

Which made me notice an issue in the partitionwise join logic:
partition_bounds_merge() assumes that all partitions of given
relations are always present; in other words, it doesn't consider
cases where some of the partitions have been pruned entirely. :-( If
that function considered the cases, the above query would use
partitionwise join, because alpha_f only matches beta_c. I don't
think this is a bug, but it causes the planner not only to fail to
chose partitionwise join but to waste CPU cycles to process pruned
partitions, so I'll propose to address it. Attached is a patch for
that (the v31-0004 patch) created on top of the main patch (the
v31-0001 patch), which is also attached. With the attached, the plan
for the above query would be changed to something like this:

EXPLAIN (COSTS OFF)
SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a =
t2.a) WHERE t1.a IS NULL;
QUERY PLAN
------------------------------
Nested Loop
Join Filter: (t1.a = t2.a)
-> Seq Scan on alpha_f t1
Filter: (a IS NULL)
-> Seq Scan on beta_c t2
(5 rows)

Thanks again, Mark!

Best regards,
Etsuro Fujita

Attachments:

v31-0001-Applying-Etsuro-Fujita-s-patches.patchapplication/octet-stream; name=v31-0001-Applying-Etsuro-Fujita-s-patches.patchDownload
From 672dfb708a3a0a4f4c9f4158245ec130dcbeab15 Mon Sep 17 00:00:00 2001
From: Mark Dilger <mark.dilger@enterprisedb.com>
Date: Mon, 27 Jan 2020 18:06:37 -0800
Subject: [PATCH 1/3] Applying Etsuro Fujita's patches

This applies both these patches:

  v30-0001-Improve-partition-matching-for-partitionwise-join.patch
  v30-0002-Fix-handling-of-NULL-partitions.patch

Those patches are byh Etsuro Fujita, though I fixed one whitespace
issue after applying them.

Author: Etsuro Fujita
---
 src/backend/nodes/outfuncs.c                 |    2 +
 src/backend/optimizer/path/joinrels.c        |  245 +-
 src/backend/optimizer/util/inherit.c         |    2 +
 src/backend/optimizer/util/relnode.c         |   45 +-
 src/backend/partitioning/partbounds.c        | 1613 +++++++++++++
 src/include/nodes/pathnodes.h                |    5 +
 src/include/partitioning/partbounds.h        |    7 +
 src/test/regress/expected/partition_join.out | 2120 +++++++++++++++++-
 src/test/regress/sql/partition_join.sql      |  513 ++++-
 9 files changed, 4477 insertions(+), 75 deletions(-)

diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index d76fae44b8..e42a52fb9b 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -2285,6 +2285,8 @@ _outRelOptInfo(StringInfo str, const RelOptInfo *node)
 	WRITE_BOOL_FIELD(has_eclass_joins);
 	WRITE_BOOL_FIELD(consider_partitionwise_join);
 	WRITE_BITMAPSET_FIELD(top_parent_relids);
+	WRITE_BOOL_FIELD(merged);
+	WRITE_BITMAPSET_FIELD(all_partrels);
 	WRITE_NODE_FIELD(partitioned_child_rels);
 }
 
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index a21c295b99..77e6ff5376 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -48,6 +48,9 @@ static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 												Relids left_relids, Relids right_relids);
 static int	match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel,
 										 bool strict_op);
+static void get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2);
 
 
 /*
@@ -1357,25 +1360,30 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	bool		merged = false;
+	List	   *parts1 = NIL;
+	List	   *parts2 = NIL;
+	ListCell   *lcr1 = NULL;
+	ListCell   *lcr2 = NULL;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1390,34 +1398,107 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		   joinrel->part_scheme == rel2->part_scheme);
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * If we don't have the partition bounds for the join rel yet, try to
+	 * create it along with pairs of partitions to be joined; else generate
+	 * those using the partitioning info for the join rel we already have.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	if (joinrel->nparts == -1)
+	{
+		PartitionScheme part_scheme = joinrel->part_scheme;
+		PartitionBoundInfo boundinfo = NULL;
+		int			nparts = 0;
+
+		Assert(joinrel->boundinfo == NULL);
+		Assert(joinrel->part_rels == NULL);
+
+		/*
+		 * See if the partition bounds for inputs are exactly the same, in
+		 * which case we don't need to work hard: partitions with the same
+		 * partition indexes will form join pairs, and the join rel will have
+		 * the same partition bounds as inputs; otherwise try to merge the
+		 * partition bounds along with generating join pairs.
+		 *
+		 * Even if one or both inputs have merged partition bounds, it'd be
+		 * possible for the partition bounds to be exactly the same, but it
+		 * seems unlikely to be worth the cycles to check; do this check only
+		 * if both inputs have non-merged partition bounds.
+		 */
+		if (!rel1->merged &&
+			!rel2->merged &&
+			rel1->nparts == rel2->nparts &&
+			partition_bounds_equal(part_scheme->partnatts,
+								   part_scheme->parttyplen,
+								   part_scheme->parttypbyval,
+								   rel1->boundinfo, rel2->boundinfo))
+		{
+			boundinfo = rel1->boundinfo;
+			nparts = rel1->nparts;
+		}
+		else
+		{
+			boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											   part_scheme->parttyplen,
+											   part_scheme->parttypbyval,
+											   part_scheme->partsupfunc,
+											   part_scheme->partcollation,
+											   rel1, rel2,
+											   parent_sjinfo->jointype,
+											   &parts1, &parts2);
+			if (boundinfo == NULL)
+			{
+				joinrel->nparts = 0;
+				return;
+			}
+			nparts = list_length(parts1);
+			merged = true;
+		}
+
+		Assert(nparts > 0);
+		joinrel->boundinfo = boundinfo;
+		joinrel->merged = merged;
+		joinrel->nparts = nparts;
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+	}
+	else
+	{
+		Assert(joinrel->nparts > 0);
+		Assert(joinrel->boundinfo);
+		Assert(joinrel->part_rels);
+
+		/*
+		 * If the partition bounds for the join rel are not merged ones,
+		 * inputs are guaranteed to have the same partition bounds, so
+		 * partitions with the same partition indexes will form join pairs;
+		 * else let get_matching_part_pairs() do the work.
+		 */
+		if (joinrel->merged)
+		{
+			get_matching_part_pairs(root, joinrel, rel1, rel2,
+									&parts1, &parts2);
+			Assert(list_length(parts1) == joinrel->nparts);
+			Assert(list_length(parts2) == joinrel->nparts);
+			merged = true;
+		}
+	}
 
-	nparts = joinrel->nparts;
+	if (merged)
+	{
+		lcr1 = list_head(parts1);
+		lcr2 = list_head(parts2);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
-		bool		rel1_empty = (child_rel1 == NULL ||
-								  IS_DUMMY_REL(child_rel1));
-		bool		rel2_empty = (child_rel2 == NULL ||
-								  IS_DUMMY_REL(child_rel2));
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		bool		rel1_empty;
+		bool		rel2_empty;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1425,6 +1506,22 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		if (merged)
+		{
+			child_rel1 = lfirst_node(RelOptInfo, lcr1);
+			child_rel2 = lfirst_node(RelOptInfo, lcr2);
+			lcr1 = lnext(parts1, lcr1);
+			lcr2 = lnext(parts2, lcr2);
+		}
+		else
+		{
+			child_rel1 = rel1->part_rels[cnt_parts];
+			child_rel2 = rel2->part_rels[cnt_parts];
+		}
+
+		rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1));
+		rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2));
+
 		/*
 		 * Check for cases where we can prove that this segment of the join
 		 * returns no rows, due to one or both inputs being empty (including
@@ -1522,6 +1619,8 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 												 child_sjinfo,
 												 child_sjinfo->jointype);
 			joinrel->part_rels[cnt_parts] = child_joinrel;
+			joinrel->all_partrels = bms_add_members(joinrel->all_partrels,
+													child_joinrel->relids);
 		}
 
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
@@ -1738,3 +1837,99 @@ match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op)
 
 	return -1;
 }
+
+/*
+ * get_matching_part_pairs
+ *		Generate join pairs of partitions for the two inputs
+ */
+static void
+get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2)
+{
+	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
+	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
+	int 		cnt_parts;
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
+	{
+		RelOptInfo *child_joinrel = joinrel->part_rels[cnt_parts];
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		Relids		child_relids1;
+		Relids		child_relids2;
+
+		/*
+		 * If this segment of the join is empty, it means that this segment
+		 * was ignored when previously creating child-join paths for it in
+		 * try_partitionwise_join() as it would not contribute to the join
+		 * result, due to one or both inputs being empty; add NULL to each of
+		 * the given lists so that this segment will be ignored again in that
+		 * function.
+		 */
+		if (!child_joinrel)
+		{
+			*parts1 = lappend(*parts1, NULL);
+			*parts2 = lappend(*parts2, NULL);
+			continue;
+		}
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel1 side.
+		 */
+		child_relids1 = bms_intersect(child_joinrel->relids,
+									  rel1->all_partrels);
+		Assert(bms_num_members(child_relids1) == bms_num_members(rel1->relids));
+
+		/*
+		 * Get a child rel for rel1 with the relids.  Note that we should have
+		 * the child rel even if rel1 is a join rel, because in that case the
+		 * partitions specified in the relids would have matching/overlapping
+		 * boundaries, so those partitions should be considered as ones to be
+		 * joined even when planning partitionwise joins of rel1, meaning that
+		 * the child rel would have been built by the time we get here.
+		 */
+		if (rel1_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids1);
+
+			child_rel1 = find_base_rel(root, varno);
+		}
+		else
+			child_rel1 = find_join_rel(root, child_relids1);
+		Assert(child_rel1);
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel2 side.
+		 */
+		child_relids2 = bms_intersect(child_joinrel->relids,
+									  rel2->all_partrels);
+		Assert(bms_num_members(child_relids2) == bms_num_members(rel2->relids));
+
+		/*
+		 * Get a child rel for rel2 with the relids.  See above comments.
+		 */
+		if (rel2_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids2);
+
+			child_rel2 = find_base_rel(root, varno);
+		}
+		else
+			child_rel2 = find_join_rel(root, child_relids2);
+		Assert(child_rel2);
+
+		/*
+		 * The join of rel1 and rel2 is legal, so is the join of the child
+		 * rels obtained above; add them to the given lists as a join pair
+		 * producing this join segment.
+		 */
+		*parts1 = lappend(*parts1, child_rel1);
+		*parts2 = lappend(*parts2, child_rel2);
+	}
+}
diff --git a/src/backend/optimizer/util/inherit.c b/src/backend/optimizer/util/inherit.c
index 7db67fdf34..3132fd35a5 100644
--- a/src/backend/optimizer/util/inherit.c
+++ b/src/backend/optimizer/util/inherit.c
@@ -376,6 +376,8 @@ expand_partitioned_rtentry(PlannerInfo *root, RelOptInfo *relinfo,
 		/* Create the otherrel RelOptInfo too. */
 		childrelinfo = build_simple_rel(root, childRTindex, relinfo);
 		relinfo->part_rels[i] = childrelinfo;
+		relinfo->all_partrels = bms_add_members(relinfo->all_partrels,
+												childrelinfo->relids);
 
 		/* If this child is itself partitioned, recurse */
 		if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 374f93890b..1df6da64ed 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -240,10 +240,12 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->has_eclass_joins = false;
 	rel->consider_partitionwise_join = false;	/* might get changed later */
 	rel->part_scheme = NULL;
-	rel->nparts = 0;
+	rel->nparts = -1;
 	rel->boundinfo = NULL;
+	rel->merged = false;
 	rel->partition_qual = NIL;
 	rel->part_rels = NULL;
+	rel->all_partrels = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->partitioned_child_rels = NIL;
@@ -653,10 +655,12 @@ build_join_rel(PlannerInfo *root,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -829,10 +833,12 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -1639,7 +1645,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1652,24 +1658,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1681,17 +1669,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 95af37b9c7..43c4e03dde 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -25,6 +25,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/pathnodes.h"
 #include "parser/parse_coerce.h"
 #include "partitioning/partbounds.h"
 #include "partitioning/partdesc.h"
@@ -68,6 +69,23 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+typedef struct PartitionMap
+{
+	int			nparts;			/* number of partitions */
+	int		   *merged_indexes;	/* indexes of merged partitions */
+	bool	   *merged;			/* flags to indicate whether partitions are
+								 * merged with non-dummy partitions */
+	bool		did_remapping;	/* did we remap partitions? */
+	int		   *old_indexes;	/* old indexes of merged partitions if
+								 * did_remapping */
+} PartitionMap;
+
+#define compare_range_bounds(partnatts, partsupfunc, partcollations, \
+							 bound1, bound2) \
+	(partition_rbound_cmp(partnatts, partsupfunc, partcollations, \
+						  (bound1)->datums, (bound1)->kind, (bound1)->lower, \
+						  bound2))
+
 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);
@@ -107,6 +125,96 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 									 Expr **keyCol,
 									 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static PartitionBoundInfo partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts);
+static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							JoinType jointype,
+							List **outer_parts, List **inner_parts);
+static void init_partition_map(RelOptInfo *rel, PartitionMap *map);
+static void free_partition_map(PartitionMap *map);
+static int map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_part, int inner_part, int *next_index);
+static int merge_partition_with_dummy(PartitionMap *map, int index,
+						   int *next_index);
+static int process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index);
+static int process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index);
+static void fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes);
+static void generate_matching_part_pairs(RelOptInfo *outer_rel,
+							 RelOptInfo *inner_rel,
+							 PartitionMap *outer_map,
+							 PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_part_list,
+							 List **inner_part_list);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int get_range_partition(PartitionBoundInfo bi, int *lb_pos,
+					PartitionRangeBound *lb, PartitionRangeBound *ub);
+static bool compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+						 Oid *partcollations,
+						 PartitionRangeBound *outer_lb,
+						 PartitionRangeBound *outer_ub,
+						 PartitionRangeBound *inner_lb,
+						 PartitionRangeBound *inner_ub,
+						 int *lb_cmpval, int *ub_cmpval);
+static void get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations, JoinType jointype,
+						PartitionRangeBound *outer_lb,
+						PartitionRangeBound *outer_ub,
+						PartitionRangeBound *inner_lb,
+						PartitionRangeBound *inner_ub,
+						int	lb_cmpval, int ub_cmpval,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub);
+static void add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub,
+						int merged_index,
+						List **merged_datums,
+						List **merged_kinds,
+						List **merged_indexes);
+static void merge_default_partitions(PartitionMap *outer_map,
+						 PartitionMap *inner_map,
+						 bool outer_has_default,
+						 bool inner_has_default,
+						 int outer_default,
+						 int inner_default,
+						 JoinType jointype,
+						 int *next_index,
+						 int *default_index);
+static void merge_null_partitions(PartitionMap *outer_map,
+					  PartitionMap *inner_map,
+					  bool outer_has_null,
+					  bool inner_has_null,
+					  int outer_null,
+					  int inner_null,
+					  JoinType jointype,
+					  int *next_index,
+					  int *null_index);
 
 /*
  * get_qual_from_partbound
@@ -2999,3 +3107,1508 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * This function builds and returns the partition bounds for a join relation
+ * between input relations, creating two lists of partitions, which are
+ * returned to *outer_parts and *inner_parts respectively.  The lists contain
+ * the same number of partitions, and the partitions at the same positions in
+ * the lists indicate join pairs used for partitioned join.
+ *
+ * This function returns NULL, setting *outer_parts and *inner_parts to NIL,
+ * if a partition on one side matches multiple partitions on the other side,
+ * in which case we currently don't support partitioned join.
+ */
+PartitionBoundInfo
+partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
+{
+	PartitionBoundInfo 	merged_bounds;
+	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
+						inner_binfo = inner_rel->boundinfo;
+	char				strategy = outer_binfo->strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = NULL;
+
+			break;
+
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partsupfunc,
+														partcollation,
+														outer_rel,
+														inner_rel,
+														jointype,
+														outer_parts,
+														inner_parts);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(partnatts,
+														 partsupfunc,
+														 partcollation,
+														 outer_rel,
+														 inner_rel,
+														 jointype,
+														 outer_parts,
+														 inner_parts);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	return merged_bounds;
+}
+
+/*
+ * get_range_partition
+ *		Returns the index of the range partition with the given lower bound
+ *
+ * *lb and *ub are set to the lower and upper bounds of the range partition
+ * respectively, and *lb_index is advanced to the next lower bound, if any.
+ */
+static int
+get_range_partition(PartitionBoundInfo bi, int *lb_index,
+					PartitionRangeBound *lb, PartitionRangeBound *ub)
+{
+	/* Return the index as -1 if we've exhausted all the lower bounds. */
+	if (*lb_index >= bi->ndatums)
+		return -1;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(*lb_index + 1 < bi->ndatums);
+
+	lb->index = bi->indexes[*lb_index];
+	lb->kind = bi->kind[*lb_index];
+	lb->datums = bi->datums[*lb_index];
+	lb->lower = true;
+	ub->index = bi->indexes[*lb_index + 1];
+	ub->kind = bi->kind[*lb_index + 1];
+	ub->datums = bi->datums[*lb_index + 1];
+	ub->lower = false;
+
+	/* The partition index of an upper bound should be valid. */
+	Assert(ub->index >= 0);
+
+	/*
+	 * Advance the lower bound; if there are no bounds left beyond the upper
+	 * bound, we have reached the last lower bound.
+	 */
+	if (*lb_index + 2 >= bi->ndatums)
+		*lb_index = bi->ndatums;
+	else
+	{
+		/*
+		 * If the index assigned to the bound next to the upper bound isn't
+		 * valid, that is the lower bound of the next range partition; else,
+		 * the upper bound of the current range partition is also the lower
+		 * bound of the next range partition.
+		 */
+		if (bi->indexes[*lb_index + 2] < 0)
+			*lb_index = *lb_index + 2;
+		else
+			*lb_index = *lb_index + 1;
+	}
+
+	return ub->index;
+}
+
+/*
+ * compare_range_partitions
+ *		Compares the bounds of two range partitions, and returns true if the
+ *		ranges of the partitions overlap, false otherwise
+ *
+ * *lb_cmpval is set to -1, 0 or 1 if the outer partition's lower bound is
+ * lower than, equal to or higher than the inner partition's lower bound
+ * respectively.  Likewise, *ub_cmpval is set to -1, 0 or 1 if the outer
+ * partition's upper bound is lower than, equal to or higher than the inner
+ * partition's upper bound respectively.
+ */
+static bool
+compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+						 Oid *partcollations,
+						 PartitionRangeBound *outer_lb,
+						 PartitionRangeBound *outer_ub,
+						 PartitionRangeBound *inner_lb,
+						 PartitionRangeBound *inner_ub,
+						 int *lb_cmpval, int *ub_cmpval)
+{
+	/*
+	 * Check to see if the upper bound of the outer partition is lower than
+	 * the lower bound of the inner partition; in which case the partitions
+	 * aren't overlapping.
+	 */
+	if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+							 outer_ub, inner_lb) < 0)
+	{
+		*lb_cmpval = -1;
+		*ub_cmpval = -1;
+		return false;
+	}
+
+	/*
+	 * Check to see if the lower bound of the outer partition is higher than
+	 * the upper bound of the inner partition; in which case the partitions
+	 * aren't overlapping.
+	 */
+	if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+							 outer_lb, inner_ub) > 0)
+	{
+		*lb_cmpval = 1;
+		*ub_cmpval = 1;
+		return false;
+	}
+
+	/* All other cases indicate overlapping partitions. */
+	*lb_cmpval = compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									  outer_lb, inner_lb);
+	*ub_cmpval = compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									  outer_ub, inner_ub);
+	return true;
+}
+
+/*
+ * get_merged_range_bounds
+ *		Given the bounds of range partitions to be join, determine the range
+ *		bounds of the merged partition produced from the range partitions
+ *
+ * *merged_lb and *merged_ub are set to the lower and upper bounds of the
+ * merged partition.
+ */
+static void
+get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations, JoinType jointype,
+						PartitionRangeBound *outer_lb,
+						PartitionRangeBound *outer_ub,
+						PartitionRangeBound *inner_lb,
+						PartitionRangeBound *inner_ub,
+						int	lb_cmpval, int ub_cmpval,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub)
+{
+	Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								outer_lb, inner_lb) == lb_cmpval);
+	Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								outer_ub, inner_ub) == ub_cmpval);
+
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			*merged_lb = (lb_cmpval > 0) ? *outer_lb : *inner_lb;
+			*merged_ub = (ub_cmpval < 0) ? *outer_ub : *inner_ub;
+			break;
+
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = *outer_ub;
+			*merged_lb = *outer_lb;
+			break;
+
+		case JOIN_FULL:
+			*merged_lb = (lb_cmpval < 0) ? *outer_lb : *inner_lb;
+			*merged_ub = (ub_cmpval > 0) ? *outer_ub : *inner_ub;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+}
+
+/*
+ * add_merged_range_bounds
+ *		Add the range bounds of a merged partition to the lists of range
+ *		bounds
+ */
+static void
+add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub,
+						int merged_index,
+						List **merged_datums,
+						List **merged_kinds,
+						List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		/* First merged partition */
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		Assert(*merged_kinds && *merged_indexes);
+
+		/* Get the last upper bound. */
+		prev_ub.index = llast_int(*merged_indexes);
+		prev_ub.datums = (Datum *) llast(*merged_datums);
+		prev_ub.kind = (PartitionRangeDatumKind *) llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		/*
+		 * We pass to partition_rbound_cmp() lower1 as false to prevent it
+		 * from considering the last upper bound to be smaller than the lower
+		 * bound of the merged partition when the values of the two range
+		 * bounds compare equal.
+		 */
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  merged_lb->datums, merged_lb->kind,
+									  false, &prev_ub);
+		Assert(cmpval >= 0);
+	}
+
+	/*
+	 * If the lower bound is higher than the last upper bound, add the lower
+	 * bound with the index as -1 indicating that that is a lower bound; else,
+	 * the last upper bound will be reused as the lower bound of the merged
+	 * partition, so skip this.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, merged_lb->datums);
+		*merged_kinds = lappend(*merged_kinds, merged_lb->kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	/* Add the upper bound and index of the merged partition. */
+	*merged_datums = lappend(*merged_datums, merged_ub->datums);
+	*merged_kinds = lappend(*merged_kinds, merged_ub->kind);
+	*merged_indexes = lappend_int(*merged_indexes, merged_index);
+}
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			outer_part;
+	int			inner_part;
+	int			outer_lb_index;
+	int			inner_lb_index;
+	PartitionRangeBound outer_lb;
+	PartitionRangeBound outer_ub;
+	PartitionRangeBound inner_lb;
+	PartitionRangeBound inner_ub;
+	int			next_index = 0;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_kinds = NIL;
+	List	   *merged_indexes = NIL;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = inner_lb_index = 0;
+	outer_part = get_range_partition(outer_bi, &outer_lb_index,
+									 &outer_lb, &outer_ub);
+	inner_part = get_range_partition(inner_bi, &inner_lb_index,
+									 &inner_lb, &inner_ub);
+	while (outer_part >= 0 || inner_part >= 0)
+	{
+		PartitionRangeBound merged_lb;
+		PartitionRangeBound merged_ub;
+		int			merged_index = -1;
+		bool		overlap;
+		int			ub_cmpval;
+		int			lb_cmpval;
+
+		if (outer_part >= 0)
+			Assert(outer_map.merged_indexes[outer_part] == -1 &&
+				   outer_map.merged[outer_part] == false);
+		if (inner_part >= 0)
+			Assert(inner_map.merged_indexes[inner_part] == -1 &&
+				   inner_map.merged[inner_part] == false);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (outer_part == -1)
+		{
+			overlap = false;
+			lb_cmpval = 1;
+			ub_cmpval = 1;
+		}
+		else if (inner_part == -1)
+		{
+			overlap = false;
+			lb_cmpval = -1;
+			ub_cmpval = -1;
+		}
+		else
+			overlap = compare_range_partitions(partnatts, partsupfuncs,
+											   partcollations,
+											   &outer_lb, &outer_ub,
+											   &inner_lb, &inner_ub,
+											   &lb_cmpval, &ub_cmpval);
+
+		if (overlap)
+		{
+			/* The ranges of partitions overlap; form a join pair of them. */
+
+			PartitionRangeBound save_outer_ub;
+			PartitionRangeBound save_inner_ub;
+
+			/* Get the bounds of the merged partition. */
+			get_merged_range_bounds(partnatts, partsupfuncs,
+									partcollations, jointype,
+									&outer_lb, &outer_ub,
+									&inner_lb, &inner_ub,
+									lb_cmpval, ub_cmpval,
+									&merged_lb, &merged_ub);
+
+			/*
+			 * Get the index of the merged partition.  Both partitions aren't
+			 * merged yet, so the partitions should be merged successfully.
+			 */
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
+													outer_part, inner_part,
+													&next_index);
+			Assert(merged_index >= 0);
+
+			/* Save the upper bounds of both partitions for use below. */
+			save_outer_ub = outer_ub;
+			save_inner_ub = inner_ub;
+
+			/* Move to the next pair of partitions. */
+			outer_part = get_range_partition(outer_bi, &outer_lb_index,
+											 &outer_lb, &outer_ub);
+			inner_part = get_range_partition(inner_bi, &inner_lb_index,
+											 &inner_lb, &inner_ub);
+
+			/*
+			 * If the range of a partition on one side overlaps the range of
+			 * the next partition on the other side, that will cause the
+			 * partition on one side to match at least two partitions on the
+			 * other side, which is the case that we currently don't support
+			 * partitioned join for; give up.
+			 */
+			if (ub_cmpval > 0 && inner_part >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &save_outer_ub, &inner_lb) > 0)
+				goto cleanup;
+			if (ub_cmpval < 0 && outer_part >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &outer_lb, &save_inner_ub) < 0)
+				goto cleanup;
+
+			/*
+			 * A row from a non-overlapping portion (if any) of a partition
+			 * on one side might find its join partner in the default
+			 * partition (if any) on the other side, causing the same
+			 * situation as above; if so, give up.
+			 */
+			if ((outer_has_default && (lb_cmpval > 0 || ub_cmpval < 0)) ||
+				(inner_has_default && (lb_cmpval < 0 || ub_cmpval > 0)))
+				goto cleanup;
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			merged_lb = outer_lb;
+			merged_ub = outer_ub;
+
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				merged_index = process_outer_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   outer_part,
+													   inner_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+			}
+
+			/* Move to the next partition on the outer side. */
+			outer_part = get_range_partition(outer_bi, &outer_lb_index,
+											 &outer_lb, &outer_ub);
+		}
+		else
+		{
+			/* Upper bound of outer range higher than that of the inner. */
+			Assert(ub_cmpval > 0);
+
+			merged_lb = inner_lb;
+			merged_ub = inner_ub;
+
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				merged_index = process_inner_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   inner_part,
+													   outer_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+			}
+
+			/* Move to the next partition on the inner side. */
+			inner_part = get_range_partition(inner_bi, &inner_lb_index,
+											 &inner_lb, &inner_ub);
+		}
+
+		if (merged_index >= 0 && merged_index != default_index)
+		{
+			/* Add the range bounds of the merged partition. */
+			add_merged_range_bounds(partnatts, partsupfuncs, partcollations,
+									&merged_lb, &merged_ub, merged_index,
+									&merged_datums, &merged_kinds,
+									&merged_indexes);
+		}
+	}
+
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+		merge_default_partitions(&outer_map, &inner_map,
+								 outer_has_default, inner_has_default,
+								 outer_default, inner_default,
+								 jointype, &next_index, &default_index);
+	else
+		Assert(default_index == -1);
+
+	/*
+	 * Unlike the case for list partitioning, we wouldn't have re-merged
+	 * partitions, so did_remapping should be left alone.
+	 */
+	Assert(!outer_map.did_remapping && !inner_map.did_remapping);
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 &outer_map, &inner_map,
+								 next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes,
+													  merged_kinds,
+													  -1, default_index);
+	}
+
+cleanup:
+	/* Free local memory before returning. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							JoinType jointype,
+							List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			next_index = 0;
+	int			null_index = -1;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			cnto;
+	int			cnti;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *merged_datum = NULL;
+		int			merged_index = -1;
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			cmpval;
+
+		/* Get the list datums of the next pair of partitions. */
+		odatums = cnto < outer_bi->ndatums ? outer_bi->datums[cnto] : NULL;
+		idatums = cnti < inner_bi->ndatums ? inner_bi->datums[cnti] : NULL;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+			cmpval = 1;
+		else if (cnti >= inner_bi->ndatums)
+			cmpval = -1;
+		else
+		{
+			Assert(odatums != NULL && idatums != NULL);
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0],
+													 idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			int			o_index = outer_bi->indexes[cnto];
+			int			i_index = inner_bi->indexes[cnti];
+
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			Assert(o_index >= 0 && i_index >= 0);
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
+													o_index, i_index,
+													&next_index);
+
+			if (merged_index == -1)
+				goto cleanup;
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			Assert(cnto < outer_bi->ndatums);
+
+			/* A datum missing from the inner side. */
+			merged_datum = odatums;
+
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				int			o_index = outer_bi->indexes[cnto];
+
+				Assert(o_index >= 0);
+				merged_index = process_outer_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   o_index,
+													   inner_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+			}
+
+			/* Move to the next datum on the outer side. */
+			cnto++;
+		}
+		else
+		{
+			Assert(cmpval > 0);
+			Assert(cnti < inner_bi->ndatums);
+
+			/* A datum missing from the outer side. */
+			merged_datum = idatums;
+
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				int			i_index = inner_bi->indexes[cnti];
+
+				Assert(i_index >= 0);
+				merged_index = process_inner_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   i_index,
+													   outer_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+			}
+
+			/* Move to the next datum on the inner side. */
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0 && merged_index != default_index)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	/* Merge null partitions if any. */
+	if (outer_has_null || inner_has_null)
+		merge_null_partitions(&outer_map, &inner_map,
+							  outer_has_null, inner_has_null,
+							  outer_bi->null_index, inner_bi->null_index,
+							  jointype, &next_index, &null_index);
+	else
+		Assert(null_index == -1);
+
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+		merge_default_partitions(&outer_map, &inner_map,
+								 outer_has_default, inner_has_default,
+								 outer_default, inner_default,
+								 jointype, &next_index, &default_index);
+	else
+		Assert(default_index == -1);
+
+	/* Fix the merged_indexes list if necessary. */
+	if (outer_map.did_remapping || inner_map.did_remapping)
+	{
+		Assert(jointype == JOIN_FULL);
+		fix_merged_indexes(&outer_map, &inner_map,
+						   next_index, merged_indexes);
+	}
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 &outer_map, &inner_map,
+								 next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) <= next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes, NIL,
+													  null_index, default_index);
+	}
+
+cleanup:
+	/* Free local memory before returning. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+	list_free(merged_datums);
+	list_free(merged_indexes);
+
+	return merged_bounds;
+}
+
+/*
+ * init_partition_map
+ *
+ * Initialize a PartitionMap struct for given relation.
+ */
+static void
+init_partition_map(RelOptInfo *rel, PartitionMap *map)
+{
+	int			nparts = rel->nparts;
+	int			i;
+
+	map->nparts = nparts;
+	map->merged_indexes = (int *) palloc(sizeof(int) * nparts);
+	map->merged = (bool *) palloc(sizeof(bool) * nparts);
+	map->did_remapping = false;
+	map->old_indexes = (int *) palloc(sizeof(int) * nparts);
+	for (i = 0; i < nparts; i++)
+	{
+		map->merged_indexes[i] = map->old_indexes[i] = -1;
+		map->merged[i] = false;
+	}
+}
+
+/*
+ * free_partition_map
+ */
+static void
+free_partition_map(PartitionMap *map)
+{
+	pfree(map->merged_indexes);
+	pfree(map->merged);
+	pfree(map->old_indexes);
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given partitions.
+ */
+static int
+map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_index, int inner_index, int *next_index)
+{
+	int 		outer_merged_index;
+	bool 		outer_merged;
+	int 		inner_merged_index;
+	bool 		inner_merged;
+
+	Assert(outer_index >= 0 && outer_index < outer_map->nparts);
+	outer_merged_index = outer_map->merged_indexes[outer_index];
+	outer_merged = outer_map->merged[outer_index];
+	Assert(inner_index >= 0 && inner_index < inner_map->nparts);
+	inner_merged_index = inner_map->merged_indexes[inner_index];
+	inner_merged = inner_map->merged[inner_index];
+
+	/*
+	 * Handle cases where both partitions are mapped to merged partitions.
+	 */
+	if (outer_merged_index >= 0 && inner_merged_index >= 0)
+	{
+		/*
+		 * If the mereged partitions are the same, no need to do anything;
+		 * return the index of the merged partition.  Otherwise, if both
+		 * partitions are merged with dummy partitions, re-merge them; map
+		 * them to the merged partition with the smaller of the two merged
+		 * indexes and return the smaller index.  Otherwise they can't be
+		 * merged, so return -1.
+		 */
+		if (outer_merged_index == inner_merged_index)
+		{
+			Assert(outer_merged);
+			Assert(inner_merged);
+			return outer_merged_index;
+		}
+		if (!outer_merged && !inner_merged)
+		{
+			/*
+			 * Note that we will fix the larger index that have been added to
+			 * the merged_indexes list so far in fix_merged_indexes().
+			 */
+			if (outer_merged_index < inner_merged_index)
+			{
+				outer_map->merged[outer_index] = true;
+				inner_map->merged_indexes[inner_index] = outer_merged_index;
+				inner_map->merged[inner_index] = true;
+				inner_map->did_remapping = true;
+				inner_map->old_indexes[inner_index] = inner_merged_index;
+				return outer_merged_index;
+			}
+			else
+			{
+				inner_map->merged[inner_index] = true;
+				outer_map->merged_indexes[outer_index] = inner_merged_index;
+				outer_map->merged[outer_index] = true;
+				outer_map->did_remapping = true;
+				outer_map->old_indexes[outer_index] = outer_merged_index;
+				return inner_merged_index;
+			}
+		}
+		return -1;
+	}
+
+	/* At least one partition isn't mapped to a merged partition. */
+	Assert(outer_merged_index == -1 || inner_merged_index == -1);
+
+	/*
+	 * If neither of partitions isn't mapped, assign them a new merged
+	 * partition and return the index of the merged partition.  Otherwise, if
+	 * one of partitions is merged with a dummy relation (and the other isn't
+	 * merged), re-merge it with the other, with the same index, and return
+	 * the index.  Otherwise they can't be merged, so return -1.
+	 */
+	if (outer_merged_index == -1 && inner_merged_index == -1)
+	{
+		int 		merged_index = *next_index;
+
+		Assert(!outer_merged);
+		Assert(!inner_merged);
+		outer_map->merged_indexes[outer_index] = merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged_indexes[inner_index] = merged_index;
+		inner_map->merged[inner_index] = true;
+		*next_index = *next_index + 1;
+		return merged_index;
+	}
+	if (outer_merged_index >= 0 && !outer_map->merged[outer_index])
+	{
+		Assert(inner_merged_index == -1);
+		Assert(!inner_merged);
+		inner_map->merged_indexes[inner_index] = outer_merged_index;
+		inner_map->merged[inner_index] = true;
+		outer_map->merged[outer_index] = true;
+		return outer_merged_index;
+	}
+	if (inner_merged_index >= 0 && !inner_map->merged[inner_index])
+	{
+		Assert(outer_merged_index == -1);
+		Assert(!outer_merged);
+		outer_map->merged_indexes[outer_index] = inner_merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged[inner_index] = true;
+		return inner_merged_index;
+	}
+	return -1;
+}
+
+/*
+ * merge_partition_with_dummy
+ *
+ * *next_index is incremented.
+ */
+static int
+merge_partition_with_dummy(PartitionMap *map, int index, int *next_index)
+{
+	int 		merged_index = *next_index;
+
+	Assert(index >= 0 && index < map->nparts);
+	Assert(map->merged_indexes[index] == -1);
+	Assert(!map->merged[index]);
+	map->merged_indexes[index] = merged_index;
+	/* Leave the merged flag alone! */
+	*next_index = *next_index + 1;
+	return merged_index;
+}
+
+/*
+ * process_outer_partition
+ *
+ * Determine the merged partition associated with the given outer partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given outer partition.
+ */
+static int
+process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index)
+{
+	int 		merged_index = -1;
+
+	Assert(outer_index >= 0);
+
+	/*
+	 * If the inner side has the default partition, the outer partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an outer join, in which case the outer partition has to be
+	 * scanned all the way anyway; if the outer partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the outer partition with a dummy partition.
+	 */
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0);
+
+		/*
+		 * If the outer side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (outer_has_default)
+			return -1;
+
+		merged_index = map_and_merge_partitions(outer_map, inner_map,
+												outer_index, inner_default,
+												next_index);
+		if (merged_index == -1)
+			return -1;
+
+		/*
+		 * If this is a FULL join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (*default_index == -1)
+				*default_index = merged_index;
+			else
+				Assert(*default_index == merged_index);
+		}
+	}
+	else
+	{
+		Assert(IS_OUTER_JOIN(jointype));
+		Assert(jointype != JOIN_RIGHT);
+
+		merged_index = outer_map->merged_indexes[outer_index];
+		if (merged_index == -1)
+			merged_index = merge_partition_with_dummy(outer_map, outer_index,
+													  next_index);
+	}
+	return merged_index;
+}
+
+/*
+ * process_inner_partition
+ *
+ * Determine the merged partition associated with the given inner partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given inner partition.
+ */
+static int
+process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index)
+{
+	int 		merged_index = -1;
+
+	Assert(inner_index >= 0);
+
+	/*
+	 * If the outer side has the default partition, the inner partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an FULL join, in which case the inner partition has to be
+	 * scanned all the way anyway; if the inner partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the inner partition with a dummy partition.
+	 */
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0);
+
+		/*
+		 * If the inner side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (inner_has_default)
+			return -1;
+
+		merged_index = map_and_merge_partitions(outer_map, inner_map,
+												outer_default, inner_index,
+												next_index);
+		if (merged_index == -1)
+			return -1;
+
+		/*
+		 * If this is an outer join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			if (*default_index == -1)
+				*default_index = merged_index;
+			else
+				Assert(*default_index == merged_index);
+		}
+	}
+	else
+	{
+		Assert(jointype == JOIN_FULL);
+
+		merged_index = inner_map->merged_indexes[inner_index];
+		if (merged_index == -1)
+			merged_index = merge_partition_with_dummy(inner_map, inner_index,
+													  next_index);
+	}
+	return merged_index;
+}
+
+/*
+ * fix_merged_indexes
+ */
+static void
+fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes)
+{
+	int		   *new_indexes;
+	int			merged_index;
+	int			i;
+	ListCell   *lc;
+
+	new_indexes = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		new_indexes[i] = -1;
+
+	/* Build the mapping of old merged indexes to new merged indexes. */
+	if (outer_map->did_remapping)
+	{
+		for (i = 0; i < outer_map->nparts; i++)
+		{
+			merged_index = outer_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = outer_map->merged_indexes[i];
+		}
+	}
+	if (inner_map->did_remapping)
+	{
+		for (i = 0; i < inner_map->nparts; i++)
+		{
+			merged_index = inner_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = inner_map->merged_indexes[i];
+		}
+	}
+
+	/* Fix the merged_indexes list using the mapping. */
+	foreach(lc, merged_indexes)
+	{
+		merged_index = lfirst_int(lc);
+		Assert(merged_index >= 0);
+		if (new_indexes[merged_index] >= 0)
+			lfirst_int(lc) = new_indexes[merged_index];
+	}
+
+	pfree(new_indexes);
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * This function produces the list pairs of partitions which produce merged
+ * partitions in the order of merged partition indexes.
+ *
+ * nmerged is the number of merged partitions.
+ *
+ * If successful, the list pairs are returned as two separate lists,
+ * outer_part_list and inner_part_list, one for each side.  Otherwise, those
+ * lists will be set to NIL.
+ */
+static void
+generate_matching_part_pairs(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 PartitionMap *outer_map, PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_part_list, List **inner_part_list)
+{
+	int			outer_nparts = outer_map->nparts;
+	int			inner_nparts = inner_map->nparts;
+	int		   *outer_part_array;
+	int		   *inner_part_array;
+	int			max_nparts;
+	int 		i;
+
+	Assert(outer_nparts == outer_rel->nparts);
+	Assert(inner_nparts == inner_rel->nparts);
+
+	outer_part_array = (int *) palloc(sizeof(int) * nmerged);
+	inner_part_array = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		outer_part_array[i] = inner_part_array[i] = -1;
+
+	/* Set pairs of matching partitions. */
+	max_nparts = Max(outer_nparts, inner_nparts);
+	for (i = 0; i < max_nparts; i++)
+	{
+		if (i < outer_nparts)
+		{
+			int 		merged_index = outer_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				outer_part_array[merged_index] = i;
+			}
+		}
+		if (i < inner_nparts)
+		{
+			int 		merged_index = inner_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				inner_part_array[merged_index] = i;
+			}
+		}
+	}
+
+	*outer_part_list = NIL;
+	*inner_part_list = NIL;
+
+	/* Build the list pairs. */
+	for (i = 0; i < nmerged; i++)
+	{
+		int			outer_part = outer_part_array[i];
+		int			inner_part = inner_part_array[i];
+
+		/*
+		 * If both partitions are dummy, it means the merged partition was
+		 * removed by partition re-merging in map_and_merge_partitions();
+		 * ignore the merged partition
+		 */
+		if (outer_part == -1 && inner_part == -1)
+			continue;
+
+		*outer_part_list = lappend(*outer_part_list, outer_part >= 0 ?
+								   outer_rel->part_rels[outer_part] : NULL);
+		*inner_part_list = lappend(*inner_part_list, inner_part >= 0 ?
+								   inner_rel->part_rels[inner_part] : NULL);
+	}
+
+	pfree(outer_part_array);
+	pfree(inner_part_array);
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * merge_default_partitions
+ *		Merge the default partitions from both sides, if any
+ *
+ * *default_index is set to the index of the default partition of the joinrel,
+ * if appropriate.
+ */
+static void
+merge_default_partitions(PartitionMap *outer_map,
+						 PartitionMap *inner_map,
+						 bool outer_has_default,
+						 bool inner_has_default,
+						 int outer_default,
+						 int inner_default,
+						 JoinType jointype,
+						 int *next_index,
+						 int *default_index)
+{
+	int 		outer_merged_index = -1;
+	int 		inner_merged_index = -1;
+
+	Assert(outer_has_default || inner_has_default);
+
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0 && outer_default < outer_map->nparts);
+		outer_merged_index = outer_map->merged_indexes[outer_default];
+	}
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0 && inner_default < inner_map->nparts);
+		inner_merged_index = inner_map->merged_indexes[inner_default];
+	}
+
+	if (outer_has_default && !inner_has_default)
+	{
+		/*
+		 * If this is an outer join, assign a new merged partition if not
+		 * already done.  The resulting partition should be the default
+		 * partition of the joinrel.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			if (outer_merged_index == -1)
+				*default_index = merge_partition_with_dummy(outer_map,
+															outer_default,
+															next_index);
+			else
+				Assert(*default_index == outer_merged_index);
+		}
+		else
+			Assert(*default_index == -1);
+	}
+	else if (!outer_has_default && inner_has_default)
+	{
+		/*
+		 * If this is a FULL join, assign a new merged partition if not
+		 * already done.  The resulting partition should be the default
+		 * partition of the joinrel.
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (inner_merged_index == -1)
+				*default_index = merge_partition_with_dummy(inner_map,
+															inner_default,
+															next_index);
+			else
+				Assert(*default_index == inner_merged_index);
+		}
+		else
+			Assert(*default_index == -1);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+		Assert(outer_merged_index == -1);
+		Assert(inner_merged_index == -1);
+		Assert(*default_index == -1);
+
+		/*
+		 * Merge the default partitions.  Both partitions aren't merged yet,
+		 * so the partitions should be merged successfully.  The resulting
+		 * partition should be the default partition of the joinrel.
+		 */
+		*default_index = map_and_merge_partitions(outer_map,
+												  inner_map,
+												  outer_default,
+												  inner_default,
+												  next_index);
+		Assert(*default_index >= 0);
+	}
+}
+
+/*
+ * merge_null_partitions
+ *		Merge the NULL partitions from both sides, if any
+ *
+ * *null_index is set to the index of the NULL partition of the joinrel, if
+ * appropriate.
+ */
+static void
+merge_null_partitions(PartitionMap *outer_map,
+					  PartitionMap *inner_map,
+					  bool outer_has_null,
+					  bool inner_has_null,
+					  int outer_null,
+					  int inner_null,
+					  JoinType jointype,
+					  int *next_index,
+					  int *null_index)
+{
+	bool 		outer_null_unmerged = false;
+	bool 		inner_null_unmerged = false;
+
+	Assert(outer_has_null || inner_has_null);
+	Assert(*null_index == -1);
+
+	/*
+	 * If this is an INNER or SEMI join, two NULL values won't be considered
+	 * to be equal, in which case we don't need to do anything.
+	 */
+	if (jointype == JOIN_INNER || jointype == JOIN_SEMI)
+		return;
+
+	Assert(IS_OUTER_JOIN(jointype));
+	Assert(jointype != JOIN_RIGHT);
+
+	if (outer_has_null)
+	{
+		/*
+		 * Since this is an outer join, if the NULL partition of the outer
+		 * side isn't merged yet, it means that the partition contains only
+		 * NULL values as the key values, in which case we need to create a
+		 * merged partition below.
+		 */
+		Assert(outer_null >= 0 && outer_null < outer_map->nparts);
+		if (outer_map->merged_indexes[outer_null] == -1)
+			outer_null_unmerged = true;
+	}
+	if (inner_has_null)
+	{
+		/*
+		 * If this is a FULL join, and the NULL partition of the inner side
+		 * isn't merged yet, it means that the partition contains only NULL
+		 * values as the key values, in which case we need to create a merged
+		 * partition below.
+		 */
+		Assert(inner_null >= 0 && inner_null < inner_map->nparts);
+		if (inner_map->merged_indexes[inner_null] == -1)
+			inner_null_unmerged = true;
+	}
+
+	if (!outer_null_unmerged && !inner_null_unmerged)
+		return;
+
+	if (outer_null_unmerged && !inner_null_unmerged)
+	{
+		Assert(outer_has_null);
+
+		/*
+		 * Assign a new merged partition.  The resulting partition should be
+		 * the NULL partition of the joinrel.
+		 */
+		*null_index = merge_partition_with_dummy(outer_map, outer_null,
+												 next_index);
+	}
+	else if (!outer_null_unmerged && inner_null_unmerged)
+	{
+		Assert(inner_has_null);
+
+		/*
+		 * Assign a new merged partition.  The resulting partition should be
+		 * the NULL partition of the joinrel.
+		 */
+		if (jointype == JOIN_FULL)
+			*null_index = merge_partition_with_dummy(inner_map, inner_null,
+													 next_index);
+	}
+	else
+	{
+		Assert(outer_null_unmerged && inner_null_unmerged);
+		Assert(outer_has_null);
+		Assert(inner_has_null);
+
+		/*
+		 * Merge the NULL partitions.  Both partitions aren't merged yet, so
+		 * the partitions should be merged successfully.  The resulting
+		 * partition should be the NULL partition of the joinrel.
+		 */
+		*null_index = map_and_merge_partitions(outer_map, inner_map,
+											   outer_null, inner_null,
+											   next_index);
+		Assert(*null_index >= 0);
+	}
+}
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index 3d3be197e0..1545877d8c 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -579,8 +579,10 @@ typedef struct PartitionSchemeData *PartitionScheme;
  *		part_scheme - Partitioning scheme of the relation
  *		nparts - Number of partitions
  *		boundinfo - Partition bounds
+ *		merged - true if partition bounds are merged ones
  *		partition_qual - Partition constraint if not the root
  *		part_rels - RelOptInfos for each partition
+ *		all_partrels - Relids set of all partition relids
  *		partexprs, nullable_partexprs - Partition key expressions
  *		partitioned_child_rels - RT indexes of unpruned partitions of
  *								 this relation that are partitioned tables
@@ -720,9 +722,12 @@ typedef struct RelOptInfo
 	PartitionScheme part_scheme;	/* Partitioning scheme. */
 	int			nparts;			/* number of partitions */
 	struct PartitionBoundInfoData *boundinfo;	/* Partition bounds */
+	bool		merged;			/* true if partition bounds were created by
+								 * partition_bounds_merge() */
 	List	   *partition_qual; /* partition constraint */
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of partitions,
 									 * stored in the same order of bounds */
+	Relids		all_partrels;	/* Relids set of all partition relids */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 	List	   *partitioned_child_rels; /* List of RT indexes. */
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index be940b5192..5442e843af 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -16,6 +16,7 @@
 #include "nodes/pg_list.h"
 #include "partitioning/partdefs.h"
 #include "utils/relcache.h"
+struct RelOptInfo;				/* avoid including pathnodes.h here */
 
 
 /*
@@ -108,5 +109,11 @@ extern int	partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 										  int nvalues, Datum *values, bool *is_equal);
 extern int	partition_hash_bsearch(PartitionBoundInfo boundinfo,
 								   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   struct RelOptInfo *outer_rel, struct RelOptInfo *inner_rel,
+					   JoinType jointype,
+					   List **outer_parts, List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index b3fbe47bde..924cb99863 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -2055,30 +2055,2116 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
                ->  Seq Scan on prt1_n_p2 t1_2
 (10 rows)
 
--- partitionwise join can not be applied if only one of joining tables has
--- default partition
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
-ANALYZE prt2;
+--
+-- Test advanced partition-matching algorithm for partitioned join
+--
+-- Tests for range-partitioned tables
+CREATE TABLE prt1_ad (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_ad_p1 PARTITION OF prt1_ad FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_ad_p2 PARTITION OF prt1_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_ad_p3 PARTITION OF prt1_ad FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_ad_a_idx on prt1_ad (a);
+INSERT INTO prt1_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_ad;
+CREATE TABLE prt2_ad (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_ad_p1 PARTITION OF prt2_ad FOR VALUES FROM (100) TO (150);
+CREATE TABLE prt2_ad_p2 PARTITION OF prt2_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt2_ad_p3 PARTITION OF prt2_ad FOR VALUES FROM (350) TO (500);
+CREATE INDEX prt2_ad_b_idx on prt2_ad (b);
+INSERT INTO prt2_ad_p1 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 149) i;
+INSERT INTO prt2_ad_p2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(200, 299) i;
+INSERT INTO prt2_ad_p3 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_ad;
+-- inner join
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                     QUERY PLAN                      
+-----------------------------------------------------
  Sort
    Sort Key: t1.a
-   ->  Hash Join
-         Hash Cond: (t2.b = t1.a)
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_ad_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_ad_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p2 t2_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_ad_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 100 | 0 | 0100
+ 125 | 0 | 0125
+ 200 | 0 | 0200
+ 225 | 0 | 0225
+ 250 | 0 | 0250
+ 275 | 0 | 0275
+ 350 | 0 | 0350
+ 375 | 0 | 0375
+(8 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 |     | 
+ 175 | 0175 |     | 
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 300 | 0300 |     | 
+ 325 | 0325 |     | 
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(12 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+                  QUERY PLAN                   
+-----------------------------------------------
+ Append
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.a = t2_1.b)
+         ->  Seq Scan on prt1_ad_p1 t1_1
+               Filter: (b = 0)
+         ->  Hash
+               ->  Seq Scan on prt2_ad_p1 t2_1
+   ->  Hash Anti Join
+         Hash Cond: (t1_2.a = t2_2.b)
+         ->  Seq Scan on prt1_ad_p2 t1_2
+               Filter: (b = 0)
+         ->  Hash
+               ->  Seq Scan on prt2_ad_p2 t2_2
+   ->  Hash Anti Join
+         Hash Cond: (t1_3.a = t2_3.b)
+         ->  Seq Scan on prt1_ad_p3 t1_3
+               Filter: (b = 0)
+         ->  Hash
+               ->  Seq Scan on prt2_ad_p3 t2_3
+(19 rows)
+
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+  a  | b |  c   
+-----+---+------
+ 150 | 0 | 0150
+ 175 | 0 | 0175
+ 300 | 0 | 0300
+ 325 | 0 | 0325
+(4 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Sort
+   Sort Key: prt1_ad.a, prt2_ad.b
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_ad_1.a = prt2_ad_1.b)
+               Filter: (((175) = prt1_ad_1.a) OR ((425) = prt2_ad_1.b))
+               ->  Seq Scan on prt1_ad_p1 prt1_ad_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p1 prt2_ad_1
+                           Filter: (a = 0)
+         ->  Hash Full Join
+               Hash Cond: (prt1_ad_2.a = prt2_ad_2.b)
+               Filter: (((175) = prt1_ad_2.a) OR ((425) = prt2_ad_2.b))
+               ->  Seq Scan on prt1_ad_p2 prt1_ad_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p2 prt2_ad_2
+                           Filter: (a = 0)
+         ->  Hash Full Join
+               Hash Cond: (prt2_ad_3.b = prt1_ad_3.a)
+               Filter: (((175) = prt1_ad_3.a) OR ((425) = prt2_ad_3.b))
+               ->  Seq Scan on prt2_ad_p3 prt2_ad_3
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p3 prt1_ad_3
+                           Filter: (b = 0)
+(27 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 175 | 0175 |     | 
+     |      | 425 | 0425
+(2 rows)
+
+-- Test cases where one side has an extra partition
+-- Add an extra partition to prt2_ad
+CREATE TABLE prt2_ad_extra PARTITION OF prt2_ad FOR VALUES FROM (500) TO (MAXVALUE);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(500, 599) i;
+ANALYZE prt2_ad;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_ad_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_ad_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p2 t2_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_ad_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 100 | 0 | 0100
+ 125 | 0 | 0125
+ 200 | 0 | 0200
+ 225 | 0 | 0225
+ 250 | 0 | 0250
+ 275 | 0 | 0275
+ 350 | 0 | 0350
+ 375 | 0 | 0375
+(8 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 |     | 
+ 175 | 0175 |     | 
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 300 | 0300 |     | 
+ 325 | 0325 |     | 
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(12 rows)
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c FROM prt2_ad t1 LEFT JOIN prt1_ad t2 ON (t1.b = t2.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t2.a
+   ->  Hash Right Join
+         Hash Cond: (t2.a = t1.b)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2_1
-               ->  Seq Scan on prt2_p2 t2_2
-               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt1_ad_p1 t2_1
+               ->  Seq Scan on prt1_ad_p2 t2_2
+               ->  Seq Scan on prt1_ad_p3 t2_3
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt2_ad_p1 t1_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_ad_p2 t1_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_ad_p3 t1_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_ad_extra t1_4
+                           Filter: (a = 0)
+(18 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+                  QUERY PLAN                   
+-----------------------------------------------
+ Append
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.a = t2_1.b)
+         ->  Seq Scan on prt1_ad_p1 t1_1
+               Filter: (b = 0)
+         ->  Hash
+               ->  Seq Scan on prt2_ad_p1 t2_1
+   ->  Hash Anti Join
+         Hash Cond: (t1_2.a = t2_2.b)
+         ->  Seq Scan on prt1_ad_p2 t1_2
+               Filter: (b = 0)
+         ->  Hash
+               ->  Seq Scan on prt2_ad_p2 t2_2
+   ->  Hash Anti Join
+         Hash Cond: (t1_3.a = t2_3.b)
+         ->  Seq Scan on prt1_ad_p3 t1_3
+               Filter: (b = 0)
+         ->  Hash
+               ->  Seq Scan on prt2_ad_p3 t2_3
+(19 rows)
+
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+  a  | b |  c   
+-----+---+------
+ 150 | 0 | 0150
+ 175 | 0 | 0175
+ 300 | 0 | 0300
+ 325 | 0 | 0325
+(4 rows)
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt1_ad t2 WHERE t1.b = t2.a) AND t1.a = 0;
+                  QUERY PLAN                   
+-----------------------------------------------
+ Hash Anti Join
+   Hash Cond: (t1.b = t2.a)
+   ->  Append
+         ->  Seq Scan on prt2_ad_p1 t1_1
+               Filter: (a = 0)
+         ->  Seq Scan on prt2_ad_p2 t1_2
+               Filter: (a = 0)
+         ->  Seq Scan on prt2_ad_p3 t1_3
+               Filter: (a = 0)
+         ->  Seq Scan on prt2_ad_extra t1_4
+               Filter: (a = 0)
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_ad_p1 t2_1
+               ->  Seq Scan on prt1_ad_p2 t2_2
+               ->  Seq Scan on prt1_ad_p3 t2_3
+(16 rows)
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: prt1_ad.a, prt2_ad.b
+   ->  Hash Full Join
+         Hash Cond: (prt2_ad.b = prt1_ad.a)
+         Filter: (((175) = prt1_ad.a) OR ((425) = prt2_ad.b))
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 prt2_ad_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_ad_p2 prt2_ad_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_ad_p3 prt2_ad_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_ad_extra prt2_ad_4
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_ad_p1 prt1_ad_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_ad_p2 prt1_ad_2
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_ad_p3 prt1_ad_3
                            Filter: (b = 0)
-(16 rows)
+(22 rows)
+
+-- 3-way join where not every pair of relations can do partitioned join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1_ad t1 RIGHT JOIN prt2_ad t2 ON (t1.a = t2.b) INNER JOIN prt1_ad t3 ON (t2.b = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.a, t3.c;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a, t3.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: (t1_1.a = t3_1.a)
+               ->  Hash Join
+                     Hash Cond: (t2_1.b = t1_1.a)
+                     ->  Seq Scan on prt2_ad_p1 t2_1
+                     ->  Hash
+                           ->  Seq Scan on prt1_ad_p1 t1_1
+                                 Filter: (b = 0)
+               ->  Index Scan using prt1_ad_p1_a_idx on prt1_ad_p1 t3_1
+                     Index Cond: (a = t2_1.b)
+         ->  Nested Loop
+               Join Filter: (t1_2.a = t3_2.a)
+               ->  Hash Join
+                     Hash Cond: (t2_2.b = t1_2.a)
+                     ->  Seq Scan on prt2_ad_p2 t2_2
+                     ->  Hash
+                           ->  Seq Scan on prt1_ad_p2 t1_2
+                                 Filter: (b = 0)
+               ->  Index Scan using prt1_ad_p2_a_idx on prt1_ad_p2 t3_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t2_3.b)
+               ->  Hash Join
+                     Hash Cond: (t3_3.a = t1_3.a)
+                     ->  Seq Scan on prt1_ad_p3 t3_3
+                     ->  Hash
+                           ->  Seq Scan on prt1_ad_p3 t1_3
+                                 Filter: (b = 0)
+               ->  Index Scan using prt2_ad_p3_b_idx on prt2_ad_p3 t2_3
+                     Index Cond: (b = t3_3.a)
+(33 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1_ad t1 RIGHT JOIN prt2_ad t2 ON (t1.a = t2.b) INNER JOIN prt1_ad t3 ON (t2.b = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.a, t3.c;
+  a  | a |  c   
+-----+---+------
+ 100 | 0 | 0100
+ 125 | 0 | 0125
+ 200 | 0 | 0200
+ 225 | 0 | 0225
+ 250 | 0 | 0250
+ 275 | 0 | 0275
+ 350 | 0 | 0350
+ 375 | 0 | 0375
+(8 rows)
+
+DROP TABLE prt2_ad_extra;
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in those cases
+-- Split prt2_ad_p3 into two partitions so that prt1_ad_p3 matches both
+-- partitions
+ALTER TABLE prt2_ad DETACH PARTITION prt2_ad_p3;
+CREATE TABLE prt2_ad_p3_350_375 PARTITION OF prt2_ad FOR VALUES FROM (350) TO (375);
+CREATE TABLE prt2_ad_p3_375_500 PARTITION OF prt2_ad FOR VALUES FROM (375) TO (500);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_ad;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Seq Scan on prt2_ad_p3_350_375 t2_3
+               ->  Seq Scan on prt2_ad_p3_375_500 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p3 t1_3
+                           Filter: (b = 0)
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                          
+-------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_ad_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_ad_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_ad_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_ad_p1 t2_1
+                     ->  Seq Scan on prt2_ad_p2 t2_2
+                     ->  Seq Scan on prt2_ad_p3_350_375 t2_3
+                     ->  Seq Scan on prt2_ad_p3_375_500 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Seq Scan on prt2_ad_p3_350_375 t2_3
+               ->  Seq Scan on prt2_ad_p3_375_500 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p3 t1_3
+                           Filter: (b = 0)
+(17 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Hash Anti Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_ad_p1 t1_1
+               Filter: (b = 0)
+         ->  Seq Scan on prt1_ad_p2 t1_2
+               Filter: (b = 0)
+         ->  Seq Scan on prt1_ad_p3 t1_3
+               Filter: (b = 0)
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Seq Scan on prt2_ad_p3_350_375 t2_3
+               ->  Seq Scan on prt2_ad_p3_375_500 t2_4
+(15 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: prt1_ad.a, prt2_ad.b
+   ->  Hash Full Join
+         Hash Cond: (prt2_ad.b = prt1_ad.a)
+         Filter: (((175) = prt1_ad.a) OR ((425) = prt2_ad.b))
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 prt2_ad_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_ad_p2 prt2_ad_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_ad_p3_350_375 prt2_ad_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_ad_p3_375_500 prt2_ad_4
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_ad_p1 prt1_ad_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p2 prt1_ad_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p3 prt1_ad_3
+                           Filter: (b = 0)
+(22 rows)
+
+DROP TABLE prt2_ad_p3_350_375;
+DROP TABLE prt2_ad_p3_375_500;
+-- Test default partitions
+-- Change prt1_ad_p1 to the default partition
+ALTER TABLE prt1_ad DETACH PARTITION prt1_ad_p1;
+ALTER TABLE prt1_ad ATTACH PARTITION prt1_ad_p1 DEFAULT;
+ANALYZE prt1_ad;
+CREATE TABLE prt2_ad_p3_300_400 PARTITION OF prt2_ad FOR VALUES FROM (300) TO (400);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(300, 399) i;
+ANALYZE prt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_3.a)
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p1 t1_3
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p2 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_3.b = t1_2.a)
+               ->  Seq Scan on prt2_ad_p3_300_400 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p3 t1_2
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 300 | 0300 | 300 | 0300
+ 325 | 0325 | 325 | 0325
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(10 rows)
+
+DROP TABLE prt2_ad_p3_300_400;
+-- Restore prt2_ad_p3
+ALTER TABLE prt2_ad ATTACH PARTITION prt2_ad_p3 FOR VALUES FROM (350) TO (500);
+ANALYZE prt2_ad;
+-- Partitioned join can't be applied because the default partition of prt1_ad
+-- prt1_ad_p1 matches prt2_ad_p1 and prt2_ad_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Seq Scan on prt2_ad_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_ad_p2 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p3 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p1 t1_3
+                           Filter: (b = 0)
+(16 rows)
+
+-- Change prt2_ad_p3 to the default partition
+ALTER TABLE prt2_ad DETACH PARTITION prt2_ad_p3;
+ALTER TABLE prt2_ad ATTACH PARTITION prt2_ad_p3 DEFAULT;
+ANALYZE prt2_ad;
+-- Partitioned join can't be applied because the default partition of prt1_ad
+-- prt1_ad_p1 matches prt2_ad_p1 and prt2_ad_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Seq Scan on prt2_ad_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_ad_p2 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p3 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p1 t1_3
+                           Filter: (b = 0)
+(16 rows)
+
+DROP TABLE prt1_ad_p3;
+ANALYZE prt1_ad;
+DROP TABLE prt2_ad_p3;
+ANALYZE prt2_ad;
+CREATE TABLE prt3_ad (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt3_ad_p1 PARTITION OF prt3_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt3_ad_p2 PARTITION OF prt3_ad FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt3_ad_a_idx on prt3_ad (a);
+INSERT INTO prt3_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(200, 399) i;
+ANALYZE prt3_ad;
+-- 3-way join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) LEFT JOIN prt3_ad t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+                           QUERY PLAN                            
+-----------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b, t3.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t3_1.a = t1_1.a)
+               ->  Seq Scan on prt3_ad_p1 t3_1
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_2.b = t1_1.a)
+                           ->  Seq Scan on prt2_ad_p2 t2_2
+                           ->  Hash
+                                 ->  Seq Scan on prt1_ad_p2 t1_1
+                                       Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t3_2.a = t1_2.a)
+               ->  Seq Scan on prt3_ad_p2 t3_2
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_1.b = t1_2.a)
+                           ->  Seq Scan on prt2_ad_p1 t2_1
+                           ->  Hash
+                                 ->  Seq Scan on prt1_ad_p1 t1_2
+                                       Filter: (b = 0)
+(23 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) LEFT JOIN prt3_ad t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+  a  |  c   |  b  |  c   |  a  |  c   
+-----+------+-----+------+-----+------
+ 100 | 0100 | 100 | 0100 |     | 
+ 125 | 0125 | 125 | 0125 |     | 
+ 150 | 0150 |     |      |     | 
+ 175 | 0175 |     |      |     | 
+ 200 | 0200 | 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275 | 275 | 0275
+(8 rows)
+
+-- Tests for list-partitioned tables
+CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0001', '0003');
+CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt1_ad_p3 PARTITION OF plt1_ad FOR VALUES IN ('0008', '0009');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 NOT IN (0, 2, 5, 7);
+ANALYZE plt1_ad;
+CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt2_ad_p3 PARTITION OF plt2_ad FOR VALUES IN ('0007', '0009');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 NOT IN (0, 1, 5, 8);
+ANALYZE plt2_ad;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_ad_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+(6 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_ad_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 1 | 1 | 0001
+ 8 | 8 | 0008
+(2 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p3 t2_3
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+   |      | 2 | 0002
+   |      | 7 | 0007
+(8 rows)
+
+-- Test cases where one side has an extra partition
+-- Add an extra partition to plt2_ad
+CREATE TABLE plt2_ad_extra PARTITION OF plt2_ad FOR VALUES IN ('0000');
+INSERT INTO plt2_ad_extra VALUES (0, 0, '0000');
+ANALYZE plt2_ad;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_ad_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+(6 rows)
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2_ad t1 LEFT JOIN plt1_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt1_ad_p1 t2_1
+               ->  Seq Scan on plt1_ad_p2 t2_2
+               ->  Seq Scan on plt1_ad_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_ad_extra t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_ad_p1 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_ad_p2 t1_3
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_ad_p3 t1_4
+                           Filter: (b < 10)
+(18 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_ad_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 1 | 1 | 0001
+ 8 | 8 | 0008
+(2 rows)
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt2_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt1_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Anti Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt2_ad_extra t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p1 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3 t1_4
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p1 t2_1
+                     ->  Seq Scan on plt1_ad_p2 t2_2
+                     ->  Seq Scan on plt1_ad_p3 t2_3
+(18 rows)
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt2_ad_extra t2_1
+               ->  Seq Scan on plt2_ad_p1 t2_2
+               ->  Seq Scan on plt2_ad_p2 t2_3
+               ->  Seq Scan on plt2_ad_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+(15 rows)
+
+DROP TABLE plt2_ad_extra;
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in those cases
+-- Split plt2_ad_p2 into two partitions so that prt1_ad_p2 matches both
+-- partitions
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2;
+CREATE TABLE plt2_ad_p2_0004 PARTITION OF plt2_ad FOR VALUES IN ('0004');
+CREATE TABLE plt2_ad_p2_0006 PARTITION OF plt2_ad FOR VALUES IN ('0006');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6);
+ANALYZE plt2_ad;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Seq Scan on plt2_ad_p2_0004 t2_2
+               ->  Seq Scan on plt2_ad_p2_0006 t2_3
+               ->  Seq Scan on plt2_ad_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                        QUERY PLAN                        
+----------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Semi Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt1_ad_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+                     ->  Seq Scan on plt2_ad_p2_0004 t2_2
+                     ->  Seq Scan on plt2_ad_p2_0006 t2_3
+                     ->  Seq Scan on plt2_ad_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Seq Scan on plt2_ad_p2_0004 t2_2
+               ->  Seq Scan on plt2_ad_p2_0006 t2_3
+               ->  Seq Scan on plt2_ad_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(17 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                        QUERY PLAN                        
+----------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Anti Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt1_ad_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+                     ->  Seq Scan on plt2_ad_p2_0004 t2_2
+                     ->  Seq Scan on plt2_ad_p2_0006 t2_3
+                     ->  Seq Scan on plt2_ad_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Seq Scan on plt2_ad_p2_0004 t2_2
+               ->  Seq Scan on plt2_ad_p2_0006 t2_3
+               ->  Seq Scan on plt2_ad_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+(15 rows)
+
+DROP TABLE plt2_ad_p2_0004;
+DROP TABLE plt2_ad_p2_0006;
+-- Restore plt2_ad_p2
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_ad;
+-- Test NULL partitions
+-- Change plt1_ad_p1 to the NULL partition
+ALTER TABLE plt1_ad DETACH PARTITION plt1_ad_p1;
+CREATE TABLE plt1_ad_p1_null PARTITION OF plt1_ad FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt1_ad VALUES (-1, -1, NULL);
+ANALYZE plt1_ad;
+-- Change plt2_ad_p3 to the NULL partition
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p3;
+CREATE TABLE plt2_ad_p3_null PARTITION OF plt2_ad FOR VALUES IN (NULL, '0007', '0009');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (7, 9);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1_null t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_ad_p1_null t1_1
+                     Filter: (b < 10)
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3_null t2_3
+(19 rows)
+
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1_null t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a  |  c   | a |  c   
+----+------+---+------
+ -1 |      |   | 
+  1 | 0001 |   | 
+  3 | 0003 | 3 | 0003
+  4 | 0004 | 4 | 0004
+  6 | 0006 | 6 | 0006
+  8 | 0008 |   | 
+  9 | 0009 | 9 | 0009
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_ad_p1_null t1_1
+                     Filter: (b < 10)
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3_null t2_3
+(19 rows)
+
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a  | b  |  c   
+----+----+------
+ -1 | -1 | 
+  1 |  1 | 0001
+  8 |  8 | 0008
+(3 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p1_null t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt2_ad_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+ -1 |      |    | 
+  1 | 0001 |    | 
+  3 | 0003 |  3 | 0003
+  4 | 0004 |  4 | 0004
+  6 | 0006 |  6 | 0006
+  8 | 0008 |    | 
+  9 | 0009 |  9 | 0009
+    |      | -1 | 
+    |      |  2 | 0002
+    |      |  7 | 0007
+(10 rows)
+
+DROP TABLE plt1_ad_p1_null;
+DROP TABLE plt2_ad_p3_null;
+-- Restore plt1_ad_p1
+ALTER TABLE plt1_ad ATTACH PARTITION plt1_ad_p1 FOR VALUES IN ('0001', '0003');
+-- Add to plt1_ad the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt1_ad_extra PARTITION OF plt1_ad FOR VALUES IN (NULL);
+INSERT INTO plt1_ad VALUES (-1, -1, NULL);
+ANALYZE plt1_ad;
+-- Restore plt2_ad_p3
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p3 FOR VALUES IN ('0007', '0009');
+ANALYZE plt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- Partitioned join can't be applied because there isn't any partition on the
+-- plt2_ad side that matches the NULL partition of plt1_ad plt1_ad_extra
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Seq Scan on plt2_ad_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_extra t1_4
+                           Filter: (b < 10)
+(18 rows)
+
+-- Partitioned join can't be applied because there isn't any partition on the
+-- plt2_ad side that matches the NULL partition of plt1_ad plt1_ad_extra
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt1_ad_p1 t1_1
+               ->  Seq Scan on plt1_ad_p2 t1_2
+               ->  Seq Scan on plt1_ad_p3 t1_3
+               ->  Seq Scan on plt1_ad_extra t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+                     ->  Seq Scan on plt2_ad_p2 t2_2
+                     ->  Seq Scan on plt2_ad_p3 t2_3
+(15 rows)
+
+-- Add to plt2_ad the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt2_ad_extra PARTITION OF plt2_ad FOR VALUES IN (NULL);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+         ->  Nested Loop Left Join
+               Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+               ->  Seq Scan on plt1_ad_extra t1_4
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_extra t2_4
+(26 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a  |  c   | a |  c   
+----+------+---+------
+ -1 |      |   | 
+  1 | 0001 |   | 
+  3 | 0003 | 3 | 0003
+  4 | 0004 | 4 | 0004
+  6 | 0006 | 6 | 0006
+  8 | 0008 |   | 
+  9 | 0009 | 9 | 0009
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p3 t2_3
+         ->  Hash Full Join
+               Hash Cond: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+               Filter: ((COALESCE(t1_4.b, 0) < 10) AND (COALESCE(t2_4.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_extra t1_4
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_extra t2_4
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+ -1 |      |    | 
+  1 | 0001 |    | 
+  3 | 0003 |  3 | 0003
+  4 | 0004 |  4 | 0004
+  6 | 0006 |  6 | 0006
+  8 | 0008 |    | 
+  9 | 0009 |  9 | 0009
+    |      | -1 | 
+    |      |  2 | 0002
+    |      |  7 | 0007
+(10 rows)
+
+DROP TABLE plt1_ad_extra;
+DROP TABLE plt2_ad_extra;
+-- Test default partitions
+-- Change plt1_ad_p1 to the default partition
+ALTER TABLE plt1_ad DETACH PARTITION plt1_ad_p1;
+ALTER TABLE plt1_ad ATTACH PARTITION plt1_ad_p1 DEFAULT;
+DROP TABLE plt1_ad_p3;
+ANALYZE plt1_ad;
+DROP TABLE plt2_ad_p3;
+ANALYZE plt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_1
+                           Filter: (b < 10)
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(3 rows)
+
+-- Change plt2_ad_p2 to contain '0005' in addition to '0004' and '0006' as
+-- the key values
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2;
+CREATE TABLE plt2_ad_p2_ext PARTITION OF plt2_ad FOR VALUES IN ('0004', '0005', '0006');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 5, 6);
+ANALYZE plt2_ad;
+-- Partitioned join can't be applied because the default partition of plt1_ad
+-- plt1_ad_p1 matches plt2_ad_p1 and plt2_ad_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Seq Scan on plt2_ad_p2_ext t2_2
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p2 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p1 t1_2
+                           Filter: (b < 10)
+(13 rows)
+
+-- Change plt2_ad_p2_ext to the default partition
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2_ext;
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2_ext DEFAULT;
+ANALYZE plt2_ad;
+-- Partitioned join can't be applied because the default partition of plt1_ad
+-- plt1_ad_p1 matches plt2_ad_p1 and plt2_ad_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Seq Scan on plt2_ad_p2_ext t2_2
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p2 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p1 t1_2
+                           Filter: (b < 10)
+(13 rows)
+
+DROP TABLE plt2_ad_p2_ext;
+-- Restore plt2_ad_p2
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_ad;
+CREATE TABLE plt3_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_ad_p1 PARTITION OF plt3_ad FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt3_ad_p2 PARTITION OF plt3_ad FOR VALUES IN ('0007', '0009');
+INSERT INTO plt3_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6, 7, 9);
+ANALYZE plt3_ad;
+-- 3-way join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_ad t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t3_1.a = t1_1.a) AND (t3_1.c = t1_1.c))
+               ->  Seq Scan on plt3_ad_p1 t3_1
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+                           ->  Seq Scan on plt2_ad_p2 t2_2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_ad_p2 t1_1
+                                       Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t3_2.a = t1_2.a) AND (t3_2.c = t1_2.c))
+               ->  Seq Scan on plt3_ad_p2 t3_2
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+                           ->  Seq Scan on plt2_ad_p1 t2_1
+                           ->  Hash
+                                 ->  Seq Scan on plt1_ad_p1 t1_2
+                                       Filter: (b < 10)
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_ad t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   | a |  c   
+---+------+---+------+---+------
+ 1 | 0001 |   |      |   | 
+ 3 | 0003 | 3 | 0003 |   | 
+ 4 | 0004 | 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006 | 6 | 0006
+(4 rows)
+
+-- Test cases where one side has the default partition while the other side
+-- has the NULL partition
+-- Change plt2_ad_p1 to the NULL partition
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p1;
+CREATE TABLE plt2_ad_p1_null PARTITION OF plt2_ad FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p1_null t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_1
+                           Filter: (b < 10)
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 | 1 | 0001
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(4 rows)
+
+-- Change the NULL partition to contain only NULL values as the key values
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p1_null;
+CREATE TABLE plt2_ad_p1_nullonly PARTITION OF plt2_ad FOR VALUES IN (NULL);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Seq Scan on plt2_ad_p2 t2
+         ->  Hash
+               ->  Seq Scan on plt1_ad_p2 t1
+                     Filter: (b < 10)
+(8 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(2 rows)
+
+DROP TABLE plt2_ad_p1_nullonly;
+DROP TABLE plt2_ad_p1_null;
+DROP TABLE plt2_ad_p1;
+DROP TABLE plt1_ad;
+DROP TABLE plt2_ad;
+DROP TABLE plt3_ad;
+CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1_ad;
+CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0002');
+CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2_ad;
+CREATE TABLE plt3_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_ad_p1 PARTITION OF plt3_ad FOR VALUES IN ('0001');
+CREATE TABLE plt3_ad_p2 PARTITION OF plt3_ad FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.c = t2.c)) FULL JOIN plt3_ad t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2.a, t3.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t3_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_1.c = t2_1.c)
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                     ->  Hash
+                           ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt3_ad_p1 t3_1
+         ->  Hash Full Join
+               Hash Cond: (t1_2.c = t3_2.c)
+               Filter: (((COALESCE(t1_2.a, 0) % 5) <> 3) AND ((COALESCE(t1_2.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_2.c = t2_2.c)
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                     ->  Hash
+                           ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt3_ad_p2 t3_2
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.c = t2.c)) FULL JOIN plt3_ad t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+ a  |  c   | a  |  c   | a  |  c   
+----+------+----+------+----+------
+  0 | 0000 |    |      |    | 
+  5 | 0000 |    |      |    | 
+ 10 | 0000 |    |      |    | 
+ 15 | 0000 |    |      |    | 
+ 20 | 0000 |    |      |    | 
+  1 | 0001 |    |      |  1 | 0001
+  1 | 0001 |    |      |  6 | 0001
+  1 | 0001 |    |      | 11 | 0001
+  1 | 0001 |    |      | 16 | 0001
+  1 | 0001 |    |      | 21 | 0001
+  6 | 0001 |    |      |  1 | 0001
+  6 | 0001 |    |      |  6 | 0001
+  6 | 0001 |    |      | 11 | 0001
+  6 | 0001 |    |      | 16 | 0001
+  6 | 0001 |    |      | 21 | 0001
+ 11 | 0001 |    |      |  1 | 0001
+ 11 | 0001 |    |      |  6 | 0001
+ 11 | 0001 |    |      | 11 | 0001
+ 11 | 0001 |    |      | 16 | 0001
+ 11 | 0001 |    |      | 21 | 0001
+ 16 | 0001 |    |      |  1 | 0001
+ 16 | 0001 |    |      |  6 | 0001
+ 16 | 0001 |    |      | 11 | 0001
+ 16 | 0001 |    |      | 16 | 0001
+ 16 | 0001 |    |      | 21 | 0001
+ 21 | 0001 |    |      |  1 | 0001
+ 21 | 0001 |    |      |  6 | 0001
+ 21 | 0001 |    |      | 11 | 0001
+ 21 | 0001 |    |      | 16 | 0001
+ 21 | 0001 |    |      | 21 | 0001
+  2 | 0002 |  2 | 0002 |    | 
+  2 | 0002 |  7 | 0002 |    | 
+  2 | 0002 | 12 | 0002 |    | 
+  2 | 0002 | 17 | 0002 |    | 
+  2 | 0002 | 22 | 0002 |    | 
+  7 | 0002 |  2 | 0002 |    | 
+  7 | 0002 |  7 | 0002 |    | 
+  7 | 0002 | 12 | 0002 |    | 
+  7 | 0002 | 17 | 0002 |    | 
+  7 | 0002 | 22 | 0002 |    | 
+ 12 | 0002 |  2 | 0002 |    | 
+ 12 | 0002 |  7 | 0002 |    | 
+ 12 | 0002 | 12 | 0002 |    | 
+ 12 | 0002 | 17 | 0002 |    | 
+ 12 | 0002 | 22 | 0002 |    | 
+ 17 | 0002 |  2 | 0002 |    | 
+ 17 | 0002 |  7 | 0002 |    | 
+ 17 | 0002 | 12 | 0002 |    | 
+ 17 | 0002 | 17 | 0002 |    | 
+ 17 | 0002 | 22 | 0002 |    | 
+ 22 | 0002 |  2 | 0002 |    | 
+ 22 | 0002 |  7 | 0002 |    | 
+ 22 | 0002 | 12 | 0002 |    | 
+ 22 | 0002 | 17 | 0002 |    | 
+ 22 | 0002 | 22 | 0002 |    | 
+(55 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 575ba7b8d4..418ffa117f 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -445,11 +445,512 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
 
--- partitionwise join can not be applied if only one of joining tables has
--- default partition
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
-ANALYZE prt2;
 
+--
+-- Test advanced partition-matching algorithm for partitioned join
+--
+
+-- Tests for range-partitioned tables
+CREATE TABLE prt1_ad (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_ad_p1 PARTITION OF prt1_ad FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_ad_p2 PARTITION OF prt1_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_ad_p3 PARTITION OF prt1_ad FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_ad_a_idx on prt1_ad (a);
+INSERT INTO prt1_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_ad;
+
+CREATE TABLE prt2_ad (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_ad_p1 PARTITION OF prt2_ad FOR VALUES FROM (100) TO (150);
+CREATE TABLE prt2_ad_p2 PARTITION OF prt2_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt2_ad_p3 PARTITION OF prt2_ad FOR VALUES FROM (350) TO (500);
+CREATE INDEX prt2_ad_b_idx on prt2_ad (b);
+INSERT INTO prt2_ad_p1 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 149) i;
+INSERT INTO prt2_ad_p2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(200, 299) i;
+INSERT INTO prt2_ad_p3 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_ad;
+
+-- inner join
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+-- Test cases where one side has an extra partition
+
+-- Add an extra partition to prt2_ad
+CREATE TABLE prt2_ad_extra PARTITION OF prt2_ad FOR VALUES FROM (500) TO (MAXVALUE);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(500, 599) i;
+ANALYZE prt2_ad;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c FROM prt2_ad t1 LEFT JOIN prt1_ad t2 ON (t1.b = t2.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt1_ad t2 WHERE t1.b = t2.a) AND t1.a = 0;
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+-- 3-way join where not every pair of relations can do partitioned join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1_ad t1 RIGHT JOIN prt2_ad t2 ON (t1.a = t2.b) INNER JOIN prt1_ad t3 ON (t2.b = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1_ad t1 RIGHT JOIN prt2_ad t2 ON (t1.a = t2.b) INNER JOIN prt1_ad t3 ON (t2.b = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.a, t3.c;
+
+DROP TABLE prt2_ad_extra;
+
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in those cases
+
+-- Split prt2_ad_p3 into two partitions so that prt1_ad_p3 matches both
+-- partitions
+ALTER TABLE prt2_ad DETACH PARTITION prt2_ad_p3;
+CREATE TABLE prt2_ad_p3_350_375 PARTITION OF prt2_ad FOR VALUES FROM (350) TO (375);
+CREATE TABLE prt2_ad_p3_375_500 PARTITION OF prt2_ad FOR VALUES FROM (375) TO (500);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_ad;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+DROP TABLE prt2_ad_p3_350_375;
+DROP TABLE prt2_ad_p3_375_500;
+
+-- Test default partitions
+
+-- Change prt1_ad_p1 to the default partition
+ALTER TABLE prt1_ad DETACH PARTITION prt1_ad_p1;
+ALTER TABLE prt1_ad ATTACH PARTITION prt1_ad_p1 DEFAULT;
+ANALYZE prt1_ad;
+
+CREATE TABLE prt2_ad_p3_300_400 PARTITION OF prt2_ad FOR VALUES FROM (300) TO (400);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(300, 399) i;
+ANALYZE prt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt2_ad_p3_300_400;
+
+-- Restore prt2_ad_p3
+ALTER TABLE prt2_ad ATTACH PARTITION prt2_ad_p3 FOR VALUES FROM (350) TO (500);
+ANALYZE prt2_ad;
+
+-- Partitioned join can't be applied because the default partition of prt1_ad
+-- prt1_ad_p1 matches prt2_ad_p1 and prt2_ad_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- Change prt2_ad_p3 to the default partition
+ALTER TABLE prt2_ad DETACH PARTITION prt2_ad_p3;
+ALTER TABLE prt2_ad ATTACH PARTITION prt2_ad_p3 DEFAULT;
+ANALYZE prt2_ad;
+
+-- Partitioned join can't be applied because the default partition of prt1_ad
+-- prt1_ad_p1 matches prt2_ad_p1 and prt2_ad_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt1_ad_p3;
+ANALYZE prt1_ad;
+
+DROP TABLE prt2_ad_p3;
+ANALYZE prt2_ad;
+
+CREATE TABLE prt3_ad (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt3_ad_p1 PARTITION OF prt3_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt3_ad_p2 PARTITION OF prt3_ad FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt3_ad_a_idx on prt3_ad (a);
+INSERT INTO prt3_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(200, 399) i;
+ANALYZE prt3_ad;
+
+-- 3-way join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) LEFT JOIN prt3_ad t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) LEFT JOIN prt3_ad t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+
+
+-- Tests for list-partitioned tables
+CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0001', '0003');
+CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt1_ad_p3 PARTITION OF plt1_ad FOR VALUES IN ('0008', '0009');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 NOT IN (0, 2, 5, 7);
+ANALYZE plt1_ad;
+
+CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt2_ad_p3 PARTITION OF plt2_ad FOR VALUES IN ('0007', '0009');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 NOT IN (0, 1, 5, 8);
+ANALYZE plt2_ad;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+-- Test cases where one side has an extra partition
+
+-- Add an extra partition to plt2_ad
+CREATE TABLE plt2_ad_extra PARTITION OF plt2_ad FOR VALUES IN ('0000');
+INSERT INTO plt2_ad_extra VALUES (0, 0, '0000');
+ANALYZE plt2_ad;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2_ad t1 LEFT JOIN plt1_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt2_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt1_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt2_ad_extra;
+
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in those cases
+
+-- Split plt2_ad_p2 into two partitions so that prt1_ad_p2 matches both
+-- partitions
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2;
+CREATE TABLE plt2_ad_p2_0004 PARTITION OF plt2_ad FOR VALUES IN ('0004');
+CREATE TABLE plt2_ad_p2_0006 PARTITION OF plt2_ad FOR VALUES IN ('0006');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6);
+ANALYZE plt2_ad;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt2_ad_p2_0004;
+DROP TABLE plt2_ad_p2_0006;
+
+-- Restore plt2_ad_p2
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_ad;
+
+-- Test NULL partitions
+
+-- Change plt1_ad_p1 to the NULL partition
+ALTER TABLE plt1_ad DETACH PARTITION plt1_ad_p1;
+CREATE TABLE plt1_ad_p1_null PARTITION OF plt1_ad FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt1_ad VALUES (-1, -1, NULL);
+ANALYZE plt1_ad;
+
+-- Change plt2_ad_p3 to the NULL partition
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p3;
+CREATE TABLE plt2_ad_p3_null PARTITION OF plt2_ad FOR VALUES IN (NULL, '0007', '0009');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (7, 9);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt1_ad_p1_null;
+DROP TABLE plt2_ad_p3_null;
+
+-- Restore plt1_ad_p1
+ALTER TABLE plt1_ad ATTACH PARTITION plt1_ad_p1 FOR VALUES IN ('0001', '0003');
+
+-- Add to plt1_ad the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt1_ad_extra PARTITION OF plt1_ad FOR VALUES IN (NULL);
+INSERT INTO plt1_ad VALUES (-1, -1, NULL);
+ANALYZE plt1_ad;
+
+-- Restore plt2_ad_p3
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p3 FOR VALUES IN ('0007', '0009');
+ANALYZE plt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Partitioned join can't be applied because there isn't any partition on the
+-- plt2_ad side that matches the NULL partition of plt1_ad plt1_ad_extra
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Partitioned join can't be applied because there isn't any partition on the
+-- plt2_ad side that matches the NULL partition of plt1_ad plt1_ad_extra
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+-- Add to plt2_ad the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt2_ad_extra PARTITION OF plt2_ad FOR VALUES IN (NULL);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt1_ad_extra;
+DROP TABLE plt2_ad_extra;
+
+-- Test default partitions
+
+-- Change plt1_ad_p1 to the default partition
+ALTER TABLE plt1_ad DETACH PARTITION plt1_ad_p1;
+ALTER TABLE plt1_ad ATTACH PARTITION plt1_ad_p1 DEFAULT;
+DROP TABLE plt1_ad_p3;
+ANALYZE plt1_ad;
+
+DROP TABLE plt2_ad_p3;
+ANALYZE plt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Change plt2_ad_p2 to contain '0005' in addition to '0004' and '0006' as
+-- the key values
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2;
+CREATE TABLE plt2_ad_p2_ext PARTITION OF plt2_ad FOR VALUES IN ('0004', '0005', '0006');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 5, 6);
+ANALYZE plt2_ad;
+
+-- Partitioned join can't be applied because the default partition of plt1_ad
+-- plt1_ad_p1 matches plt2_ad_p1 and plt2_ad_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Change plt2_ad_p2_ext to the default partition
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2_ext;
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2_ext DEFAULT;
+ANALYZE plt2_ad;
+
+-- Partitioned join can't be applied because the default partition of plt1_ad
+-- plt1_ad_p1 matches plt2_ad_p1 and plt2_ad_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt2_ad_p2_ext;
+
+-- Restore plt2_ad_p2
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_ad;
+
+CREATE TABLE plt3_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_ad_p1 PARTITION OF plt3_ad FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt3_ad_p2 PARTITION OF plt3_ad FOR VALUES IN ('0007', '0009');
+INSERT INTO plt3_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6, 7, 9);
+ANALYZE plt3_ad;
+
+-- 3-way join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_ad t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_ad t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Test cases where one side has the default partition while the other side
+-- has the NULL partition
+
+-- Change plt2_ad_p1 to the NULL partition
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p1;
+CREATE TABLE plt2_ad_p1_null PARTITION OF plt2_ad FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Change the NULL partition to contain only NULL values as the key values
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p1_null;
+CREATE TABLE plt2_ad_p1_nullonly PARTITION OF plt2_ad FOR VALUES IN (NULL);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt2_ad_p1_nullonly;
+DROP TABLE plt2_ad_p1_null;
+DROP TABLE plt2_ad_p1;
+
+DROP TABLE plt1_ad;
+DROP TABLE plt2_ad;
+DROP TABLE plt3_ad;
+
+
+CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1_ad;
+
+CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0002');
+CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2_ad;
+
+CREATE TABLE plt3_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_ad_p1 PARTITION OF plt3_ad FOR VALUES IN ('0001');
+CREATE TABLE plt3_ad_p2 PARTITION OF plt3_ad FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.c = t2.c)) FULL JOIN plt3_ad t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.c = t2.c)) FULL JOIN plt3_ad t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
-- 
2.21.1 (Apple Git-122.3)

v31-0004-Consider-pruned-partitions.patchapplication/octet-stream; name=v31-0004-Consider-pruned-partitions.patchDownload
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 43c4e03dde..3c7b6030c0 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -136,6 +136,7 @@ static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid
 							List **outer_parts, List **inner_parts);
 static void init_partition_map(RelOptInfo *rel, PartitionMap *map);
 static void free_partition_map(PartitionMap *map);
+static bool is_dummy_partition(RelOptInfo *rel, int part_index);
 static int map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
 						 int outer_part, int inner_part, int *next_index);
 static int merge_partition_with_dummy(PartitionMap *map, int index,
@@ -171,8 +172,15 @@ static PartitionBoundInfo build_merged_partition_bounds(char strategy,
 							  List *merged_datums, List *merged_indexes,
 							  List *merged_contents, int null_index,
 							  int default_index);
-static int get_range_partition(PartitionBoundInfo bi, int *lb_pos,
-					PartitionRangeBound *lb, PartitionRangeBound *ub);
+static int get_range_partition(RelOptInfo *rel,
+							   PartitionBoundInfo bi,
+							   int *lb_index,
+							   PartitionRangeBound *lb,
+							   PartitionRangeBound *ub);
+static int get_range_partition_internal(PartitionBoundInfo bi,
+										int *lb_pos,
+										PartitionRangeBound *lb,
+										PartitionRangeBound *ub);
 static bool compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
 						 Oid *partcollations,
 						 PartitionRangeBound *outer_lb,
@@ -3186,14 +3194,37 @@ partition_bounds_merge(int partnatts,
 
 /*
  * get_range_partition
- *		Returns the index of the range partition with the given lower bound
+ *		Returns the next non-dummy partition of a given range-partitioned
+ *		relation
  *
- * *lb and *ub are set to the lower and upper bounds of the range partition
+ * *lb and *ub are set to the lower and upper bounds of that partition
  * respectively, and *lb_index is advanced to the next lower bound, if any.
  */
 static int
-get_range_partition(PartitionBoundInfo bi, int *lb_index,
-					PartitionRangeBound *lb, PartitionRangeBound *ub)
+get_range_partition(RelOptInfo *rel,
+					PartitionBoundInfo bi,
+					int *lb_index,
+					PartitionRangeBound *lb,
+					PartitionRangeBound *ub)
+{
+	int			part_index;
+
+	Assert(bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	do {
+		part_index = get_range_partition_internal(bi, lb_index, lb, ub);
+		if (part_index == -1)
+			return -1;
+	} while (is_dummy_partition(rel, part_index));
+
+	return part_index;
+}
+
+static int
+get_range_partition_internal(PartitionBoundInfo bi,
+							 int *lb_index,
+							 PartitionRangeBound *lb,
+							 PartitionRangeBound *ub)
 {
 	/* Return the index as -1 if we've exhausted all the lower bounds. */
 	if (*lb_index >= bi->ndatums)
@@ -3456,6 +3487,15 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 	init_partition_map(outer_rel, &outer_map);
 	init_partition_map(inner_rel, &inner_map);
 
+	/*
+	 * If the default partitions (if any) have been proven empty, deem them
+	 * non-existent.
+	 */
+	if (outer_has_default && is_dummy_partition(outer_rel, outer_default))
+		outer_has_default = false;
+	if (inner_has_default && is_dummy_partition(inner_rel, inner_default))
+		inner_has_default = false;
+
 	/*
 	 * Merge the ranges (partitions) from both sides. Every iteration compares
 	 * a pair of ranges, one from each side, advancing to the next range from
@@ -3466,9 +3506,9 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 	 * datum in PartitionBoundInfo::datums of that side.
 	 */
 	outer_lb_index = inner_lb_index = 0;
-	outer_part = get_range_partition(outer_bi, &outer_lb_index,
+	outer_part = get_range_partition(outer_rel, outer_bi, &outer_lb_index,
 									 &outer_lb, &outer_ub);
-	inner_part = get_range_partition(inner_bi, &inner_lb_index,
+	inner_part = get_range_partition(inner_rel, inner_bi, &inner_lb_index,
 									 &inner_lb, &inner_ub);
 	while (outer_part >= 0 || inner_part >= 0)
 	{
@@ -3544,9 +3584,9 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 			save_inner_ub = inner_ub;
 
 			/* Move to the next pair of partitions. */
-			outer_part = get_range_partition(outer_bi, &outer_lb_index,
+			outer_part = get_range_partition(outer_rel, outer_bi, &outer_lb_index,
 											 &outer_lb, &outer_ub);
-			inner_part = get_range_partition(inner_bi, &inner_lb_index,
+			inner_part = get_range_partition(inner_rel, inner_bi, &inner_lb_index,
 											 &inner_lb, &inner_ub);
 
 			/*
@@ -3598,7 +3638,7 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 			}
 
 			/* Move to the next partition on the outer side. */
-			outer_part = get_range_partition(outer_bi, &outer_lb_index,
+			outer_part = get_range_partition(outer_rel, outer_bi, &outer_lb_index,
 											 &outer_lb, &outer_ub);
 		}
 		else
@@ -3625,7 +3665,7 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 			}
 
 			/* Move to the next partition on the inner side. */
-			inner_part = get_range_partition(inner_bi, &inner_lb_index,
+			inner_part = get_range_partition(inner_rel, inner_bi, &inner_lb_index,
 											 &inner_lb, &inner_ub);
 		}
 
@@ -3726,6 +3766,15 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 	init_partition_map(outer_rel, &outer_map);
 	init_partition_map(inner_rel, &inner_map);
 
+	/*
+	 * If the default partitions (if any) have been proven empty, deem them
+	 * non-existent.
+	 */
+	if (outer_has_default && is_dummy_partition(outer_rel, outer_default))
+		outer_has_default = false;
+	if (inner_has_default && is_dummy_partition(inner_rel, inner_default))
+		inner_has_default = false;
+
 	/*
 	 * Merge the list value datums from both sides. Every iteration compares a
 	 * pair of datums, one from each side, advancing to the next datum from the
@@ -3738,10 +3787,39 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 	{
 		Datum	   *merged_datum = NULL;
 		int			merged_index = -1;
+		int			o_index = -1;
+		int			i_index = -1;
 		Datum	   *odatums;
 		Datum	   *idatums;
 		int			cmpval;
 
+		if (cnto < outer_bi->ndatums)
+		{
+			/*
+			 * If the partition on the outer side has been proven empty, ignore
+			 * it and move to the next datum on the outer side.
+			 */
+			o_index = outer_bi->indexes[cnto];
+			if (is_dummy_partition(outer_rel, o_index))
+			{
+				cnto++;
+				continue;
+			}
+		}
+		if (cnti < inner_bi->ndatums)
+		{
+			/*
+			 * If the partition on the inner side has been proven empty, ignore
+			 * it and move to the next datum on the inner side.
+			 */
+			i_index = inner_bi->indexes[cnti];
+			if (is_dummy_partition(inner_rel, i_index))
+			{
+				cnti++;
+				continue;
+			}
+		}
+
 		/* Get the list datums of the next pair of partitions. */
 		odatums = cnto < outer_bi->ndatums ? outer_bi->datums[cnto] : NULL;
 		idatums = cnti < inner_bi->ndatums ? inner_bi->datums[cnti] : NULL;
@@ -3770,9 +3848,6 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 
 		if (cmpval == 0)
 		{
-			int			o_index = outer_bi->indexes[cnto];
-			int			i_index = inner_bi->indexes[cnti];
-
 			/*
 			 * Datums match. Rows on either side with these datums as partition
 			 * key value will join and will be part of the partition of the
@@ -3797,14 +3872,13 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 		else if (cmpval < 0)
 		{
 			Assert(cnto < outer_bi->ndatums);
+			Assert(o_index >= 0);
 
 			/* A datum missing from the inner side. */
 			merged_datum = odatums;
 
 			if (inner_has_default || IS_OUTER_JOIN(jointype))
 			{
-				int			o_index = outer_bi->indexes[cnto];
-
 				Assert(o_index >= 0);
 				merged_index = process_outer_partition(&outer_map,
 													   &inner_map,
@@ -3826,14 +3900,13 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 		{
 			Assert(cmpval > 0);
 			Assert(cnti < inner_bi->ndatums);
+			Assert(i_index >= 0);
 
 			/* A datum missing from the outer side. */
 			merged_datum = idatums;
 
 			if (outer_has_default || jointype == JOIN_FULL)
 			{
-				int			i_index = inner_bi->indexes[cnti];
-
 				Assert(i_index >= 0);
 				merged_index = process_inner_partition(&outer_map,
 													   &inner_map,
@@ -3863,6 +3936,17 @@ partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
 		}
 	}
 
+	/*
+	 * If the NULL partitions (if any) have been proven empty, deem them
+	 * non-existent.
+	 */
+	if (outer_has_null &&
+		is_dummy_partition(outer_rel, outer_bi->null_index))
+		outer_has_null = false;
+	if (inner_has_null &&
+		is_dummy_partition(inner_rel, inner_bi->null_index))
+		inner_has_null = false;
+
 	/* Merge null partitions if any. */
 	if (outer_has_null || inner_has_null)
 		merge_null_partitions(&outer_map, &inner_map,
@@ -3950,6 +4034,21 @@ free_partition_map(PartitionMap *map)
 	pfree(map->old_indexes);
 }
 
+/*
+ * is_dummy_partition --- has partition been proven empty?
+ */
+static bool
+is_dummy_partition(RelOptInfo *rel, int part_index)
+{
+	RelOptInfo *part_rel;
+
+	Assert(part_index >= 0);
+	part_rel = rel->part_rels[part_index];
+	if (part_rel == NULL || IS_DUMMY_REL(part_rel))
+		return true;
+	return false;
+}
+
 /*
  * map_and_merge_partitions
  *
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 924cb99863..d3f4e50878 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -2872,6 +2872,98 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 O
  275 | 0275 | 275 | 0275 | 275 | 0275
 (8 rows)
 
+DROP TABLE prt1_ad;
+DROP TABLE prt2_ad;
+DROP TABLE prt3_ad;
+-- Test interaction of partitioned join with partition pruning
+CREATE TABLE prt1_ad (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_ad_p1 PARTITION OF prt1_ad FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_ad_p2 PARTITION OF prt1_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_ad_p3 PARTITION OF prt1_ad FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_ad_a_idx on prt1_ad (a);
+INSERT INTO prt1_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_ad;
+CREATE TABLE prt2_ad (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_ad_p1 PARTITION OF prt2_ad FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt2_ad_p2 PARTITION OF prt2_ad FOR VALUES FROM (200) TO (400);
+CREATE INDEX prt2_ad_b_idx on prt2_ad (b);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+                        QUERY PLAN                         
+-----------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: ((a < 300) AND (b = 0))
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: ((a < 300) AND (b = 0))
+(15 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 | 150 | 0150
+ 175 | 0175 | 175 | 0175
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+(8 rows)
+
+DROP TABLE prt1_ad_p3;
+CREATE TABLE prt1_ad_default PARTITION OF prt1_ad DEFAULT;
+ANALYZE prt1_ad;
+CREATE TABLE prt2_ad_default PARTITION OF prt2_ad DEFAULT;
+ANALYZE prt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+                                QUERY PLAN                                
+--------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: ((a >= 100) AND (a < 300) AND (b = 0))
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: ((a >= 100) AND (a < 300) AND (b = 0))
+(15 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 | 150 | 0150
+ 175 | 0175 | 175 | 0175
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+(8 rows)
+
+DROP TABLE prt1_ad;
+DROP TABLE prt2_ad;
 -- Tests for list-partitioned tables
 CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
 CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0001', '0003');
@@ -4168,3 +4260,131 @@ SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2
  22 | 0002 | 22 | 0002 |    | 
 (55 rows)
 
+DROP TABLE plt1_ad;
+DROP TABLE plt2_ad;
+DROP TABLE plt3_ad;
+-- Test interaction of partitioned join with partition pruning
+CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0001');
+CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0002');
+CREATE TABLE plt1_ad_p3 PARTITION OF plt1_ad FOR VALUES IN ('0003');
+CREATE TABLE plt1_ad_p4 PARTITION OF plt1_ad FOR VALUES IN (NULL, '0004', '0005');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt1_ad VALUES (-1, -1, NULL);
+ANALYZE plt1_ad;
+CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0003');
+CREATE TABLE plt2_ad_p3 PARTITION OF plt2_ad FOR VALUES IN (NULL);
+CREATE TABLE plt2_ad_p4 PARTITION OF plt2_ad FOR VALUES IN ('0004', '0005');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 AND t2.c IS NOT NULL ORDER BY t1.a;
+                                       QUERY PLAN                                        
+-----------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p2 t2_1
+                     Filter: (c IS NOT NULL)
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_1
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p4 t2_2
+                     Filter: (c IS NOT NULL)
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p4 t1_2
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 AND t2.c IS NOT NULL ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 5 | 0005 | 5 | 0005
+(3 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Seq Scan on plt2_ad_p4 t2
+         ->  Hash
+               ->  Seq Scan on plt1_ad_p4 t1
+                     Filter: ((c IS NULL) AND (b < 10))
+(8 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+ a  | c | a | c 
+----+---+---+---
+ -1 |   |   | 
+(1 row)
+
+CREATE TABLE plt1_ad_default PARTITION OF plt1_ad DEFAULT;
+ANALYZE plt1_ad;
+CREATE TABLE plt2_ad_default PARTITION OF plt2_ad DEFAULT;
+ANALYZE plt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 AND t2.c IS NOT NULL ORDER BY t1.a;
+                                       QUERY PLAN                                        
+-----------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p2 t2_1
+                     Filter: (c IS NOT NULL)
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_1
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p4 t2_2
+                     Filter: (c IS NOT NULL)
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p4 t1_2
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 AND t2.c IS NOT NULL ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 5 | 0005 | 5 | 0005
+(3 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Seq Scan on plt2_ad_p4 t2
+         ->  Hash
+               ->  Seq Scan on plt1_ad_p4 t1
+                     Filter: ((c IS NULL) AND (b < 10))
+(8 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+ a  | c | a | c 
+----+---+---+---
+ -1 |   |   | 
+(1 row)
+
+DROP TABLE plt1_ad;
+DROP TABLE plt2_ad;
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 418ffa117f..069ec7ca5d 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -631,6 +631,44 @@ EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) LEFT JOIN prt3_ad t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
 SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) LEFT JOIN prt3_ad t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
 
+DROP TABLE prt1_ad;
+DROP TABLE prt2_ad;
+DROP TABLE prt3_ad;
+
+-- Test interaction of partitioned join with partition pruning
+
+CREATE TABLE prt1_ad (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_ad_p1 PARTITION OF prt1_ad FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_ad_p2 PARTITION OF prt1_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_ad_p3 PARTITION OF prt1_ad FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_ad_a_idx on prt1_ad (a);
+INSERT INTO prt1_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_ad;
+
+CREATE TABLE prt2_ad (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_ad_p1 PARTITION OF prt2_ad FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt2_ad_p2 PARTITION OF prt2_ad FOR VALUES FROM (200) TO (400);
+CREATE INDEX prt2_ad_b_idx on prt2_ad (b);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt1_ad_p3;
+CREATE TABLE prt1_ad_default PARTITION OF prt1_ad DEFAULT;
+ANALYZE prt1_ad;
+
+CREATE TABLE prt2_ad_default PARTITION OF prt2_ad DEFAULT;
+ANALYZE prt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt1_ad;
+DROP TABLE prt2_ad;
 
 -- Tests for list-partitioned tables
 CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
@@ -954,3 +992,52 @@ ANALYZE plt3_ad;
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.c = t2.c)) FULL JOIN plt3_ad t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
 SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.c = t2.c)) FULL JOIN plt3_ad t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+
+DROP TABLE plt1_ad;
+DROP TABLE plt2_ad;
+DROP TABLE plt3_ad;
+
+-- Test interaction of partitioned join with partition pruning
+
+CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0001');
+CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0002');
+CREATE TABLE plt1_ad_p3 PARTITION OF plt1_ad FOR VALUES IN ('0003');
+CREATE TABLE plt1_ad_p4 PARTITION OF plt1_ad FOR VALUES IN (NULL, '0004', '0005');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt1_ad VALUES (-1, -1, NULL);
+ANALYZE plt1_ad;
+
+CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0003');
+CREATE TABLE plt2_ad_p3 PARTITION OF plt2_ad FOR VALUES IN (NULL);
+CREATE TABLE plt2_ad_p4 PARTITION OF plt2_ad FOR VALUES IN ('0004', '0005');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 AND t2.c IS NOT NULL ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 AND t2.c IS NOT NULL ORDER BY t1.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+
+CREATE TABLE plt1_ad_default PARTITION OF plt1_ad DEFAULT;
+ANALYZE plt1_ad;
+
+CREATE TABLE plt2_ad_default PARTITION OF plt2_ad DEFAULT;
+ANALYZE plt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 AND t2.c IS NOT NULL ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 AND t2.c IS NOT NULL ORDER BY t1.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt1_ad;
+DROP TABLE plt2_ad;
#116Mark Dilger
mark.dilger@enterprisedb.com
In reply to: Etsuro Fujita (#115)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Feb 5, 2020, at 4:51 AM, Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

On Wed, Jan 29, 2020 at 9:15 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

On Tue, Jan 28, 2020 at 1:39 PM Mark Dilger
<mark.dilger@enterprisedb.com> wrote:

I have added tests checking correctness and showing some partition pruning limitations. Find three patches, attached.

The v31-0001-… patch merely applies your patches as a starting point for the next two patches. It is your work, not mine.

The v31-0002-… patch adds more regression tests for range partitioning. The commit message contains my comments about that.

The v31-0003-… patch adds more regression tests for list partitioning, and again, the commit message contains my comments about that.

I'll dig into it more closely.

I tested the new test patches. The patches are applied to the
partition_join.sql regression test cleanly, but the new tests failed
in my environment (even with make check LANG=C). I think I should set
the right locale for the testing. Which one did you use?

I did not set a locale in the tests. My environment has:

LANG="en_US.UTF-8"
LC_COLLATE="en_US.UTF-8"
LC_CTYPE="en_US.UTF-8"
LC_MESSAGES="en_US.UTF-8"
LC_MONETARY="en_US.UTF-8"
LC_NUMERIC="en_US.UTF-8"
LC_TIME="en_US.UTF-8"
LC_ALL=

Maybe I'm
missing something, but let me comment on the new tests. This is the
one proposed in the v31-0002 patch:

+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a)
WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+                            QUERY PLAN
+------------------------------------------------------------------
+ Hash Join
+   Hash Cond: (t2.a = t1.a)
+   ->  Append
+         ->  Seq Scan on beta_a t2_1
+         ->  Seq Scan on beta_b t2_2
+         ->  Seq Scan on beta_c t2_3
+         ->  Seq Scan on beta_d t2_4
+         ->  Seq Scan on beta_e t2_5
+         ->  Seq Scan on beta_f t2_6
+         ->  Seq Scan on beta_g t2_7
+         ->  Seq Scan on beta_h t2_8
+         ->  Seq Scan on beta_default t2_9
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on alpha_e t1_1
+                     Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+               ->  Seq Scan on alpha_default t1_2
+                     Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+(18 rows)

The commit message says:

When joining with

alpha.a = beta.a and alpha.a IN ('äbç', 'ὀδυσσεύς')

the planner does the right thing for one side of the query, but
hits all partitions for the other side, which it doesn't need to
do.

Yeah, I agree that the resulting plan is not efficient. The reason
for that would be that the planner doesn't generate a qual on the
outer side matching the ScalarArrayOpExpr qual "a = ANY
('{äbç,ὀδυσσεύς}'::text[])" on the inner side, which I think would be
a restriction caused by the equivalence machinery not by the
partitionwise join logic IIUC.

It’s fine if this is beyond the scope of the patch.

I think the critique would be useful,
so I don't object to adding this test case, but the critique would be
more about query planning that is actually not related to the
partitionwise join logic, so I'm not sure that the partition_join.sql
regression test is the best place to add it. I guess that there would
be much better places than partition_join.sql.

You don’t need to add the test anywhere. It’s enough for me that you looked at it and considered whether the partition-wise join patch should do anything differently in this case. Again, it sounds like this is beyond the scope of the patch.

(This is nitpicking;
but another thing I noticed about this test case is that the join
query contains only a single join condition "t1.a = t2.a", but the
queried tables alpha and beta are range-partitioned by multiple
columns a and b, so the query should have a join condition for each of
the columns like "t1.a = t2.a AND t1.b = t2.b" if adding this as a
test case for partitionwise join.)

Well, it is important that partition-wise join does not break such queries. I added the column ‘b’ to the partitioning logic to verify that did not confuse the code in your patch.

I feel almost the same to other
test cases in the patch (and the v31-0003 patch), except this one
proposed in the v31-0003 patch:

+CREATE TABLE alpha (a TEXT) PARTITION BY LIST(a);
+CREATE TABLE alpha_a PARTITION OF alpha FOR VALUES IN ('Turkiye', 'TURKIYE');
+CREATE TABLE alpha_b PARTITION OF alpha FOR VALUES IN ('b?t', 'BIT');
+CREATE TABLE alpha_c PARTITION OF alpha FOR VALUES IN ('abc', 'ABC');
+CREATE TABLE alpha_d PARTITION OF alpha FOR VALUES IN ('aaa', 'cote', 'Gotz');
+CREATE TABLE alpha_e PARTITION OF alpha FOR VALUES IN ('?δυσσε??', '?ΔΥΣΣΕ?Σ');
+CREATE TABLE alpha_f PARTITION OF alpha FOR VALUES IN ('を読み取り用',
'にオープンできませんでした', NULL);
+CREATE TABLE alpha_default PARTITION OF alpha DEFAULT;
+CREATE TABLE beta (a TEXT) PARTITION BY LIST(a);
+CREATE TABLE beta_a PARTITION OF beta FOR VALUES IN ('Turkiye',
'cote', '?ΔΥΣΣΕ?Σ');
+CREATE TABLE beta_b PARTITION OF beta FOR VALUES IN ('b?t', 'TURKIYE');
+CREATE TABLE beta_c PARTITION OF beta FOR VALUES IN ('abc', 'を読み取り用',
'にオープンできませんでした');
+CREATE TABLE beta_d PARTITION OF beta FOR VALUES IN ('aaa', 'Gotz',
'BIT', '?δυσσε??', 'ABC', NULL);
+CREATE TABLE beta_default PARTITION OF beta DEFAULT;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a)
WHERE t1.a IS NULL;
+                QUERY PLAN
+-------------------------------------------
+ Hash Join
+   Hash Cond: (t2.a = t1.a)
+   ->  Append
+         ->  Seq Scan on beta_d t2_1
+         ->  Seq Scan on beta_b t2_2
+         ->  Seq Scan on beta_a t2_3
+         ->  Seq Scan on beta_c t2_4
+         ->  Seq Scan on beta_default t2_5
+   ->  Hash
+         ->  Seq Scan on alpha_f t1
+               Filter: (a IS NULL)
+(11 rows)

Which made me notice an issue in the partitionwise join logic:
partition_bounds_merge() assumes that all partitions of given
relations are always present; in other words, it doesn't consider
cases where some of the partitions have been pruned entirely. :-( If
that function considered the cases, the above query would use
partitionwise join, because alpha_f only matches beta_c. I don't
think this is a bug, but it causes the planner not only to fail to
chose partitionwise join but to waste CPU cycles to process pruned
partitions, so I'll propose to address it. Attached is a patch for
that (the v31-0004 patch) created on top of the main patch (the
v31-0001 patch), which is also attached. With the attached, the plan
for the above query would be changed to something like this:

EXPLAIN (COSTS OFF)
SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a =
t2.a) WHERE t1.a IS NULL;
QUERY PLAN
------------------------------
Nested Loop
Join Filter: (t1.a = t2.a)
-> Seq Scan on alpha_f t1
Filter: (a IS NULL)
-> Seq Scan on beta_c t2
(5 rows)

Thanks again, Mark!

Thank you for working on this important patch!


Mark Dilger
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#117Mark Dilger
mark.dilger@enterprisedb.com
In reply to: Etsuro Fujita (#115)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Feb 5, 2020, at 4:51 AM, Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

<v31-0001-Applying-Etsuro-Fujita-s-patches.patch><v31-0004-Consider-pruned-partitions.patch>

The patches apply and pass all tests. A review of the patch vs. master looks reasonable.

The partition_join.sql test has multiple levels of partitioning, but when your patch extends that test with “advanced partition-wise join”, none of the tables for the new section have multiple levels. I spent a little while reviewing the code and inventing multiple level partitioning tests for advanced partition-wise join and did not encounter any problems. I don’t care whether you use this particular example, but do you want to have multiple level partitioning in the new test section?

CREATE TABLE alpha (a double precision, b double precision) PARTITION BY RANGE (a);
CREATE TABLE alpha_neg PARTITION OF alpha FOR VALUES FROM ('-Infinity') TO (0) PARTITION BY RANGE (b);
CREATE TABLE alpha_pos PARTITION OF alpha FOR VALUES FROM (0) TO ('Infinity') PARTITION BY RANGE (b);
CREATE TABLE alpha_nan PARTITION OF alpha FOR VALUES FROM ('Infinity') TO ('NaN');
CREATE TABLE alpha_neg_neg PARTITION OF alpha_neg FOR VALUES FROM ('-Infinity') TO (0);
CREATE TABLE alpha_neg_pos PARTITION OF alpha_neg FOR VALUES FROM (0) TO ('Infinity');
CREATE TABLE alpha_neg_nan PARTITION OF alpha_neg FOR VALUES FROM ('Infinity') TO ('NaN');
CREATE TABLE alpha_pos_neg PARTITION OF alpha_pos FOR VALUES FROM ('-Infinity') TO (0);
CREATE TABLE alpha_pos_pos PARTITION OF alpha_pos FOR VALUES FROM (0) TO ('Infinity');
CREATE TABLE alpha_pos_nan PARTITION OF alpha_pos FOR VALUES FROM ('Infinity') TO ('NaN');
INSERT INTO alpha (a, b)
(SELECT * FROM
(VALUES (-1.0::float8), (0.0::float8), (1.0::float8), ('Infinity'::float8)) a,
(VALUES (-1.0::float8), (0.0::float8), (1.0::float8), ('Infinity'::float8)) b
);
ANALYZE alpha;
ANALYZE alpha_neg;
ANALYZE alpha_pos;
ANALYZE alpha_nan;
ANALYZE alpha_neg_neg;
ANALYZE alpha_neg_pos;
ANALYZE alpha_neg_nan;
ANALYZE alpha_pos_neg;
ANALYZE alpha_pos_pos;
ANALYZE alpha_pos_nan;
CREATE TABLE beta (a double precision, b double precision) PARTITION BY RANGE (a, b);
CREATE TABLE beta_lo PARTITION OF beta FOR VALUES FROM (-5, -5) TO (0, 0);
CREATE TABLE beta_me PARTITION OF beta FOR VALUES FROM (0, 0) TO (0, 5);
CREATE TABLE beta_hi PARTITION OF beta FOR VALUES FROM (0, 5) TO (5, 5);
INSERT INTO beta (a, b)
(SELECT * FROM
(VALUES (-1.0::float8), (0.0::float8), (1.0::float8)) a,
(VALUES (-1.0::float8), (0.0::float8), (1.0::float8)) b
);
ANALYZE beta;
ANALYZE beta_lo;
ANALYZE beta_me;
ANALYZE beta_hi;
EXPLAIN SELECT * FROM alpha INNER JOIN beta ON (alpha.a = beta.a AND alpha.b = beta.b) WHERE alpha.a = 1 AND beta.b = 1;
QUERY PLAN
-------------------------------------------------------------------------------
Nested Loop (cost=0.00..2.11 rows=1 width=32)
-> Seq Scan on alpha_pos_pos alpha (cost=0.00..1.06 rows=1 width=16)
Filter: ((b = '1'::double precision) AND (a = '1'::double precision))
-> Seq Scan on beta_hi beta (cost=0.00..1.04 rows=1 width=16)
Filter: ((b = '1'::double precision) AND (a = '1'::double precision))
(5 rows)


Mark Dilger
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#118Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Mark Dilger (#116)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Thu, Feb 6, 2020 at 2:03 AM Mark Dilger <mark.dilger@enterprisedb.com> wrote:

On Feb 5, 2020, at 4:51 AM, Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

On Tue, Jan 28, 2020 at 1:39 PM Mark Dilger
<mark.dilger@enterprisedb.com> wrote:

I have added tests checking correctness and showing some partition pruning limitations. Find three patches, attached.

The v31-0001-… patch merely applies your patches as a starting point for the next two patches. It is your work, not mine.

The v31-0002-… patch adds more regression tests for range partitioning. The commit message contains my comments about that.

The v31-0003-… patch adds more regression tests for list partitioning, and again, the commit message contains my comments about that.

I tested the new test patches. The patches are applied to the
partition_join.sql regression test cleanly, but the new tests failed
in my environment (even with make check LANG=C). I think I should set
the right locale for the testing. Which one did you use?

I did not set a locale in the tests. My environment has:

LANG="en_US.UTF-8"
LC_COLLATE="en_US.UTF-8"
LC_CTYPE="en_US.UTF-8"
LC_MESSAGES="en_US.UTF-8"
LC_MONETARY="en_US.UTF-8"
LC_NUMERIC="en_US.UTF-8"
LC_TIME="en_US.UTF-8"
LC_ALL=

Thanks for the information!

Maybe I'm
missing something, but let me comment on the new tests. This is the
one proposed in the v31-0002 patch:

+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a)
WHERE t1.a IN ('äbç', 'ὀδυσσεύς');
+                            QUERY PLAN
+------------------------------------------------------------------
+ Hash Join
+   Hash Cond: (t2.a = t1.a)
+   ->  Append
+         ->  Seq Scan on beta_a t2_1
+         ->  Seq Scan on beta_b t2_2
+         ->  Seq Scan on beta_c t2_3
+         ->  Seq Scan on beta_d t2_4
+         ->  Seq Scan on beta_e t2_5
+         ->  Seq Scan on beta_f t2_6
+         ->  Seq Scan on beta_g t2_7
+         ->  Seq Scan on beta_h t2_8
+         ->  Seq Scan on beta_default t2_9
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on alpha_e t1_1
+                     Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+               ->  Seq Scan on alpha_default t1_2
+                     Filter: (a = ANY ('{äbç,ὀδυσσεύς}'::text[]))
+(18 rows)

The commit message says:

When joining with

alpha.a = beta.a and alpha.a IN ('äbç', 'ὀδυσσεύς')

the planner does the right thing for one side of the query, but
hits all partitions for the other side, which it doesn't need to
do.

Yeah, I agree that the resulting plan is not efficient. The reason
for that would be that the planner doesn't generate a qual on the
outer side matching the ScalarArrayOpExpr qual "a = ANY
('{äbç,ὀδυσσεύς}'::text[])" on the inner side, which I think would be
a restriction caused by the equivalence machinery not by the
partitionwise join logic IIUC.

It’s fine if this is beyond the scope of the patch.

I think the critique would be useful,
so I don't object to adding this test case, but the critique would be
more about query planning that is actually not related to the
partitionwise join logic, so I'm not sure that the partition_join.sql
regression test is the best place to add it. I guess that there would
be much better places than partition_join.sql.

You don’t need to add the test anywhere. It’s enough for me that you looked at it and considered whether the partition-wise join patch should do anything differently in this case. Again, it sounds like this is beyond the scope of the patch.

OK

(This is nitpicking;
but another thing I noticed about this test case is that the join
query contains only a single join condition "t1.a = t2.a", but the
queried tables alpha and beta are range-partitioned by multiple
columns a and b, so the query should have a join condition for each of
the columns like "t1.a = t2.a AND t1.b = t2.b" if adding this as a
test case for partitionwise join.)

Well, it is important that partition-wise join does not break such queries. I added the column ‘b’ to the partitioning logic to verify that did not confuse the code in your patch.

OK, thanks for the testing!

Best regards,
Etsuro Fujita

#119Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Mark Dilger (#117)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Thu, Feb 6, 2020 at 3:55 AM Mark Dilger <mark.dilger@enterprisedb.com> wrote:

The patches apply and pass all tests. A review of the patch vs. master looks reasonable.

Thanks for the review!

The partition_join.sql test has multiple levels of partitioning, but when your patch extends that test with “advanced partition-wise join”, none of the tables for the new section have multiple levels. I spent a little while reviewing the code and inventing multiple level partitioning tests for advanced partition-wise join and did not encounter any problems. I don’t care whether you use this particular example, but do you want to have multiple level partitioning in the new test section?

Yes, I do.

CREATE TABLE alpha (a double precision, b double precision) PARTITION BY RANGE (a);
CREATE TABLE alpha_neg PARTITION OF alpha FOR VALUES FROM ('-Infinity') TO (0) PARTITION BY RANGE (b);
CREATE TABLE alpha_pos PARTITION OF alpha FOR VALUES FROM (0) TO ('Infinity') PARTITION BY RANGE (b);
CREATE TABLE alpha_nan PARTITION OF alpha FOR VALUES FROM ('Infinity') TO ('NaN');
CREATE TABLE alpha_neg_neg PARTITION OF alpha_neg FOR VALUES FROM ('-Infinity') TO (0);
CREATE TABLE alpha_neg_pos PARTITION OF alpha_neg FOR VALUES FROM (0) TO ('Infinity');
CREATE TABLE alpha_neg_nan PARTITION OF alpha_neg FOR VALUES FROM ('Infinity') TO ('NaN');
CREATE TABLE alpha_pos_neg PARTITION OF alpha_pos FOR VALUES FROM ('-Infinity') TO (0);
CREATE TABLE alpha_pos_pos PARTITION OF alpha_pos FOR VALUES FROM (0) TO ('Infinity');
CREATE TABLE alpha_pos_nan PARTITION OF alpha_pos FOR VALUES FROM ('Infinity') TO ('NaN');
INSERT INTO alpha (a, b)
(SELECT * FROM
(VALUES (-1.0::float8), (0.0::float8), (1.0::float8), ('Infinity'::float8)) a,
(VALUES (-1.0::float8), (0.0::float8), (1.0::float8), ('Infinity'::float8)) b
);
ANALYZE alpha;
ANALYZE alpha_neg;
ANALYZE alpha_pos;
ANALYZE alpha_nan;
ANALYZE alpha_neg_neg;
ANALYZE alpha_neg_pos;
ANALYZE alpha_neg_nan;
ANALYZE alpha_pos_neg;
ANALYZE alpha_pos_pos;
ANALYZE alpha_pos_nan;
CREATE TABLE beta (a double precision, b double precision) PARTITION BY RANGE (a, b);
CREATE TABLE beta_lo PARTITION OF beta FOR VALUES FROM (-5, -5) TO (0, 0);
CREATE TABLE beta_me PARTITION OF beta FOR VALUES FROM (0, 0) TO (0, 5);
CREATE TABLE beta_hi PARTITION OF beta FOR VALUES FROM (0, 5) TO (5, 5);
INSERT INTO beta (a, b)
(SELECT * FROM
(VALUES (-1.0::float8), (0.0::float8), (1.0::float8)) a,
(VALUES (-1.0::float8), (0.0::float8), (1.0::float8)) b
);
ANALYZE beta;
ANALYZE beta_lo;
ANALYZE beta_me;
ANALYZE beta_hi;
EXPLAIN SELECT * FROM alpha INNER JOIN beta ON (alpha.a = beta.a AND alpha.b = beta.b) WHERE alpha.a = 1 AND beta.b = 1;
QUERY PLAN
-------------------------------------------------------------------------------
Nested Loop (cost=0.00..2.11 rows=1 width=32)
-> Seq Scan on alpha_pos_pos alpha (cost=0.00..1.06 rows=1 width=16)
Filter: ((b = '1'::double precision) AND (a = '1'::double precision))
-> Seq Scan on beta_hi beta (cost=0.00..1.04 rows=1 width=16)
Filter: ((b = '1'::double precision) AND (a = '1'::double precision))
(5 rows)

Hmm, I'm not sure this is a good test case for that, because this
result would be due to partition pruning applied to each side of the
join before considering partition-wise join; you could get the same
result even with enable_partitionwise_join=off. I think it's
important that the partition-wise join logic doesn't break this query,
though.

Best regards,
Etsuro Fujita

#120Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Etsuro Fujita (#119)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Fri, Feb 7, 2020 at 9:57 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

On Thu, Feb 6, 2020 at 3:55 AM Mark Dilger <mark.dilger@enterprisedb.com> wrote:

The patches apply and pass all tests. A review of the patch vs. master looks reasonable.

I've merged the patches. Attached is a new version of the patch.

The partition_join.sql test has multiple levels of partitioning, but when your patch extends that test with “advanced partition-wise join”, none of the tables for the new section have multiple levels. I spent a little while reviewing the code and inventing multiple level partitioning tests for advanced partition-wise join and did not encounter any problems. I don’t care whether you use this particular example, but do you want to have multiple level partitioning in the new test section?

Yes, I do.

CREATE TABLE alpha (a double precision, b double precision) PARTITION BY RANGE (a);
CREATE TABLE alpha_neg PARTITION OF alpha FOR VALUES FROM ('-Infinity') TO (0) PARTITION BY RANGE (b);
CREATE TABLE alpha_pos PARTITION OF alpha FOR VALUES FROM (0) TO ('Infinity') PARTITION BY RANGE (b);
CREATE TABLE alpha_nan PARTITION OF alpha FOR VALUES FROM ('Infinity') TO ('NaN');
CREATE TABLE alpha_neg_neg PARTITION OF alpha_neg FOR VALUES FROM ('-Infinity') TO (0);
CREATE TABLE alpha_neg_pos PARTITION OF alpha_neg FOR VALUES FROM (0) TO ('Infinity');
CREATE TABLE alpha_neg_nan PARTITION OF alpha_neg FOR VALUES FROM ('Infinity') TO ('NaN');
CREATE TABLE alpha_pos_neg PARTITION OF alpha_pos FOR VALUES FROM ('-Infinity') TO (0);
CREATE TABLE alpha_pos_pos PARTITION OF alpha_pos FOR VALUES FROM (0) TO ('Infinity');
CREATE TABLE alpha_pos_nan PARTITION OF alpha_pos FOR VALUES FROM ('Infinity') TO ('NaN');
INSERT INTO alpha (a, b)
(SELECT * FROM
(VALUES (-1.0::float8), (0.0::float8), (1.0::float8), ('Infinity'::float8)) a,
(VALUES (-1.0::float8), (0.0::float8), (1.0::float8), ('Infinity'::float8)) b
);
ANALYZE alpha;
ANALYZE alpha_neg;
ANALYZE alpha_pos;
ANALYZE alpha_nan;
ANALYZE alpha_neg_neg;
ANALYZE alpha_neg_pos;
ANALYZE alpha_neg_nan;
ANALYZE alpha_pos_neg;
ANALYZE alpha_pos_pos;
ANALYZE alpha_pos_nan;
CREATE TABLE beta (a double precision, b double precision) PARTITION BY RANGE (a, b);
CREATE TABLE beta_lo PARTITION OF beta FOR VALUES FROM (-5, -5) TO (0, 0);
CREATE TABLE beta_me PARTITION OF beta FOR VALUES FROM (0, 0) TO (0, 5);
CREATE TABLE beta_hi PARTITION OF beta FOR VALUES FROM (0, 5) TO (5, 5);
INSERT INTO beta (a, b)
(SELECT * FROM
(VALUES (-1.0::float8), (0.0::float8), (1.0::float8)) a,
(VALUES (-1.0::float8), (0.0::float8), (1.0::float8)) b
);
ANALYZE beta;
ANALYZE beta_lo;
ANALYZE beta_me;
ANALYZE beta_hi;
EXPLAIN SELECT * FROM alpha INNER JOIN beta ON (alpha.a = beta.a AND alpha.b = beta.b) WHERE alpha.a = 1 AND beta.b = 1;
QUERY PLAN
-------------------------------------------------------------------------------
Nested Loop (cost=0.00..2.11 rows=1 width=32)
-> Seq Scan on alpha_pos_pos alpha (cost=0.00..1.06 rows=1 width=16)
Filter: ((b = '1'::double precision) AND (a = '1'::double precision))
-> Seq Scan on beta_hi beta (cost=0.00..1.04 rows=1 width=16)
Filter: ((b = '1'::double precision) AND (a = '1'::double precision))
(5 rows)

Hmm, I'm not sure this is a good test case for that, because this
result would be due to partition pruning applied to each side of the
join before considering partition-wise join; you could get the same
result even with enable_partitionwise_join=off. I think it's
important that the partition-wise join logic doesn't break this query,
though.

I think this would be beyond the scope of the patch, so I added
different test cases that I think would be better as ones for
multi-level partitioning.

Thanks!

Best regards,
Etsuro Fujita

Attachments:

v32-0001-Improve-partition-matching-for-partitionwise-join.patchapplication/octet-stream; name=v32-0001-Improve-partition-matching-for-partitionwise-join.patchDownload
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index d76fae44b8..e42a52fb9b 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -2285,6 +2285,8 @@ _outRelOptInfo(StringInfo str, const RelOptInfo *node)
 	WRITE_BOOL_FIELD(has_eclass_joins);
 	WRITE_BOOL_FIELD(consider_partitionwise_join);
 	WRITE_BITMAPSET_FIELD(top_parent_relids);
+	WRITE_BOOL_FIELD(merged);
+	WRITE_BITMAPSET_FIELD(all_partrels);
 	WRITE_NODE_FIELD(partitioned_child_rels);
 }
 
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index a21c295b99..77e6ff5376 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -48,6 +48,9 @@ static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 												Relids left_relids, Relids right_relids);
 static int	match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel,
 										 bool strict_op);
+static void get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2);
 
 
 /*
@@ -1357,25 +1360,30 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	bool		merged = false;
+	List	   *parts1 = NIL;
+	List	   *parts2 = NIL;
+	ListCell   *lcr1 = NULL;
+	ListCell   *lcr2 = NULL;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partition-wise join if either of the joining
+	 * relations is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1390,34 +1398,107 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		   joinrel->part_scheme == rel2->part_scheme);
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * If we don't have the partition bounds for the join rel yet, try to
+	 * create it along with pairs of partitions to be joined; else generate
+	 * those using the partitioning info for the join rel we already have.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	if (joinrel->nparts == -1)
+	{
+		PartitionScheme part_scheme = joinrel->part_scheme;
+		PartitionBoundInfo boundinfo = NULL;
+		int			nparts = 0;
+
+		Assert(joinrel->boundinfo == NULL);
+		Assert(joinrel->part_rels == NULL);
+
+		/*
+		 * See if the partition bounds for inputs are exactly the same, in
+		 * which case we don't need to work hard: partitions with the same
+		 * partition indexes will form join pairs, and the join rel will have
+		 * the same partition bounds as inputs; otherwise try to merge the
+		 * partition bounds along with generating join pairs.
+		 *
+		 * Even if one or both inputs have merged partition bounds, it'd be
+		 * possible for the partition bounds to be exactly the same, but it
+		 * seems unlikely to be worth the cycles to check; do this check only
+		 * if both inputs have non-merged partition bounds.
+		 */
+		if (!rel1->merged &&
+			!rel2->merged &&
+			rel1->nparts == rel2->nparts &&
+			partition_bounds_equal(part_scheme->partnatts,
+								   part_scheme->parttyplen,
+								   part_scheme->parttypbyval,
+								   rel1->boundinfo, rel2->boundinfo))
+		{
+			boundinfo = rel1->boundinfo;
+			nparts = rel1->nparts;
+		}
+		else
+		{
+			boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											   part_scheme->parttyplen,
+											   part_scheme->parttypbyval,
+											   part_scheme->partsupfunc,
+											   part_scheme->partcollation,
+											   rel1, rel2,
+											   parent_sjinfo->jointype,
+											   &parts1, &parts2);
+			if (boundinfo == NULL)
+			{
+				joinrel->nparts = 0;
+				return;
+			}
+			nparts = list_length(parts1);
+			merged = true;
+		}
+
+		Assert(nparts > 0);
+		joinrel->boundinfo = boundinfo;
+		joinrel->merged = merged;
+		joinrel->nparts = nparts;
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+	}
+	else
+	{
+		Assert(joinrel->nparts > 0);
+		Assert(joinrel->boundinfo);
+		Assert(joinrel->part_rels);
+
+		/*
+		 * If the partition bounds for the join rel are not merged ones,
+		 * inputs are guaranteed to have the same partition bounds, so
+		 * partitions with the same partition indexes will form join pairs;
+		 * else let get_matching_part_pairs() do the work.
+		 */
+		if (joinrel->merged)
+		{
+			get_matching_part_pairs(root, joinrel, rel1, rel2,
+									&parts1, &parts2);
+			Assert(list_length(parts1) == joinrel->nparts);
+			Assert(list_length(parts2) == joinrel->nparts);
+			merged = true;
+		}
+	}
 
-	nparts = joinrel->nparts;
+	if (merged)
+	{
+		lcr1 = list_head(parts1);
+		lcr2 = list_head(parts2);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
-		bool		rel1_empty = (child_rel1 == NULL ||
-								  IS_DUMMY_REL(child_rel1));
-		bool		rel2_empty = (child_rel2 == NULL ||
-								  IS_DUMMY_REL(child_rel2));
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		bool		rel1_empty;
+		bool		rel2_empty;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1425,6 +1506,22 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		if (merged)
+		{
+			child_rel1 = lfirst_node(RelOptInfo, lcr1);
+			child_rel2 = lfirst_node(RelOptInfo, lcr2);
+			lcr1 = lnext(parts1, lcr1);
+			lcr2 = lnext(parts2, lcr2);
+		}
+		else
+		{
+			child_rel1 = rel1->part_rels[cnt_parts];
+			child_rel2 = rel2->part_rels[cnt_parts];
+		}
+
+		rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1));
+		rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2));
+
 		/*
 		 * Check for cases where we can prove that this segment of the join
 		 * returns no rows, due to one or both inputs being empty (including
@@ -1522,6 +1619,8 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 												 child_sjinfo,
 												 child_sjinfo->jointype);
 			joinrel->part_rels[cnt_parts] = child_joinrel;
+			joinrel->all_partrels = bms_add_members(joinrel->all_partrels,
+													child_joinrel->relids);
 		}
 
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
@@ -1738,3 +1837,99 @@ match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op)
 
 	return -1;
 }
+
+/*
+ * get_matching_part_pairs
+ *		Generate join pairs of partitions for the two inputs
+ */
+static void
+get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2)
+{
+	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
+	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
+	int 		cnt_parts;
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
+	{
+		RelOptInfo *child_joinrel = joinrel->part_rels[cnt_parts];
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		Relids		child_relids1;
+		Relids		child_relids2;
+
+		/*
+		 * If this segment of the join is empty, it means that this segment
+		 * was ignored when previously creating child-join paths for it in
+		 * try_partitionwise_join() as it would not contribute to the join
+		 * result, due to one or both inputs being empty; add NULL to each of
+		 * the given lists so that this segment will be ignored again in that
+		 * function.
+		 */
+		if (!child_joinrel)
+		{
+			*parts1 = lappend(*parts1, NULL);
+			*parts2 = lappend(*parts2, NULL);
+			continue;
+		}
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel1 side.
+		 */
+		child_relids1 = bms_intersect(child_joinrel->relids,
+									  rel1->all_partrels);
+		Assert(bms_num_members(child_relids1) == bms_num_members(rel1->relids));
+
+		/*
+		 * Get a child rel for rel1 with the relids.  Note that we should have
+		 * the child rel even if rel1 is a join rel, because in that case the
+		 * partitions specified in the relids would have matching/overlapping
+		 * boundaries, so those partitions should be considered as ones to be
+		 * joined even when planning partitionwise joins of rel1, meaning that
+		 * the child rel would have been built by the time we get here.
+		 */
+		if (rel1_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids1);
+
+			child_rel1 = find_base_rel(root, varno);
+		}
+		else
+			child_rel1 = find_join_rel(root, child_relids1);
+		Assert(child_rel1);
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel2 side.
+		 */
+		child_relids2 = bms_intersect(child_joinrel->relids,
+									  rel2->all_partrels);
+		Assert(bms_num_members(child_relids2) == bms_num_members(rel2->relids));
+
+		/*
+		 * Get a child rel for rel2 with the relids.  See above comments.
+		 */
+		if (rel2_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids2);
+
+			child_rel2 = find_base_rel(root, varno);
+		}
+		else
+			child_rel2 = find_join_rel(root, child_relids2);
+		Assert(child_rel2);
+
+		/*
+		 * The join of rel1 and rel2 is legal, so is the join of the child
+		 * rels obtained above; add them to the given lists as a join pair
+		 * producing this join segment.
+		 */
+		*parts1 = lappend(*parts1, child_rel1);
+		*parts2 = lappend(*parts2, child_rel2);
+	}
+}
diff --git a/src/backend/optimizer/util/inherit.c b/src/backend/optimizer/util/inherit.c
index 7db67fdf34..3132fd35a5 100644
--- a/src/backend/optimizer/util/inherit.c
+++ b/src/backend/optimizer/util/inherit.c
@@ -376,6 +376,8 @@ expand_partitioned_rtentry(PlannerInfo *root, RelOptInfo *relinfo,
 		/* Create the otherrel RelOptInfo too. */
 		childrelinfo = build_simple_rel(root, childRTindex, relinfo);
 		relinfo->part_rels[i] = childrelinfo;
+		relinfo->all_partrels = bms_add_members(relinfo->all_partrels,
+												childrelinfo->relids);
 
 		/* If this child is itself partitioned, recurse */
 		if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 374f93890b..1df6da64ed 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -240,10 +240,12 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->has_eclass_joins = false;
 	rel->consider_partitionwise_join = false;	/* might get changed later */
 	rel->part_scheme = NULL;
-	rel->nparts = 0;
+	rel->nparts = -1;
 	rel->boundinfo = NULL;
+	rel->merged = false;
 	rel->partition_qual = NIL;
 	rel->part_rels = NULL;
+	rel->all_partrels = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->partitioned_child_rels = NIL;
@@ -653,10 +655,12 @@ build_join_rel(PlannerInfo *root,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -829,10 +833,12 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -1639,7 +1645,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1652,24 +1658,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1681,17 +1669,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	/*
 	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * joining relations.
+	 *
+	 * Because of restrictions in partition_bounds_merge(), not every pair of
+	 * joining relations (including the one presented to this function) for the
+	 * same joinrel can use partition-wise join or has both the relations
+	 * partitioned. Hence we calculate the partition bounds, number of
+	 * partitions and child-join relations of the join relation when and if we
+	 * find a suitable pair in try_partition_wise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 95af37b9c7..3c7b6030c0 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -25,6 +25,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/pathnodes.h"
 #include "parser/parse_coerce.h"
 #include "partitioning/partbounds.h"
 #include "partitioning/partdesc.h"
@@ -68,6 +69,23 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+typedef struct PartitionMap
+{
+	int			nparts;			/* number of partitions */
+	int		   *merged_indexes;	/* indexes of merged partitions */
+	bool	   *merged;			/* flags to indicate whether partitions are
+								 * merged with non-dummy partitions */
+	bool		did_remapping;	/* did we remap partitions? */
+	int		   *old_indexes;	/* old indexes of merged partitions if
+								 * did_remapping */
+} PartitionMap;
+
+#define compare_range_bounds(partnatts, partsupfunc, partcollations, \
+							 bound1, bound2) \
+	(partition_rbound_cmp(partnatts, partsupfunc, partcollations, \
+						  (bound1)->datums, (bound1)->kind, (bound1)->lower, \
+						  bound2))
+
 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);
@@ -107,6 +125,104 @@ static void get_range_key_properties(PartitionKey key, int keynum,
 									 Expr **keyCol,
 									 Const **lower_val, Const **upper_val);
 static List *get_range_nulltest(PartitionKey key);
+static PartitionBoundInfo partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts);
+static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							JoinType jointype,
+							List **outer_parts, List **inner_parts);
+static void init_partition_map(RelOptInfo *rel, PartitionMap *map);
+static void free_partition_map(PartitionMap *map);
+static bool is_dummy_partition(RelOptInfo *rel, int part_index);
+static int map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_part, int inner_part, int *next_index);
+static int merge_partition_with_dummy(PartitionMap *map, int index,
+						   int *next_index);
+static int process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index);
+static int process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index);
+static void fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes);
+static void generate_matching_part_pairs(RelOptInfo *outer_rel,
+							 RelOptInfo *inner_rel,
+							 PartitionMap *outer_map,
+							 PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_part_list,
+							 List **inner_part_list);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+							  List *merged_datums, List *merged_indexes,
+							  List *merged_contents, int null_index,
+							  int default_index);
+static int get_range_partition(RelOptInfo *rel,
+							   PartitionBoundInfo bi,
+							   int *lb_index,
+							   PartitionRangeBound *lb,
+							   PartitionRangeBound *ub);
+static int get_range_partition_internal(PartitionBoundInfo bi,
+										int *lb_pos,
+										PartitionRangeBound *lb,
+										PartitionRangeBound *ub);
+static bool compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+						 Oid *partcollations,
+						 PartitionRangeBound *outer_lb,
+						 PartitionRangeBound *outer_ub,
+						 PartitionRangeBound *inner_lb,
+						 PartitionRangeBound *inner_ub,
+						 int *lb_cmpval, int *ub_cmpval);
+static void get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations, JoinType jointype,
+						PartitionRangeBound *outer_lb,
+						PartitionRangeBound *outer_ub,
+						PartitionRangeBound *inner_lb,
+						PartitionRangeBound *inner_ub,
+						int	lb_cmpval, int ub_cmpval,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub);
+static void add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub,
+						int merged_index,
+						List **merged_datums,
+						List **merged_kinds,
+						List **merged_indexes);
+static void merge_default_partitions(PartitionMap *outer_map,
+						 PartitionMap *inner_map,
+						 bool outer_has_default,
+						 bool inner_has_default,
+						 int outer_default,
+						 int inner_default,
+						 JoinType jointype,
+						 int *next_index,
+						 int *default_index);
+static void merge_null_partitions(PartitionMap *outer_map,
+					  PartitionMap *inner_map,
+					  bool outer_has_null,
+					  bool inner_has_null,
+					  int outer_null,
+					  int inner_null,
+					  JoinType jointype,
+					  int *next_index,
+					  int *null_index);
 
 /*
  * get_qual_from_partbound
@@ -2999,3 +3115,1599 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * partition_bounds_merge
+ *
+ * This function builds and returns the partition bounds for a join relation
+ * between input relations, creating two lists of partitions, which are
+ * returned to *outer_parts and *inner_parts respectively.  The lists contain
+ * the same number of partitions, and the partitions at the same positions in
+ * the lists indicate join pairs used for partitioned join.
+ *
+ * This function returns NULL, setting *outer_parts and *inner_parts to NIL,
+ * if a partition on one side matches multiple partitions on the other side,
+ * in which case we currently don't support partitioned join.
+ */
+PartitionBoundInfo
+partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype, List **outer_parts,
+					   List **inner_parts)
+{
+	PartitionBoundInfo 	merged_bounds;
+	PartitionBoundInfo 	outer_binfo = outer_rel->boundinfo,
+						inner_binfo = inner_rel->boundinfo;
+	char				strategy = outer_binfo->strategy;
+
+	/* Bail out if partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	if (jointype != JOIN_LEFT && jointype != JOIN_INNER &&
+		jointype != JOIN_SEMI && jointype != JOIN_ANTI &&
+		jointype != JOIN_FULL)
+		elog(ERROR, "unexpected join type %d", jointype);
+
+	*outer_parts = NIL;
+	*inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+			merged_bounds = NULL;
+
+			break;
+
+		case PARTITION_STRATEGY_LIST:
+			merged_bounds = partition_list_bounds_merge(partsupfunc,
+														partcollation,
+														outer_rel,
+														inner_rel,
+														jointype,
+														outer_parts,
+														inner_parts);
+			break;
+
+		case PARTITION_STRATEGY_RANGE:
+			merged_bounds = partition_range_bounds_merge(partnatts,
+														 partsupfunc,
+														 partcollation,
+														 outer_rel,
+														 inner_rel,
+														 jointype,
+														 outer_parts,
+														 inner_parts);
+			break;
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", strategy);
+	}
+
+	Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+	Assert(list_length(*outer_parts) == list_length(*inner_parts));
+
+	return merged_bounds;
+}
+
+/*
+ * get_range_partition
+ *		Returns the next non-dummy partition of a given range-partitioned
+ *		relation
+ *
+ * *lb and *ub are set to the lower and upper bounds of that partition
+ * respectively, and *lb_index is advanced to the next lower bound, if any.
+ */
+static int
+get_range_partition(RelOptInfo *rel,
+					PartitionBoundInfo bi,
+					int *lb_index,
+					PartitionRangeBound *lb,
+					PartitionRangeBound *ub)
+{
+	int			part_index;
+
+	Assert(bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	do {
+		part_index = get_range_partition_internal(bi, lb_index, lb, ub);
+		if (part_index == -1)
+			return -1;
+	} while (is_dummy_partition(rel, part_index));
+
+	return part_index;
+}
+
+static int
+get_range_partition_internal(PartitionBoundInfo bi,
+							 int *lb_index,
+							 PartitionRangeBound *lb,
+							 PartitionRangeBound *ub)
+{
+	/* Return the index as -1 if we've exhausted all the lower bounds. */
+	if (*lb_index >= bi->ndatums)
+		return -1;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(*lb_index + 1 < bi->ndatums);
+
+	lb->index = bi->indexes[*lb_index];
+	lb->kind = bi->kind[*lb_index];
+	lb->datums = bi->datums[*lb_index];
+	lb->lower = true;
+	ub->index = bi->indexes[*lb_index + 1];
+	ub->kind = bi->kind[*lb_index + 1];
+	ub->datums = bi->datums[*lb_index + 1];
+	ub->lower = false;
+
+	/* The partition index of an upper bound should be valid. */
+	Assert(ub->index >= 0);
+
+	/*
+	 * Advance the lower bound; if there are no bounds left beyond the upper
+	 * bound, we have reached the last lower bound.
+	 */
+	if (*lb_index + 2 >= bi->ndatums)
+		*lb_index = bi->ndatums;
+	else
+	{
+		/*
+		 * If the index assigned to the bound next to the upper bound isn't
+		 * valid, that is the lower bound of the next range partition; else,
+		 * the upper bound of the current range partition is also the lower
+		 * bound of the next range partition.
+		 */
+		if (bi->indexes[*lb_index + 2] < 0)
+			*lb_index = *lb_index + 2;
+		else
+			*lb_index = *lb_index + 1;
+	}
+
+	return ub->index;
+}
+
+/*
+ * compare_range_partitions
+ *		Compares the bounds of two range partitions, and returns true if the
+ *		ranges of the partitions overlap, false otherwise
+ *
+ * *lb_cmpval is set to -1, 0 or 1 if the outer partition's lower bound is
+ * lower than, equal to or higher than the inner partition's lower bound
+ * respectively.  Likewise, *ub_cmpval is set to -1, 0 or 1 if the outer
+ * partition's upper bound is lower than, equal to or higher than the inner
+ * partition's upper bound respectively.
+ */
+static bool
+compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+						 Oid *partcollations,
+						 PartitionRangeBound *outer_lb,
+						 PartitionRangeBound *outer_ub,
+						 PartitionRangeBound *inner_lb,
+						 PartitionRangeBound *inner_ub,
+						 int *lb_cmpval, int *ub_cmpval)
+{
+	/*
+	 * Check to see if the upper bound of the outer partition is lower than
+	 * the lower bound of the inner partition; in which case the partitions
+	 * aren't overlapping.
+	 */
+	if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+							 outer_ub, inner_lb) < 0)
+	{
+		*lb_cmpval = -1;
+		*ub_cmpval = -1;
+		return false;
+	}
+
+	/*
+	 * Check to see if the lower bound of the outer partition is higher than
+	 * the upper bound of the inner partition; in which case the partitions
+	 * aren't overlapping.
+	 */
+	if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+							 outer_lb, inner_ub) > 0)
+	{
+		*lb_cmpval = 1;
+		*ub_cmpval = 1;
+		return false;
+	}
+
+	/* All other cases indicate overlapping partitions. */
+	*lb_cmpval = compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									  outer_lb, inner_lb);
+	*ub_cmpval = compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									  outer_ub, inner_ub);
+	return true;
+}
+
+/*
+ * get_merged_range_bounds
+ *		Given the bounds of range partitions to be join, determine the range
+ *		bounds of the merged partition produced from the range partitions
+ *
+ * *merged_lb and *merged_ub are set to the lower and upper bounds of the
+ * merged partition.
+ */
+static void
+get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations, JoinType jointype,
+						PartitionRangeBound *outer_lb,
+						PartitionRangeBound *outer_ub,
+						PartitionRangeBound *inner_lb,
+						PartitionRangeBound *inner_ub,
+						int	lb_cmpval, int ub_cmpval,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub)
+{
+	Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								outer_lb, inner_lb) == lb_cmpval);
+	Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								outer_ub, inner_ub) == ub_cmpval);
+
+	/*
+	 * An outer join will have all the rows from the outer side, so merged
+	 * bounds will be same as the outer bounds. An inner join will have rows
+	 * that fit both the bounds, thus lower merged bound will be higher of two
+	 * lower bounds and upper merged bound will be lower of the two upper
+	 * bounds.
+	 */
+	switch (jointype)
+	{
+		case JOIN_INNER:
+		case JOIN_SEMI:
+			*merged_lb = (lb_cmpval > 0) ? *outer_lb : *inner_lb;
+			*merged_ub = (ub_cmpval < 0) ? *outer_ub : *inner_ub;
+			break;
+
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+			*merged_ub = *outer_ub;
+			*merged_lb = *outer_lb;
+			break;
+
+		case JOIN_FULL:
+			*merged_lb = (lb_cmpval < 0) ? *outer_lb : *inner_lb;
+			*merged_ub = (ub_cmpval > 0) ? *outer_ub : *inner_ub;
+			break;
+
+		default:
+			elog(ERROR, "unexpected join type %d", jointype);
+	}
+}
+
+/*
+ * add_merged_range_bounds
+ *		Add the range bounds of a merged partition to the lists of range
+ *		bounds
+ */
+static void
+add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub,
+						int merged_index,
+						List **merged_datums,
+						List **merged_kinds,
+						List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		/* First merged partition */
+		Assert(!*merged_kinds && !*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		Assert(*merged_kinds && *merged_indexes);
+
+		/* Get the last upper bound. */
+		prev_ub.index = llast_int(*merged_indexes);
+		prev_ub.datums = (Datum *) llast(*merged_datums);
+		prev_ub.kind = (PartitionRangeDatumKind *) llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		/*
+		 * We pass to partition_rbound_cmp() lower1 as false to prevent it
+		 * from considering the last upper bound to be smaller than the lower
+		 * bound of the merged partition when the values of the two range
+		 * bounds compare equal.
+		 */
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  merged_lb->datums, merged_lb->kind,
+									  false, &prev_ub);
+		Assert(cmpval >= 0);
+	}
+
+	/*
+	 * If the lower bound is higher than the last upper bound, add the lower
+	 * bound with the index as -1 indicating that that is a lower bound; else,
+	 * the last upper bound will be reused as the lower bound of the merged
+	 * partition, so skip this.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, merged_lb->datums);
+		*merged_kinds = lappend(*merged_kinds, merged_lb->kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	/* Add the upper bound and index of the merged partition. */
+	*merged_datums = lappend(*merged_datums, merged_ub->datums);
+	*merged_kinds = lappend(*merged_kinds, merged_ub->kind);
+	*merged_indexes = lappend_int(*merged_indexes, merged_index);
+}
+
+/*
+ * partition_range_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for range partitioned tables.
+ */
+static PartitionBoundInfo
+partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
+							 Oid *partcollations,
+							 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 JoinType jointype,
+							 List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			outer_part;
+	int			inner_part;
+	int			outer_lb_index;
+	int			inner_lb_index;
+	PartitionRangeBound outer_lb;
+	PartitionRangeBound outer_ub;
+	PartitionRangeBound inner_lb;
+	PartitionRangeBound inner_ub;
+	int			next_index = 0;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_kinds = NIL;
+	List	   *merged_indexes = NIL;
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * If the default partitions (if any) have been proven empty, deem them
+	 * non-existent.
+	 */
+	if (outer_has_default && is_dummy_partition(outer_rel, outer_default))
+		outer_has_default = false;
+	if (inner_has_default && is_dummy_partition(inner_rel, inner_default))
+		inner_has_default = false;
+
+	/*
+	 * Merge the ranges (partitions) from both sides. Every iteration compares
+	 * a pair of ranges, one from each side, advancing to the next range from
+	 * the side with smaller upper range bound. If upper bounds of ranges from
+	 * both sides match exactly, both the sides are advanced. For a given pair
+	 * of ranges, we decide whether the corresponding partition match or not.
+	 * lb_index, for inner or outer side, keeps track of the index of lower bound
+	 * datum in PartitionBoundInfo::datums of that side.
+	 */
+	outer_lb_index = inner_lb_index = 0;
+	outer_part = get_range_partition(outer_rel, outer_bi, &outer_lb_index,
+									 &outer_lb, &outer_ub);
+	inner_part = get_range_partition(inner_rel, inner_bi, &inner_lb_index,
+									 &inner_lb, &inner_ub);
+	while (outer_part >= 0 || inner_part >= 0)
+	{
+		PartitionRangeBound merged_lb;
+		PartitionRangeBound merged_ub;
+		int			merged_index = -1;
+		bool		overlap;
+		int			ub_cmpval;
+		int			lb_cmpval;
+
+		if (outer_part >= 0)
+			Assert(outer_map.merged_indexes[outer_part] == -1 &&
+				   outer_map.merged[outer_part] == false);
+		if (inner_part >= 0)
+			Assert(inner_map.merged_indexes[inner_part] == -1 &&
+				   inner_map.merged[inner_part] == false);
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining partitions on the side
+		 * which finishes later. For that we set the comparison parameters
+		 * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if
+		 * the side which finishes earlier has an extra partition with lower
+		 * and upper bounds higher than any other partition of the unfinished
+		 * side. That way we advance the partitions on that side till all of
+		 * them are  exhausted.
+		 */
+		if (outer_part == -1)
+		{
+			overlap = false;
+			lb_cmpval = 1;
+			ub_cmpval = 1;
+		}
+		else if (inner_part == -1)
+		{
+			overlap = false;
+			lb_cmpval = -1;
+			ub_cmpval = -1;
+		}
+		else
+			overlap = compare_range_partitions(partnatts, partsupfuncs,
+											   partcollations,
+											   &outer_lb, &outer_ub,
+											   &inner_lb, &inner_ub,
+											   &lb_cmpval, &ub_cmpval);
+
+		if (overlap)
+		{
+			/* The ranges of partitions overlap; form a join pair of them. */
+
+			PartitionRangeBound save_outer_ub;
+			PartitionRangeBound save_inner_ub;
+
+			/* Get the bounds of the merged partition. */
+			get_merged_range_bounds(partnatts, partsupfuncs,
+									partcollations, jointype,
+									&outer_lb, &outer_ub,
+									&inner_lb, &inner_ub,
+									lb_cmpval, ub_cmpval,
+									&merged_lb, &merged_ub);
+
+			/*
+			 * Get the index of the merged partition.  Both partitions aren't
+			 * merged yet, so the partitions should be merged successfully.
+			 */
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
+													outer_part, inner_part,
+													&next_index);
+			Assert(merged_index >= 0);
+
+			/* Save the upper bounds of both partitions for use below. */
+			save_outer_ub = outer_ub;
+			save_inner_ub = inner_ub;
+
+			/* Move to the next pair of partitions. */
+			outer_part = get_range_partition(outer_rel, outer_bi, &outer_lb_index,
+											 &outer_lb, &outer_ub);
+			inner_part = get_range_partition(inner_rel, inner_bi, &inner_lb_index,
+											 &inner_lb, &inner_ub);
+
+			/*
+			 * If the range of a partition on one side overlaps the range of
+			 * the next partition on the other side, that will cause the
+			 * partition on one side to match at least two partitions on the
+			 * other side, which is the case that we currently don't support
+			 * partitioned join for; give up.
+			 */
+			if (ub_cmpval > 0 && inner_part >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &save_outer_ub, &inner_lb) > 0)
+				goto cleanup;
+			if (ub_cmpval < 0 && outer_part >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &outer_lb, &save_inner_ub) < 0)
+				goto cleanup;
+
+			/*
+			 * A row from a non-overlapping portion (if any) of a partition
+			 * on one side might find its join partner in the default
+			 * partition (if any) on the other side, causing the same
+			 * situation as above; if so, give up.
+			 */
+			if ((outer_has_default && (lb_cmpval > 0 || ub_cmpval < 0)) ||
+				(inner_has_default && (lb_cmpval < 0 || ub_cmpval > 0)))
+				goto cleanup;
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* Upper bound of inner range higher than that of the outer. */
+
+			merged_lb = outer_lb;
+			merged_ub = outer_ub;
+
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				merged_index = process_outer_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   outer_part,
+													   inner_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+			}
+
+			/* Move to the next partition on the outer side. */
+			outer_part = get_range_partition(outer_rel, outer_bi, &outer_lb_index,
+											 &outer_lb, &outer_ub);
+		}
+		else
+		{
+			/* Upper bound of outer range higher than that of the inner. */
+			Assert(ub_cmpval > 0);
+
+			merged_lb = inner_lb;
+			merged_ub = inner_ub;
+
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				merged_index = process_inner_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   inner_part,
+													   outer_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+			}
+
+			/* Move to the next partition on the inner side. */
+			inner_part = get_range_partition(inner_rel, inner_bi, &inner_lb_index,
+											 &inner_lb, &inner_ub);
+		}
+
+		if (merged_index >= 0 && merged_index != default_index)
+		{
+			/* Add the range bounds of the merged partition. */
+			add_merged_range_bounds(partnatts, partsupfuncs, partcollations,
+									&merged_lb, &merged_ub, merged_index,
+									&merged_datums, &merged_kinds,
+									&merged_indexes);
+		}
+	}
+
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+		merge_default_partitions(&outer_map, &inner_map,
+								 outer_has_default, inner_has_default,
+								 outer_default, inner_default,
+								 jointype, &next_index, &default_index);
+	else
+		Assert(default_index == -1);
+
+	/*
+	 * Unlike the case for list partitioning, we wouldn't have re-merged
+	 * partitions, so did_remapping should be left alone.
+	 */
+	Assert(!outer_map.did_remapping && !inner_map.did_remapping);
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 &outer_map, &inner_map,
+								 next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes,
+													  merged_kinds,
+													  -1, default_index);
+	}
+
+cleanup:
+	/* Free local memory before returning. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	list_free(merged_kinds);
+
+	return merged_bounds;
+}
+
+/*
+ * partition_list_bounds_merge
+ *
+ * partition_bounds_merge()'s arm for list partitioned tables.
+ *
+ */
+static PartitionBoundInfo
+partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *partcollation,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							JoinType jointype,
+							List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	int			outer_default = outer_bi->default_index;
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			inner_default = inner_bi->default_index;
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			next_index = 0;
+	int			null_index = -1;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+	int			cnto;
+	int			cnti;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+
+	/* List partitions do not require unbounded ranges. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * If the default partitions (if any) have been proven empty, deem them
+	 * non-existent.
+	 */
+	if (outer_has_default && is_dummy_partition(outer_rel, outer_default))
+		outer_has_default = false;
+	if (inner_has_default && is_dummy_partition(inner_rel, inner_default))
+		inner_has_default = false;
+
+	/*
+	 * Merge the list value datums from both sides. Every iteration compares a
+	 * pair of datums, one from each side, advancing to the next datum from the
+	 * side with smaller datum. If datums from both sides match exactly, both
+	 * the sides are advanced. For a given pair of datums, we decide whether
+	 * the corresponding partition match or not.
+	 */
+	cnto = cnti = 0;
+	while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums)
+	{
+		Datum	   *merged_datum = NULL;
+		int			merged_index = -1;
+		int			o_index = -1;
+		int			i_index = -1;
+		Datum	   *odatums;
+		Datum	   *idatums;
+		int			cmpval;
+
+		if (cnto < outer_bi->ndatums)
+		{
+			/*
+			 * If the partition on the outer side has been proven empty, ignore
+			 * it and move to the next datum on the outer side.
+			 */
+			o_index = outer_bi->indexes[cnto];
+			if (is_dummy_partition(outer_rel, o_index))
+			{
+				cnto++;
+				continue;
+			}
+		}
+		if (cnti < inner_bi->ndatums)
+		{
+			/*
+			 * If the partition on the inner side has been proven empty, ignore
+			 * it and move to the next datum on the inner side.
+			 */
+			i_index = inner_bi->indexes[cnti];
+			if (is_dummy_partition(inner_rel, i_index))
+			{
+				cnti++;
+				continue;
+			}
+		}
+
+		/* Get the list datums of the next pair of partitions. */
+		odatums = cnto < outer_bi->ndatums ? outer_bi->datums[cnto] : NULL;
+		idatums = cnti < inner_bi->ndatums ? inner_bi->datums[cnti] : NULL;
+
+		/*
+		 * We run this loop till both the sides finish. This allows to avoid
+		 * duplicating code to handle the remaining datums on the side which
+		 * finishes later. For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra datum higher than any other datum on the unfinished side.
+		 * That way we advance the datums on the unfinished side till all of
+		 * its datums are exhausted.
+		 */
+		if (cnto >= outer_bi->ndatums)
+			cmpval = 1;
+		else if (cnti >= inner_bi->ndatums)
+			cmpval = -1;
+		else
+		{
+			Assert(odatums != NULL && idatums != NULL);
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 odatums[0],
+													 idatums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			/*
+			 * Datums match. Rows on either side with these datums as partition
+			 * key value will join and will be part of the partition of the
+			 * join result produced by joining the corresponding partitions.
+			 * Match the corresponding partitions and if successful, add the
+			 * datum to the list of merged datums with index of merged
+			 * partition containing it.
+			 */
+			merged_datum = odatums;
+			Assert(o_index >= 0 && i_index >= 0);
+			merged_index = map_and_merge_partitions(&outer_map, &inner_map,
+													o_index, i_index,
+													&next_index);
+
+			if (merged_index == -1)
+				goto cleanup;
+
+			/* Move to the next pair of bounds. */
+			cnto++;
+			cnti++;
+		}
+		else if (cmpval < 0)
+		{
+			Assert(cnto < outer_bi->ndatums);
+			Assert(o_index >= 0);
+
+			/* A datum missing from the inner side. */
+			merged_datum = odatums;
+
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				Assert(o_index >= 0);
+				merged_index = process_outer_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   o_index,
+													   inner_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+			}
+
+			/* Move to the next datum on the outer side. */
+			cnto++;
+		}
+		else
+		{
+			Assert(cmpval > 0);
+			Assert(cnti < inner_bi->ndatums);
+			Assert(i_index >= 0);
+
+			/* A datum missing from the outer side. */
+			merged_datum = idatums;
+
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				Assert(i_index >= 0);
+				merged_index = process_inner_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   i_index,
+													   outer_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+			}
+
+			/* Move to the next datum on the inner side. */
+			cnti++;
+		}
+
+		/*
+		 * Add the list value with appropriate index in the list of datums, if
+		 * we have associated a partition with this list value.
+		 */
+		if (merged_index >= 0 && merged_index != default_index)
+		{
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+			merged_datums = lappend(merged_datums, merged_datum);
+		}
+	}
+
+	/*
+	 * If the NULL partitions (if any) have been proven empty, deem them
+	 * non-existent.
+	 */
+	if (outer_has_null &&
+		is_dummy_partition(outer_rel, outer_bi->null_index))
+		outer_has_null = false;
+	if (inner_has_null &&
+		is_dummy_partition(inner_rel, inner_bi->null_index))
+		inner_has_null = false;
+
+	/* Merge null partitions if any. */
+	if (outer_has_null || inner_has_null)
+		merge_null_partitions(&outer_map, &inner_map,
+							  outer_has_null, inner_has_null,
+							  outer_bi->null_index, inner_bi->null_index,
+							  jointype, &next_index, &null_index);
+	else
+		Assert(null_index == -1);
+
+	/* Merge default partitions if any. */
+	if (outer_has_default || inner_has_default)
+		merge_default_partitions(&outer_map, &inner_map,
+								 outer_has_default, inner_has_default,
+								 outer_default, inner_default,
+								 jointype, &next_index, &default_index);
+	else
+		Assert(default_index == -1);
+
+	/* Fix the merged_indexes list if necessary. */
+	if (outer_map.did_remapping || inner_map.did_remapping)
+	{
+		Assert(jointype == JOIN_FULL);
+		fix_merged_indexes(&outer_map, &inner_map,
+						   next_index, merged_indexes);
+	}
+
+	/* Use maps to match partition from the joining relations. */
+	generate_matching_part_pairs(outer_rel, inner_rel,
+								 &outer_map, &inner_map,
+								 next_index,
+								 outer_parts, inner_parts);
+
+	/* Craft a PartitionBoundInfo to return. */
+	if (*outer_parts && *inner_parts)
+	{
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) <= next_index);
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_indexes, NIL,
+													  null_index, default_index);
+	}
+
+cleanup:
+	/* Free local memory before returning. */
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+	list_free(merged_datums);
+	list_free(merged_indexes);
+
+	return merged_bounds;
+}
+
+/*
+ * init_partition_map
+ *
+ * Initialize a PartitionMap struct for given relation.
+ */
+static void
+init_partition_map(RelOptInfo *rel, PartitionMap *map)
+{
+	int			nparts = rel->nparts;
+	int			i;
+
+	map->nparts = nparts;
+	map->merged_indexes = (int *) palloc(sizeof(int) * nparts);
+	map->merged = (bool *) palloc(sizeof(bool) * nparts);
+	map->did_remapping = false;
+	map->old_indexes = (int *) palloc(sizeof(int) * nparts);
+	for (i = 0; i < nparts; i++)
+	{
+		map->merged_indexes[i] = map->old_indexes[i] = -1;
+		map->merged[i] = false;
+	}
+}
+
+/*
+ * free_partition_map
+ */
+static void
+free_partition_map(PartitionMap *map)
+{
+	pfree(map->merged_indexes);
+	pfree(map->merged);
+	pfree(map->old_indexes);
+}
+
+/*
+ * is_dummy_partition --- has partition been proven empty?
+ */
+static bool
+is_dummy_partition(RelOptInfo *rel, int part_index)
+{
+	RelOptInfo *part_rel;
+
+	Assert(part_index >= 0);
+	part_rel = rel->part_rels[part_index];
+	if (part_rel == NULL || IS_DUMMY_REL(part_rel))
+		return true;
+	return false;
+}
+
+/*
+ * map_and_merge_partitions
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given partitions.
+ */
+static int
+map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						 int outer_index, int inner_index, int *next_index)
+{
+	int 		outer_merged_index;
+	bool 		outer_merged;
+	int 		inner_merged_index;
+	bool 		inner_merged;
+
+	Assert(outer_index >= 0 && outer_index < outer_map->nparts);
+	outer_merged_index = outer_map->merged_indexes[outer_index];
+	outer_merged = outer_map->merged[outer_index];
+	Assert(inner_index >= 0 && inner_index < inner_map->nparts);
+	inner_merged_index = inner_map->merged_indexes[inner_index];
+	inner_merged = inner_map->merged[inner_index];
+
+	/*
+	 * Handle cases where both partitions are mapped to merged partitions.
+	 */
+	if (outer_merged_index >= 0 && inner_merged_index >= 0)
+	{
+		/*
+		 * If the mereged partitions are the same, no need to do anything;
+		 * return the index of the merged partition.  Otherwise, if both
+		 * partitions are merged with dummy partitions, re-merge them; map
+		 * them to the merged partition with the smaller of the two merged
+		 * indexes and return the smaller index.  Otherwise they can't be
+		 * merged, so return -1.
+		 */
+		if (outer_merged_index == inner_merged_index)
+		{
+			Assert(outer_merged);
+			Assert(inner_merged);
+			return outer_merged_index;
+		}
+		if (!outer_merged && !inner_merged)
+		{
+			/*
+			 * Note that we will fix the larger index that have been added to
+			 * the merged_indexes list so far in fix_merged_indexes().
+			 */
+			if (outer_merged_index < inner_merged_index)
+			{
+				outer_map->merged[outer_index] = true;
+				inner_map->merged_indexes[inner_index] = outer_merged_index;
+				inner_map->merged[inner_index] = true;
+				inner_map->did_remapping = true;
+				inner_map->old_indexes[inner_index] = inner_merged_index;
+				return outer_merged_index;
+			}
+			else
+			{
+				inner_map->merged[inner_index] = true;
+				outer_map->merged_indexes[outer_index] = inner_merged_index;
+				outer_map->merged[outer_index] = true;
+				outer_map->did_remapping = true;
+				outer_map->old_indexes[outer_index] = outer_merged_index;
+				return inner_merged_index;
+			}
+		}
+		return -1;
+	}
+
+	/* At least one partition isn't mapped to a merged partition. */
+	Assert(outer_merged_index == -1 || inner_merged_index == -1);
+
+	/*
+	 * If neither of partitions isn't mapped, assign them a new merged
+	 * partition and return the index of the merged partition.  Otherwise, if
+	 * one of partitions is merged with a dummy relation (and the other isn't
+	 * merged), re-merge it with the other, with the same index, and return
+	 * the index.  Otherwise they can't be merged, so return -1.
+	 */
+	if (outer_merged_index == -1 && inner_merged_index == -1)
+	{
+		int 		merged_index = *next_index;
+
+		Assert(!outer_merged);
+		Assert(!inner_merged);
+		outer_map->merged_indexes[outer_index] = merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged_indexes[inner_index] = merged_index;
+		inner_map->merged[inner_index] = true;
+		*next_index = *next_index + 1;
+		return merged_index;
+	}
+	if (outer_merged_index >= 0 && !outer_map->merged[outer_index])
+	{
+		Assert(inner_merged_index == -1);
+		Assert(!inner_merged);
+		inner_map->merged_indexes[inner_index] = outer_merged_index;
+		inner_map->merged[inner_index] = true;
+		outer_map->merged[outer_index] = true;
+		return outer_merged_index;
+	}
+	if (inner_merged_index >= 0 && !inner_map->merged[inner_index])
+	{
+		Assert(outer_merged_index == -1);
+		Assert(!outer_merged);
+		outer_map->merged_indexes[outer_index] = inner_merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged[inner_index] = true;
+		return inner_merged_index;
+	}
+	return -1;
+}
+
+/*
+ * merge_partition_with_dummy
+ *
+ * *next_index is incremented.
+ */
+static int
+merge_partition_with_dummy(PartitionMap *map, int index, int *next_index)
+{
+	int 		merged_index = *next_index;
+
+	Assert(index >= 0 && index < map->nparts);
+	Assert(map->merged_indexes[index] == -1);
+	Assert(!map->merged[index]);
+	map->merged_indexes[index] = merged_index;
+	/* Leave the merged flag alone! */
+	*next_index = *next_index + 1;
+	return merged_index;
+}
+
+/*
+ * process_outer_partition
+ *
+ * Determine the merged partition associated with the given outer partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given outer partition.
+ */
+static int
+process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index)
+{
+	int 		merged_index = -1;
+
+	Assert(outer_index >= 0);
+
+	/*
+	 * If the inner side has the default partition, the outer partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an outer join, in which case the outer partition has to be
+	 * scanned all the way anyway; if the outer partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the outer partition with a dummy partition.
+	 */
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0);
+
+		/*
+		 * If the outer side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (outer_has_default)
+			return -1;
+
+		merged_index = map_and_merge_partitions(outer_map, inner_map,
+												outer_index, inner_default,
+												next_index);
+		if (merged_index == -1)
+			return -1;
+
+		/*
+		 * If this is a FULL join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (*default_index == -1)
+				*default_index = merged_index;
+			else
+				Assert(*default_index == merged_index);
+		}
+	}
+	else
+	{
+		Assert(IS_OUTER_JOIN(jointype));
+		Assert(jointype != JOIN_RIGHT);
+
+		merged_index = outer_map->merged_indexes[outer_index];
+		if (merged_index == -1)
+			merged_index = merge_partition_with_dummy(outer_map, outer_index,
+													  next_index);
+	}
+	return merged_index;
+}
+
+/*
+ * process_inner_partition
+ *
+ * Determine the merged partition associated with the given inner partition.
+ *
+ * *next_index is incremented when creating a new merged partition associated
+ * with the given inner partition.
+ */
+static int
+process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index)
+{
+	int 		merged_index = -1;
+
+	Assert(inner_index >= 0);
+
+	/*
+	 * If the outer side has the default partition, the inner partition has to
+	 * be joined with the default partition; try merging them.  Otherwise, we
+	 * should in an FULL join, in which case the inner partition has to be
+	 * scanned all the way anyway; if the inner partition is already mapped to
+	 * a merged partition, get it, otherwise create a new merged partition by
+	 * merging the inner partition with a dummy partition.
+	 */
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0);
+
+		/*
+		 * If the inner side has the default partition as well, we need to
+		 * merge the default partitions (see merge_default_partitions()); give
+		 * up on it.
+		 */
+		if (inner_has_default)
+			return -1;
+
+		merged_index = map_and_merge_partitions(outer_map, inner_map,
+												outer_default, inner_index,
+												next_index);
+		if (merged_index == -1)
+			return -1;
+
+		/*
+		 * If this is an outer join, the merged partition would act as the
+		 * default partition of the join; record the index in *default_index
+		 * if not done yet.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			if (*default_index == -1)
+				*default_index = merged_index;
+			else
+				Assert(*default_index == merged_index);
+		}
+	}
+	else
+	{
+		Assert(jointype == JOIN_FULL);
+
+		merged_index = inner_map->merged_indexes[inner_index];
+		if (merged_index == -1)
+			merged_index = merge_partition_with_dummy(inner_map, inner_index,
+													  next_index);
+	}
+	return merged_index;
+}
+
+/*
+ * fix_merged_indexes
+ */
+static void
+fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes)
+{
+	int		   *new_indexes;
+	int			merged_index;
+	int			i;
+	ListCell   *lc;
+
+	new_indexes = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		new_indexes[i] = -1;
+
+	/* Build the mapping of old merged indexes to new merged indexes. */
+	if (outer_map->did_remapping)
+	{
+		for (i = 0; i < outer_map->nparts; i++)
+		{
+			merged_index = outer_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = outer_map->merged_indexes[i];
+		}
+	}
+	if (inner_map->did_remapping)
+	{
+		for (i = 0; i < inner_map->nparts; i++)
+		{
+			merged_index = inner_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = inner_map->merged_indexes[i];
+		}
+	}
+
+	/* Fix the merged_indexes list using the mapping. */
+	foreach(lc, merged_indexes)
+	{
+		merged_index = lfirst_int(lc);
+		Assert(merged_index >= 0);
+		if (new_indexes[merged_index] >= 0)
+			lfirst_int(lc) = new_indexes[merged_index];
+	}
+
+	pfree(new_indexes);
+}
+
+/*
+ * generate_matching_part_pairs
+ *
+ * This function produces the list pairs of partitions which produce merged
+ * partitions in the order of merged partition indexes.
+ *
+ * nmerged is the number of merged partitions.
+ *
+ * If successful, the list pairs are returned as two separate lists,
+ * outer_part_list and inner_part_list, one for each side.  Otherwise, those
+ * lists will be set to NIL.
+ */
+static void
+generate_matching_part_pairs(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 PartitionMap *outer_map, PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_part_list, List **inner_part_list)
+{
+	int			outer_nparts = outer_map->nparts;
+	int			inner_nparts = inner_map->nparts;
+	int		   *outer_part_array;
+	int		   *inner_part_array;
+	int			max_nparts;
+	int 		i;
+
+	Assert(outer_nparts == outer_rel->nparts);
+	Assert(inner_nparts == inner_rel->nparts);
+
+	outer_part_array = (int *) palloc(sizeof(int) * nmerged);
+	inner_part_array = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		outer_part_array[i] = inner_part_array[i] = -1;
+
+	/* Set pairs of matching partitions. */
+	max_nparts = Max(outer_nparts, inner_nparts);
+	for (i = 0; i < max_nparts; i++)
+	{
+		if (i < outer_nparts)
+		{
+			int 		merged_index = outer_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				outer_part_array[merged_index] = i;
+			}
+		}
+		if (i < inner_nparts)
+		{
+			int 		merged_index = inner_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				inner_part_array[merged_index] = i;
+			}
+		}
+	}
+
+	*outer_part_list = NIL;
+	*inner_part_list = NIL;
+
+	/* Build the list pairs. */
+	for (i = 0; i < nmerged; i++)
+	{
+		int			outer_part = outer_part_array[i];
+		int			inner_part = inner_part_array[i];
+
+		/*
+		 * If both partitions are dummy, it means the merged partition was
+		 * removed by partition re-merging in map_and_merge_partitions();
+		 * ignore the merged partition
+		 */
+		if (outer_part == -1 && inner_part == -1)
+			continue;
+
+		*outer_part_list = lappend(*outer_part_list, outer_part >= 0 ?
+								   outer_rel->part_rels[outer_part] : NULL);
+		*inner_part_list = lappend(*inner_part_list, inner_part >= 0 ?
+								   inner_rel->part_rels[inner_part] : NULL);
+	}
+
+	pfree(outer_part_array);
+	pfree(inner_part_array);
+}
+
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_indexes, List *merged_kinds,
+							  int null_index, int default_index)
+{
+	int			cnt;
+	PartitionBoundInfo merged_bounds;
+	ListCell   *lc;
+
+	/* We expect the same number of elements in datums and indexes lists. */
+	Assert(list_length(merged_datums) == list_length(merged_indexes));
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = list_length(merged_datums);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_datums) == list_length(merged_kinds));
+		merged_bounds->kind =
+			(PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) *
+												list_length(merged_kinds));
+		cnt = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[cnt++] = lfirst(lc);
+
+		/* There are ndatums+1 indexes in case of range partitions */
+		merged_indexes = lappend_int(merged_indexes, -1);
+	}
+	else
+		merged_bounds->kind = NULL;
+
+	cnt = 0;
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) *
+											  list_length(merged_datums));
+	foreach(lc, merged_datums)
+		merged_bounds->datums[cnt++] = lfirst(lc);
+
+	merged_bounds->indexes = (int *) palloc(sizeof(int) *
+											list_length(merged_indexes));
+	cnt = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[cnt++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * merge_default_partitions
+ *		Merge the default partitions from both sides, if any
+ *
+ * *default_index is set to the index of the default partition of the joinrel,
+ * if appropriate.
+ */
+static void
+merge_default_partitions(PartitionMap *outer_map,
+						 PartitionMap *inner_map,
+						 bool outer_has_default,
+						 bool inner_has_default,
+						 int outer_default,
+						 int inner_default,
+						 JoinType jointype,
+						 int *next_index,
+						 int *default_index)
+{
+	int 		outer_merged_index = -1;
+	int 		inner_merged_index = -1;
+
+	Assert(outer_has_default || inner_has_default);
+
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0 && outer_default < outer_map->nparts);
+		outer_merged_index = outer_map->merged_indexes[outer_default];
+	}
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0 && inner_default < inner_map->nparts);
+		inner_merged_index = inner_map->merged_indexes[inner_default];
+	}
+
+	if (outer_has_default && !inner_has_default)
+	{
+		/*
+		 * If this is an outer join, assign a new merged partition if not
+		 * already done.  The resulting partition should be the default
+		 * partition of the joinrel.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			if (outer_merged_index == -1)
+				*default_index = merge_partition_with_dummy(outer_map,
+															outer_default,
+															next_index);
+			else
+				Assert(*default_index == outer_merged_index);
+		}
+		else
+			Assert(*default_index == -1);
+	}
+	else if (!outer_has_default && inner_has_default)
+	{
+		/*
+		 * If this is a FULL join, assign a new merged partition if not
+		 * already done.  The resulting partition should be the default
+		 * partition of the joinrel.
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (inner_merged_index == -1)
+				*default_index = merge_partition_with_dummy(inner_map,
+															inner_default,
+															next_index);
+			else
+				Assert(*default_index == inner_merged_index);
+		}
+		else
+			Assert(*default_index == -1);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+		Assert(outer_merged_index == -1);
+		Assert(inner_merged_index == -1);
+		Assert(*default_index == -1);
+
+		/*
+		 * Merge the default partitions.  Both partitions aren't merged yet,
+		 * so the partitions should be merged successfully.  The resulting
+		 * partition should be the default partition of the joinrel.
+		 */
+		*default_index = map_and_merge_partitions(outer_map,
+												  inner_map,
+												  outer_default,
+												  inner_default,
+												  next_index);
+		Assert(*default_index >= 0);
+	}
+}
+
+/*
+ * merge_null_partitions
+ *		Merge the NULL partitions from both sides, if any
+ *
+ * *null_index is set to the index of the NULL partition of the joinrel, if
+ * appropriate.
+ */
+static void
+merge_null_partitions(PartitionMap *outer_map,
+					  PartitionMap *inner_map,
+					  bool outer_has_null,
+					  bool inner_has_null,
+					  int outer_null,
+					  int inner_null,
+					  JoinType jointype,
+					  int *next_index,
+					  int *null_index)
+{
+	bool 		outer_null_unmerged = false;
+	bool 		inner_null_unmerged = false;
+
+	Assert(outer_has_null || inner_has_null);
+	Assert(*null_index == -1);
+
+	/*
+	 * If this is an INNER or SEMI join, two NULL values won't be considered
+	 * to be equal, in which case we don't need to do anything.
+	 */
+	if (jointype == JOIN_INNER || jointype == JOIN_SEMI)
+		return;
+
+	Assert(IS_OUTER_JOIN(jointype));
+	Assert(jointype != JOIN_RIGHT);
+
+	if (outer_has_null)
+	{
+		/*
+		 * Since this is an outer join, if the NULL partition of the outer
+		 * side isn't merged yet, it means that the partition contains only
+		 * NULL values as the key values, in which case we need to create a
+		 * merged partition below.
+		 */
+		Assert(outer_null >= 0 && outer_null < outer_map->nparts);
+		if (outer_map->merged_indexes[outer_null] == -1)
+			outer_null_unmerged = true;
+	}
+	if (inner_has_null)
+	{
+		/*
+		 * If this is a FULL join, and the NULL partition of the inner side
+		 * isn't merged yet, it means that the partition contains only NULL
+		 * values as the key values, in which case we need to create a merged
+		 * partition below.
+		 */
+		Assert(inner_null >= 0 && inner_null < inner_map->nparts);
+		if (inner_map->merged_indexes[inner_null] == -1)
+			inner_null_unmerged = true;
+	}
+
+	if (!outer_null_unmerged && !inner_null_unmerged)
+		return;
+
+	if (outer_null_unmerged && !inner_null_unmerged)
+	{
+		Assert(outer_has_null);
+
+		/*
+		 * Assign a new merged partition.  The resulting partition should be
+		 * the NULL partition of the joinrel.
+		 */
+		*null_index = merge_partition_with_dummy(outer_map, outer_null,
+												 next_index);
+	}
+	else if (!outer_null_unmerged && inner_null_unmerged)
+	{
+		Assert(inner_has_null);
+
+		/*
+		 * Assign a new merged partition.  The resulting partition should be
+		 * the NULL partition of the joinrel.
+		 */
+		if (jointype == JOIN_FULL)
+			*null_index = merge_partition_with_dummy(inner_map, inner_null,
+													 next_index);
+	}
+	else
+	{
+		Assert(outer_null_unmerged && inner_null_unmerged);
+		Assert(outer_has_null);
+		Assert(inner_has_null);
+
+		/*
+		 * Merge the NULL partitions.  Both partitions aren't merged yet, so
+		 * the partitions should be merged successfully.  The resulting
+		 * partition should be the NULL partition of the joinrel.
+		 */
+		*null_index = map_and_merge_partitions(outer_map, inner_map,
+											   outer_null, inner_null,
+											   next_index);
+		Assert(*null_index >= 0);
+	}
+}
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index 3d3be197e0..1545877d8c 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -579,8 +579,10 @@ typedef struct PartitionSchemeData *PartitionScheme;
  *		part_scheme - Partitioning scheme of the relation
  *		nparts - Number of partitions
  *		boundinfo - Partition bounds
+ *		merged - true if partition bounds are merged ones
  *		partition_qual - Partition constraint if not the root
  *		part_rels - RelOptInfos for each partition
+ *		all_partrels - Relids set of all partition relids
  *		partexprs, nullable_partexprs - Partition key expressions
  *		partitioned_child_rels - RT indexes of unpruned partitions of
  *								 this relation that are partitioned tables
@@ -720,9 +722,12 @@ typedef struct RelOptInfo
 	PartitionScheme part_scheme;	/* Partitioning scheme. */
 	int			nparts;			/* number of partitions */
 	struct PartitionBoundInfoData *boundinfo;	/* Partition bounds */
+	bool		merged;			/* true if partition bounds were created by
+								 * partition_bounds_merge() */
 	List	   *partition_qual; /* partition constraint */
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of partitions,
 									 * stored in the same order of bounds */
+	Relids		all_partrels;	/* Relids set of all partition relids */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 	List	   *partitioned_child_rels; /* List of RT indexes. */
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index be940b5192..5442e843af 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -16,6 +16,7 @@
 #include "nodes/pg_list.h"
 #include "partitioning/partdefs.h"
 #include "utils/relcache.h"
+struct RelOptInfo;				/* avoid including pathnodes.h here */
 
 
 /*
@@ -108,5 +109,11 @@ extern int	partition_range_datum_bsearch(FmgrInfo *partsupfunc,
 										  int nvalues, Datum *values, bool *is_equal);
 extern int	partition_hash_bsearch(PartitionBoundInfo boundinfo,
 								   int modulus, int remainder);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+					   int16 *parttyplen, bool *parttypbyval,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   struct RelOptInfo *outer_rel, struct RelOptInfo *inner_rel,
+					   JoinType jointype,
+					   List **outer_parts, List **inner_parts);
 
 #endif							/* PARTBOUNDS_H */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index b3fbe47bde..8e1aa49da0 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -2055,30 +2055,2671 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
                ->  Seq Scan on prt1_n_p2 t1_2
 (10 rows)
 
--- partitionwise join can not be applied if only one of joining tables has
--- default partition
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
-ANALYZE prt2;
+--
+-- Test advanced partition-matching algorithm for partitioned join
+--
+-- Tests for range-partitioned tables
+CREATE TABLE prt1_ad (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_ad_p1 PARTITION OF prt1_ad FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_ad_p2 PARTITION OF prt1_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_ad_p3 PARTITION OF prt1_ad FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_ad_a_idx on prt1_ad (a);
+INSERT INTO prt1_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_ad;
+CREATE TABLE prt2_ad (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_ad_p1 PARTITION OF prt2_ad FOR VALUES FROM (100) TO (150);
+CREATE TABLE prt2_ad_p2 PARTITION OF prt2_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt2_ad_p3 PARTITION OF prt2_ad FOR VALUES FROM (350) TO (500);
+CREATE INDEX prt2_ad_b_idx on prt2_ad (b);
+INSERT INTO prt2_ad_p1 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 149) i;
+INSERT INTO prt2_ad_p2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(200, 299) i;
+INSERT INTO prt2_ad_p3 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_ad;
+-- inner join
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                     QUERY PLAN                      
+-----------------------------------------------------
  Sort
    Sort Key: t1.a
-   ->  Hash Join
-         Hash Cond: (t2.b = t1.a)
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_ad_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_ad_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p2 t2_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_ad_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 100 | 0 | 0100
+ 125 | 0 | 0125
+ 200 | 0 | 0200
+ 225 | 0 | 0225
+ 250 | 0 | 0250
+ 275 | 0 | 0275
+ 350 | 0 | 0350
+ 375 | 0 | 0375
+(8 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 |     | 
+ 175 | 0175 |     | 
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 300 | 0300 |     | 
+ 325 | 0325 |     | 
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(12 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+                  QUERY PLAN                   
+-----------------------------------------------
+ Append
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.a = t2_1.b)
+         ->  Seq Scan on prt1_ad_p1 t1_1
+               Filter: (b = 0)
+         ->  Hash
+               ->  Seq Scan on prt2_ad_p1 t2_1
+   ->  Hash Anti Join
+         Hash Cond: (t1_2.a = t2_2.b)
+         ->  Seq Scan on prt1_ad_p2 t1_2
+               Filter: (b = 0)
+         ->  Hash
+               ->  Seq Scan on prt2_ad_p2 t2_2
+   ->  Hash Anti Join
+         Hash Cond: (t1_3.a = t2_3.b)
+         ->  Seq Scan on prt1_ad_p3 t1_3
+               Filter: (b = 0)
+         ->  Hash
+               ->  Seq Scan on prt2_ad_p3 t2_3
+(19 rows)
+
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+  a  | b |  c   
+-----+---+------
+ 150 | 0 | 0150
+ 175 | 0 | 0175
+ 300 | 0 | 0300
+ 325 | 0 | 0325
+(4 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Sort
+   Sort Key: prt1_ad.a, prt2_ad.b
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_ad_1.a = prt2_ad_1.b)
+               Filter: (((175) = prt1_ad_1.a) OR ((425) = prt2_ad_1.b))
+               ->  Seq Scan on prt1_ad_p1 prt1_ad_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p1 prt2_ad_1
+                           Filter: (a = 0)
+         ->  Hash Full Join
+               Hash Cond: (prt1_ad_2.a = prt2_ad_2.b)
+               Filter: (((175) = prt1_ad_2.a) OR ((425) = prt2_ad_2.b))
+               ->  Seq Scan on prt1_ad_p2 prt1_ad_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p2 prt2_ad_2
+                           Filter: (a = 0)
+         ->  Hash Full Join
+               Hash Cond: (prt2_ad_3.b = prt1_ad_3.a)
+               Filter: (((175) = prt1_ad_3.a) OR ((425) = prt2_ad_3.b))
+               ->  Seq Scan on prt2_ad_p3 prt2_ad_3
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p3 prt1_ad_3
+                           Filter: (b = 0)
+(27 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 175 | 0175 |     | 
+     |      | 425 | 0425
+(2 rows)
+
+-- Test cases where one side has an extra partition
+-- Add an extra partition to prt2_ad
+CREATE TABLE prt2_ad_extra PARTITION OF prt2_ad FOR VALUES FROM (500) TO (MAXVALUE);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(500, 599) i;
+ANALYZE prt2_ad;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_ad_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_ad_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p2 t2_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_ad_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_ad_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 100 | 0 | 0100
+ 125 | 0 | 0125
+ 200 | 0 | 0200
+ 225 | 0 | 0225
+ 250 | 0 | 0250
+ 275 | 0 | 0275
+ 350 | 0 | 0350
+ 375 | 0 | 0375
+(8 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 |     | 
+ 175 | 0175 |     | 
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 300 | 0300 |     | 
+ 325 | 0325 |     | 
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(12 rows)
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c FROM prt2_ad t1 LEFT JOIN prt1_ad t2 ON (t1.b = t2.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t2.a
+   ->  Hash Right Join
+         Hash Cond: (t2.a = t1.b)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2_1
-               ->  Seq Scan on prt2_p2 t2_2
-               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt1_ad_p1 t2_1
+               ->  Seq Scan on prt1_ad_p2 t2_2
+               ->  Seq Scan on prt1_ad_p3 t2_3
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt2_ad_p1 t1_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_ad_p2 t1_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_ad_p3 t1_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_ad_extra t1_4
+                           Filter: (a = 0)
+(18 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+                  QUERY PLAN                   
+-----------------------------------------------
+ Append
+   ->  Hash Anti Join
+         Hash Cond: (t1_1.a = t2_1.b)
+         ->  Seq Scan on prt1_ad_p1 t1_1
+               Filter: (b = 0)
+         ->  Hash
+               ->  Seq Scan on prt2_ad_p1 t2_1
+   ->  Hash Anti Join
+         Hash Cond: (t1_2.a = t2_2.b)
+         ->  Seq Scan on prt1_ad_p2 t1_2
+               Filter: (b = 0)
+         ->  Hash
+               ->  Seq Scan on prt2_ad_p2 t2_2
+   ->  Hash Anti Join
+         Hash Cond: (t1_3.a = t2_3.b)
+         ->  Seq Scan on prt1_ad_p3 t1_3
+               Filter: (b = 0)
+         ->  Hash
+               ->  Seq Scan on prt2_ad_p3 t2_3
+(19 rows)
+
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+  a  | b |  c   
+-----+---+------
+ 150 | 0 | 0150
+ 175 | 0 | 0175
+ 300 | 0 | 0300
+ 325 | 0 | 0325
+(4 rows)
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt1_ad t2 WHERE t1.b = t2.a) AND t1.a = 0;
+                  QUERY PLAN                   
+-----------------------------------------------
+ Hash Anti Join
+   Hash Cond: (t1.b = t2.a)
+   ->  Append
+         ->  Seq Scan on prt2_ad_p1 t1_1
+               Filter: (a = 0)
+         ->  Seq Scan on prt2_ad_p2 t1_2
+               Filter: (a = 0)
+         ->  Seq Scan on prt2_ad_p3 t1_3
+               Filter: (a = 0)
+         ->  Seq Scan on prt2_ad_extra t1_4
+               Filter: (a = 0)
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_ad_p1 t2_1
+               ->  Seq Scan on prt1_ad_p2 t2_2
+               ->  Seq Scan on prt1_ad_p3 t2_3
+(16 rows)
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: prt1_ad.a, prt2_ad.b
+   ->  Hash Full Join
+         Hash Cond: (prt2_ad.b = prt1_ad.a)
+         Filter: (((175) = prt1_ad.a) OR ((425) = prt2_ad.b))
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 prt2_ad_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_ad_p2 prt2_ad_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_ad_p3 prt2_ad_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_ad_extra prt2_ad_4
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_ad_p1 prt1_ad_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_ad_p2 prt1_ad_2
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_ad_p3 prt1_ad_3
                            Filter: (b = 0)
-(16 rows)
+(22 rows)
+
+-- 3-way join where not every pair of relations can do partitioned join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1_ad t1 RIGHT JOIN prt2_ad t2 ON (t1.a = t2.b) INNER JOIN prt1_ad t3 ON (t2.b = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.a, t3.c;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a, t3.c
+   ->  Append
+         ->  Nested Loop
+               Join Filter: (t1_1.a = t3_1.a)
+               ->  Hash Join
+                     Hash Cond: (t2_1.b = t1_1.a)
+                     ->  Seq Scan on prt2_ad_p1 t2_1
+                     ->  Hash
+                           ->  Seq Scan on prt1_ad_p1 t1_1
+                                 Filter: (b = 0)
+               ->  Index Scan using prt1_ad_p1_a_idx on prt1_ad_p1 t3_1
+                     Index Cond: (a = t2_1.b)
+         ->  Nested Loop
+               Join Filter: (t1_2.a = t3_2.a)
+               ->  Hash Join
+                     Hash Cond: (t2_2.b = t1_2.a)
+                     ->  Seq Scan on prt2_ad_p2 t2_2
+                     ->  Hash
+                           ->  Seq Scan on prt1_ad_p2 t1_2
+                                 Filter: (b = 0)
+               ->  Index Scan using prt1_ad_p2_a_idx on prt1_ad_p2 t3_2
+                     Index Cond: (a = t2_2.b)
+         ->  Nested Loop
+               Join Filter: (t1_3.a = t2_3.b)
+               ->  Hash Join
+                     Hash Cond: (t3_3.a = t1_3.a)
+                     ->  Seq Scan on prt1_ad_p3 t3_3
+                     ->  Hash
+                           ->  Seq Scan on prt1_ad_p3 t1_3
+                                 Filter: (b = 0)
+               ->  Index Scan using prt2_ad_p3_b_idx on prt2_ad_p3 t2_3
+                     Index Cond: (b = t3_3.a)
+(33 rows)
+
+SELECT t1.a, t2.a, t3.c FROM prt1_ad t1 RIGHT JOIN prt2_ad t2 ON (t1.a = t2.b) INNER JOIN prt1_ad t3 ON (t2.b = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.a, t3.c;
+  a  | a |  c   
+-----+---+------
+ 100 | 0 | 0100
+ 125 | 0 | 0125
+ 200 | 0 | 0200
+ 225 | 0 | 0225
+ 250 | 0 | 0250
+ 275 | 0 | 0275
+ 350 | 0 | 0350
+ 375 | 0 | 0375
+(8 rows)
+
+DROP TABLE prt2_ad_extra;
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in those cases
+-- Split prt2_ad_p3 into two partitions so that prt1_ad_p3 matches both
+-- partitions
+ALTER TABLE prt2_ad DETACH PARTITION prt2_ad_p3;
+CREATE TABLE prt2_ad_p3_350_375 PARTITION OF prt2_ad FOR VALUES FROM (350) TO (375);
+CREATE TABLE prt2_ad_p3_375_500 PARTITION OF prt2_ad FOR VALUES FROM (375) TO (500);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_ad;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Seq Scan on prt2_ad_p3_350_375 t2_3
+               ->  Seq Scan on prt2_ad_p3_375_500 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p3 t1_3
+                           Filter: (b = 0)
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                         QUERY PLAN                          
+-------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_ad_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_ad_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_ad_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_ad_p1 t2_1
+                     ->  Seq Scan on prt2_ad_p2 t2_2
+                     ->  Seq Scan on prt2_ad_p3_350_375 t2_3
+                     ->  Seq Scan on prt2_ad_p3_375_500 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Seq Scan on prt2_ad_p3_350_375 t2_3
+               ->  Seq Scan on prt2_ad_p3_375_500 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p3 t1_3
+                           Filter: (b = 0)
+(17 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Hash Anti Join
+   Hash Cond: (t1.a = t2.b)
+   ->  Append
+         ->  Seq Scan on prt1_ad_p1 t1_1
+               Filter: (b = 0)
+         ->  Seq Scan on prt1_ad_p2 t1_2
+               Filter: (b = 0)
+         ->  Seq Scan on prt1_ad_p3 t1_3
+               Filter: (b = 0)
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Seq Scan on prt2_ad_p3_350_375 t2_3
+               ->  Seq Scan on prt2_ad_p3_375_500 t2_4
+(15 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: prt1_ad.a, prt2_ad.b
+   ->  Hash Full Join
+         Hash Cond: (prt2_ad.b = prt1_ad.a)
+         Filter: (((175) = prt1_ad.a) OR ((425) = prt2_ad.b))
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 prt2_ad_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_ad_p2 prt2_ad_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_ad_p3_350_375 prt2_ad_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_ad_p3_375_500 prt2_ad_4
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_ad_p1 prt1_ad_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p2 prt1_ad_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p3 prt1_ad_3
+                           Filter: (b = 0)
+(22 rows)
+
+DROP TABLE prt2_ad_p3_350_375;
+DROP TABLE prt2_ad_p3_375_500;
+-- Test default partitions
+-- Change prt1_ad_p1 to the default partition
+ALTER TABLE prt1_ad DETACH PARTITION prt1_ad_p1;
+ALTER TABLE prt1_ad ATTACH PARTITION prt1_ad_p1 DEFAULT;
+ANALYZE prt1_ad;
+CREATE TABLE prt2_ad_p3_300_400 PARTITION OF prt2_ad FOR VALUES FROM (300) TO (400);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(300, 399) i;
+ANALYZE prt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_3.a)
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p1 t1_3
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p2 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_3.b = t1_2.a)
+               ->  Seq Scan on prt2_ad_p3_300_400 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p3 t1_2
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 300 | 0300 | 300 | 0300
+ 325 | 0325 | 325 | 0325
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(10 rows)
+
+DROP TABLE prt2_ad_p3_300_400;
+-- Restore prt2_ad_p3
+ALTER TABLE prt2_ad ATTACH PARTITION prt2_ad_p3 FOR VALUES FROM (350) TO (500);
+ANALYZE prt2_ad;
+-- Partitioned join can't be applied because the default partition of prt1_ad
+-- prt1_ad_p1 matches prt2_ad_p1 and prt2_ad_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Seq Scan on prt2_ad_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_ad_p2 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p3 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p1 t1_3
+                           Filter: (b = 0)
+(16 rows)
+
+-- Change prt2_ad_p3 to the default partition
+ALTER TABLE prt2_ad DETACH PARTITION prt2_ad_p3;
+ALTER TABLE prt2_ad ATTACH PARTITION prt2_ad_p3 DEFAULT;
+ANALYZE prt2_ad;
+-- Partitioned join can't be applied because the default partition of prt1_ad
+-- prt1_ad_p1 matches prt2_ad_p1 and prt2_ad_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Seq Scan on prt2_ad_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_ad_p2 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p3 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_ad_p1 t1_3
+                           Filter: (b = 0)
+(16 rows)
+
+DROP TABLE prt1_ad_p3;
+ANALYZE prt1_ad;
+DROP TABLE prt2_ad_p3;
+ANALYZE prt2_ad;
+CREATE TABLE prt3_ad (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt3_ad_p1 PARTITION OF prt3_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt3_ad_p2 PARTITION OF prt3_ad FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt3_ad_a_idx on prt3_ad (a);
+INSERT INTO prt3_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(200, 399) i;
+ANALYZE prt3_ad;
+-- 3-way join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) LEFT JOIN prt3_ad t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+                           QUERY PLAN                            
+-----------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b, t3.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t3_1.a = t1_1.a)
+               ->  Seq Scan on prt3_ad_p1 t3_1
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_2.b = t1_1.a)
+                           ->  Seq Scan on prt2_ad_p2 t2_2
+                           ->  Hash
+                                 ->  Seq Scan on prt1_ad_p2 t1_1
+                                       Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t3_2.a = t1_2.a)
+               ->  Seq Scan on prt3_ad_p2 t3_2
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_1.b = t1_2.a)
+                           ->  Seq Scan on prt2_ad_p1 t2_1
+                           ->  Hash
+                                 ->  Seq Scan on prt1_ad_p1 t1_2
+                                       Filter: (b = 0)
+(23 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) LEFT JOIN prt3_ad t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+  a  |  c   |  b  |  c   |  a  |  c   
+-----+------+-----+------+-----+------
+ 100 | 0100 | 100 | 0100 |     | 
+ 125 | 0125 | 125 | 0125 |     | 
+ 150 | 0150 |     |      |     | 
+ 175 | 0175 |     |      |     | 
+ 200 | 0200 | 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275 | 275 | 0275
+(8 rows)
+
+DROP TABLE prt1_ad;
+DROP TABLE prt2_ad;
+DROP TABLE prt3_ad;
+-- Test interaction of partitioned join with partition pruning
+CREATE TABLE prt1_ad (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_ad_p1 PARTITION OF prt1_ad FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_ad_p2 PARTITION OF prt1_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_ad_p3 PARTITION OF prt1_ad FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_ad_a_idx on prt1_ad (a);
+INSERT INTO prt1_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_ad;
+CREATE TABLE prt2_ad (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_ad_p1 PARTITION OF prt2_ad FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt2_ad_p2 PARTITION OF prt2_ad FOR VALUES FROM (200) TO (400);
+CREATE INDEX prt2_ad_b_idx on prt2_ad (b);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+                        QUERY PLAN                         
+-----------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: ((a < 300) AND (b = 0))
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: ((a < 300) AND (b = 0))
+(15 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 | 150 | 0150
+ 175 | 0175 | 175 | 0175
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+(8 rows)
+
+DROP TABLE prt1_ad_p3;
+CREATE TABLE prt1_ad_default PARTITION OF prt1_ad DEFAULT;
+ANALYZE prt1_ad;
+CREATE TABLE prt2_ad_default PARTITION OF prt2_ad DEFAULT;
+ANALYZE prt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+                                QUERY PLAN                                
+--------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p1 t1_1
+                           Filter: ((a >= 100) AND (a < 300) AND (b = 0))
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_ad_p2 t1_2
+                           Filter: ((a >= 100) AND (a < 300) AND (b = 0))
+(15 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 | 150 | 0150
+ 175 | 0175 | 175 | 0175
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+(8 rows)
+
+DROP TABLE prt1_ad;
+DROP TABLE prt2_ad;
+-- Tests for list-partitioned tables
+CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0001', '0003');
+CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt1_ad_p3 PARTITION OF plt1_ad FOR VALUES IN ('0008', '0009');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 NOT IN (0, 2, 5, 7);
+ANALYZE plt1_ad;
+CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt2_ad_p3 PARTITION OF plt2_ad FOR VALUES IN ('0007', '0009');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 NOT IN (0, 1, 5, 8);
+ANALYZE plt2_ad;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_ad_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+(6 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_ad_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 1 | 1 | 0001
+ 8 | 8 | 0008
+(2 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p3 t2_3
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+   |      | 2 | 0002
+   |      | 7 | 0007
+(8 rows)
+
+-- Test cases where one side has an extra partition
+-- Add an extra partition to plt2_ad
+CREATE TABLE plt2_ad_extra PARTITION OF plt2_ad FOR VALUES IN ('0000');
+INSERT INTO plt2_ad_extra VALUES (0, 0, '0000');
+ANALYZE plt2_ad;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_ad_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+(6 rows)
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2_ad t1 LEFT JOIN plt1_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt1_ad_p1 t2_1
+               ->  Seq Scan on plt1_ad_p2 t2_2
+               ->  Seq Scan on plt1_ad_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_ad_extra t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_ad_p1 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_ad_p2 t1_3
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_ad_p3 t1_4
+                           Filter: (b < 10)
+(18 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_ad_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 1 | 1 | 0001
+ 8 | 8 | 0008
+(2 rows)
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt2_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt1_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Anti Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt2_ad_extra t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p1 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3 t1_4
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p1 t2_1
+                     ->  Seq Scan on plt1_ad_p2 t2_2
+                     ->  Seq Scan on plt1_ad_p3 t2_3
+(18 rows)
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt2_ad_extra t2_1
+               ->  Seq Scan on plt2_ad_p1 t2_2
+               ->  Seq Scan on plt2_ad_p2 t2_3
+               ->  Seq Scan on plt2_ad_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+(15 rows)
+
+DROP TABLE plt2_ad_extra;
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in those cases
+-- Split plt2_ad_p2 into two partitions so that prt1_ad_p2 matches both
+-- partitions
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2;
+CREATE TABLE plt2_ad_p2_0004 PARTITION OF plt2_ad FOR VALUES IN ('0004');
+CREATE TABLE plt2_ad_p2_0006 PARTITION OF plt2_ad FOR VALUES IN ('0006');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6);
+ANALYZE plt2_ad;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Seq Scan on plt2_ad_p2_0004 t2_2
+               ->  Seq Scan on plt2_ad_p2_0006 t2_3
+               ->  Seq Scan on plt2_ad_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                        QUERY PLAN                        
+----------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Semi Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt1_ad_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+                     ->  Seq Scan on plt2_ad_p2_0004 t2_2
+                     ->  Seq Scan on plt2_ad_p2_0006 t2_3
+                     ->  Seq Scan on plt2_ad_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Seq Scan on plt2_ad_p2_0004 t2_2
+               ->  Seq Scan on plt2_ad_p2_0006 t2_3
+               ->  Seq Scan on plt2_ad_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(17 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                        QUERY PLAN                        
+----------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Anti Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt1_ad_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+                     ->  Seq Scan on plt2_ad_p2_0004 t2_2
+                     ->  Seq Scan on plt2_ad_p2_0006 t2_3
+                     ->  Seq Scan on plt2_ad_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Seq Scan on plt2_ad_p2_0004 t2_2
+               ->  Seq Scan on plt2_ad_p2_0006 t2_3
+               ->  Seq Scan on plt2_ad_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+(15 rows)
+
+DROP TABLE plt2_ad_p2_0004;
+DROP TABLE plt2_ad_p2_0006;
+-- Restore plt2_ad_p2
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_ad;
+-- Test NULL partitions
+-- Change plt1_ad_p1 to the NULL partition
+ALTER TABLE plt1_ad DETACH PARTITION plt1_ad_p1;
+CREATE TABLE plt1_ad_p1_null PARTITION OF plt1_ad FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt1_ad VALUES (-1, -1, NULL);
+ANALYZE plt1_ad;
+-- Change plt2_ad_p3 to the NULL partition
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p3;
+CREATE TABLE plt2_ad_p3_null PARTITION OF plt2_ad FOR VALUES IN (NULL, '0007', '0009');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (7, 9);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1_null t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_ad_p1_null t1_1
+                     Filter: (b < 10)
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3_null t2_3
+(19 rows)
+
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1_null t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a  |  c   | a |  c   
+----+------+---+------
+ -1 |      |   | 
+  1 | 0001 |   | 
+  3 | 0003 | 3 | 0003
+  4 | 0004 | 4 | 0004
+  6 | 0006 | 6 | 0006
+  8 | 0008 |   | 
+  9 | 0009 | 9 | 0009
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_ad_p1_null t1_1
+                     Filter: (b < 10)
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_p3_null t2_3
+(19 rows)
+
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a  | b  |  c   
+----+----+------
+ -1 | -1 | 
+  1 |  1 | 0001
+  8 |  8 | 0008
+(3 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p1_null t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt2_ad_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+ -1 |      |    | 
+  1 | 0001 |    | 
+  3 | 0003 |  3 | 0003
+  4 | 0004 |  4 | 0004
+  6 | 0006 |  6 | 0006
+  8 | 0008 |    | 
+  9 | 0009 |  9 | 0009
+    |      | -1 | 
+    |      |  2 | 0002
+    |      |  7 | 0007
+(10 rows)
+
+DROP TABLE plt1_ad_p1_null;
+DROP TABLE plt2_ad_p3_null;
+-- Restore plt1_ad_p1
+ALTER TABLE plt1_ad ATTACH PARTITION plt1_ad_p1 FOR VALUES IN ('0001', '0003');
+-- Add to plt1_ad the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt1_ad_extra PARTITION OF plt1_ad FOR VALUES IN (NULL);
+INSERT INTO plt1_ad VALUES (-1, -1, NULL);
+ANALYZE plt1_ad;
+-- Restore plt2_ad_p3
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p3 FOR VALUES IN ('0007', '0009');
+ANALYZE plt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- Partitioned join can't be applied because there isn't any partition on the
+-- plt2_ad side that matches the NULL partition of plt1_ad plt1_ad_extra
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Seq Scan on plt2_ad_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_extra t1_4
+                           Filter: (b < 10)
+(18 rows)
+
+-- Partitioned join can't be applied because there isn't any partition on the
+-- plt2_ad side that matches the NULL partition of plt1_ad plt1_ad_extra
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt1_ad_p1 t1_1
+               ->  Seq Scan on plt1_ad_p2 t1_2
+               ->  Seq Scan on plt1_ad_p3 t1_3
+               ->  Seq Scan on plt1_ad_extra t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+                     ->  Seq Scan on plt2_ad_p2 t2_2
+                     ->  Seq Scan on plt2_ad_p3 t2_3
+(15 rows)
+
+-- Add to plt2_ad the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt2_ad_extra PARTITION OF plt2_ad FOR VALUES IN (NULL);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_ad_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_3
+                           Filter: (b < 10)
+         ->  Nested Loop Left Join
+               Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+               ->  Seq Scan on plt1_ad_extra t1_4
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_ad_extra t2_4
+(26 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a  |  c   | a |  c   
+----+------+---+------
+ -1 |      |   | 
+  1 | 0001 |   | 
+  3 | 0003 | 3 | 0003
+  4 | 0004 | 4 | 0004
+  6 | 0006 | 6 | 0006
+  8 | 0008 |   | 
+  9 | 0009 | 9 | 0009
+(7 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_p3 t2_3
+         ->  Hash Full Join
+               Hash Cond: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+               Filter: ((COALESCE(t1_4.b, 0) < 10) AND (COALESCE(t2_4.b, 0) < 10))
+               ->  Seq Scan on plt1_ad_extra t1_4
+               ->  Hash
+                     ->  Seq Scan on plt2_ad_extra t2_4
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+ -1 |      |    | 
+  1 | 0001 |    | 
+  3 | 0003 |  3 | 0003
+  4 | 0004 |  4 | 0004
+  6 | 0006 |  6 | 0006
+  8 | 0008 |    | 
+  9 | 0009 |  9 | 0009
+    |      | -1 | 
+    |      |  2 | 0002
+    |      |  7 | 0007
+(10 rows)
+
+DROP TABLE plt1_ad_extra;
+DROP TABLE plt2_ad_extra;
+-- Test default partitions
+-- Change plt1_ad_p1 to the default partition
+ALTER TABLE plt1_ad DETACH PARTITION plt1_ad_p1;
+ALTER TABLE plt1_ad ATTACH PARTITION plt1_ad_p1 DEFAULT;
+DROP TABLE plt1_ad_p3;
+ANALYZE plt1_ad;
+DROP TABLE plt2_ad_p3;
+ANALYZE plt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_1
+                           Filter: (b < 10)
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(3 rows)
+
+-- Change plt2_ad_p2 to contain '0005' in addition to '0004' and '0006' as
+-- the key values
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2;
+CREATE TABLE plt2_ad_p2_ext PARTITION OF plt2_ad FOR VALUES IN ('0004', '0005', '0006');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 5, 6);
+ANALYZE plt2_ad;
+-- Partitioned join can't be applied because the default partition of plt1_ad
+-- plt1_ad_p1 matches plt2_ad_p1 and plt2_ad_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Seq Scan on plt2_ad_p2_ext t2_2
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p2 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p1 t1_2
+                           Filter: (b < 10)
+(13 rows)
+
+-- Change plt2_ad_p2_ext to the default partition
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2_ext;
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2_ext DEFAULT;
+ANALYZE plt2_ad;
+-- Partitioned join can't be applied because the default partition of plt1_ad
+-- plt1_ad_p1 matches plt2_ad_p1 and plt2_ad_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Seq Scan on plt2_ad_p2_ext t2_2
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_ad_p2 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_ad_p1 t1_2
+                           Filter: (b < 10)
+(13 rows)
+
+DROP TABLE plt2_ad_p2_ext;
+-- Restore plt2_ad_p2
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_ad;
+CREATE TABLE plt3_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_ad_p1 PARTITION OF plt3_ad FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt3_ad_p2 PARTITION OF plt3_ad FOR VALUES IN ('0007', '0009');
+INSERT INTO plt3_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6, 7, 9);
+ANALYZE plt3_ad;
+-- 3-way join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_ad t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t3_1.a = t1_1.a) AND (t3_1.c = t1_1.c))
+               ->  Seq Scan on plt3_ad_p1 t3_1
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+                           ->  Seq Scan on plt2_ad_p2 t2_2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_ad_p2 t1_1
+                                       Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t3_2.a = t1_2.a) AND (t3_2.c = t1_2.c))
+               ->  Seq Scan on plt3_ad_p2 t3_2
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+                           ->  Seq Scan on plt2_ad_p1 t2_1
+                           ->  Hash
+                                 ->  Seq Scan on plt1_ad_p1 t1_2
+                                       Filter: (b < 10)
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_ad t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   | a |  c   
+---+------+---+------+---+------
+ 1 | 0001 |   |      |   | 
+ 3 | 0003 | 3 | 0003 |   | 
+ 4 | 0004 | 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006 | 6 | 0006
+(4 rows)
+
+-- Test cases where one side has the default partition while the other side
+-- has the NULL partition
+-- Change plt2_ad_p1 to the NULL partition
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p1;
+CREATE TABLE plt2_ad_p1_null PARTITION OF plt2_ad FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p1_null t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p1 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p2 t1_1
+                           Filter: (b < 10)
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 | 1 | 0001
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(4 rows)
+
+-- Change the NULL partition to contain only NULL values as the key values
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p1_null;
+CREATE TABLE plt2_ad_p1_nullonly PARTITION OF plt2_ad FOR VALUES IN (NULL);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Seq Scan on plt2_ad_p2 t2
+         ->  Hash
+               ->  Seq Scan on plt1_ad_p2 t1
+                     Filter: (b < 10)
+(8 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(2 rows)
+
+DROP TABLE plt2_ad_p1_nullonly;
+DROP TABLE plt2_ad_p1_null;
+DROP TABLE plt2_ad_p1;
+DROP TABLE plt1_ad;
+DROP TABLE plt2_ad;
+DROP TABLE plt3_ad;
+CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1_ad;
+CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0002');
+CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2_ad;
+CREATE TABLE plt3_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_ad_p1 PARTITION OF plt3_ad FOR VALUES IN ('0001');
+CREATE TABLE plt3_ad_p2 PARTITION OF plt3_ad FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.c = t2.c)) FULL JOIN plt3_ad t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2.a, t3.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t3_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_1.c = t2_1.c)
+                     ->  Seq Scan on plt1_ad_p1 t1_1
+                     ->  Hash
+                           ->  Seq Scan on plt2_ad_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt3_ad_p1 t3_1
+         ->  Hash Full Join
+               Hash Cond: (t1_2.c = t3_2.c)
+               Filter: (((COALESCE(t1_2.a, 0) % 5) <> 3) AND ((COALESCE(t1_2.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_2.c = t2_2.c)
+                     ->  Seq Scan on plt1_ad_p2 t1_2
+                     ->  Hash
+                           ->  Seq Scan on plt2_ad_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt3_ad_p2 t3_2
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.c = t2.c)) FULL JOIN plt3_ad t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+ a  |  c   | a  |  c   | a  |  c   
+----+------+----+------+----+------
+  0 | 0000 |    |      |    | 
+  5 | 0000 |    |      |    | 
+ 10 | 0000 |    |      |    | 
+ 15 | 0000 |    |      |    | 
+ 20 | 0000 |    |      |    | 
+  1 | 0001 |    |      |  1 | 0001
+  1 | 0001 |    |      |  6 | 0001
+  1 | 0001 |    |      | 11 | 0001
+  1 | 0001 |    |      | 16 | 0001
+  1 | 0001 |    |      | 21 | 0001
+  6 | 0001 |    |      |  1 | 0001
+  6 | 0001 |    |      |  6 | 0001
+  6 | 0001 |    |      | 11 | 0001
+  6 | 0001 |    |      | 16 | 0001
+  6 | 0001 |    |      | 21 | 0001
+ 11 | 0001 |    |      |  1 | 0001
+ 11 | 0001 |    |      |  6 | 0001
+ 11 | 0001 |    |      | 11 | 0001
+ 11 | 0001 |    |      | 16 | 0001
+ 11 | 0001 |    |      | 21 | 0001
+ 16 | 0001 |    |      |  1 | 0001
+ 16 | 0001 |    |      |  6 | 0001
+ 16 | 0001 |    |      | 11 | 0001
+ 16 | 0001 |    |      | 16 | 0001
+ 16 | 0001 |    |      | 21 | 0001
+ 21 | 0001 |    |      |  1 | 0001
+ 21 | 0001 |    |      |  6 | 0001
+ 21 | 0001 |    |      | 11 | 0001
+ 21 | 0001 |    |      | 16 | 0001
+ 21 | 0001 |    |      | 21 | 0001
+  2 | 0002 |  2 | 0002 |    | 
+  2 | 0002 |  7 | 0002 |    | 
+  2 | 0002 | 12 | 0002 |    | 
+  2 | 0002 | 17 | 0002 |    | 
+  2 | 0002 | 22 | 0002 |    | 
+  7 | 0002 |  2 | 0002 |    | 
+  7 | 0002 |  7 | 0002 |    | 
+  7 | 0002 | 12 | 0002 |    | 
+  7 | 0002 | 17 | 0002 |    | 
+  7 | 0002 | 22 | 0002 |    | 
+ 12 | 0002 |  2 | 0002 |    | 
+ 12 | 0002 |  7 | 0002 |    | 
+ 12 | 0002 | 12 | 0002 |    | 
+ 12 | 0002 | 17 | 0002 |    | 
+ 12 | 0002 | 22 | 0002 |    | 
+ 17 | 0002 |  2 | 0002 |    | 
+ 17 | 0002 |  7 | 0002 |    | 
+ 17 | 0002 | 12 | 0002 |    | 
+ 17 | 0002 | 17 | 0002 |    | 
+ 17 | 0002 | 22 | 0002 |    | 
+ 22 | 0002 |  2 | 0002 |    | 
+ 22 | 0002 |  7 | 0002 |    | 
+ 22 | 0002 | 12 | 0002 |    | 
+ 22 | 0002 | 17 | 0002 |    | 
+ 22 | 0002 | 22 | 0002 |    | 
+(55 rows)
+
+DROP TABLE plt1_ad;
+DROP TABLE plt2_ad;
+DROP TABLE plt3_ad;
+-- Test interaction of partitioned join with partition pruning
+CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0001');
+CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0002');
+CREATE TABLE plt1_ad_p3 PARTITION OF plt1_ad FOR VALUES IN ('0003');
+CREATE TABLE plt1_ad_p4 PARTITION OF plt1_ad FOR VALUES IN (NULL, '0004', '0005');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt1_ad VALUES (-1, -1, NULL);
+ANALYZE plt1_ad;
+CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0003');
+CREATE TABLE plt2_ad_p3 PARTITION OF plt2_ad FOR VALUES IN (NULL);
+CREATE TABLE plt2_ad_p4 PARTITION OF plt2_ad FOR VALUES IN ('0004', '0005');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 AND t2.c IS NOT NULL ORDER BY t1.a;
+                                       QUERY PLAN                                        
+-----------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p2 t2_1
+                     Filter: (c IS NOT NULL)
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_1
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p4 t2_2
+                     Filter: (c IS NOT NULL)
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p4 t1_2
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 AND t2.c IS NOT NULL ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 5 | 0005 | 5 | 0005
+(3 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Seq Scan on plt2_ad_p4 t2
+         ->  Hash
+               ->  Seq Scan on plt1_ad_p4 t1
+                     Filter: ((c IS NULL) AND (b < 10))
+(8 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+ a  | c | a | c 
+----+---+---+---
+ -1 |   |   | 
+(1 row)
+
+CREATE TABLE plt1_ad_default PARTITION OF plt1_ad DEFAULT;
+ANALYZE plt1_ad;
+CREATE TABLE plt2_ad_default PARTITION OF plt2_ad DEFAULT;
+ANALYZE plt2_ad;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 AND t2.c IS NOT NULL ORDER BY t1.a;
+                                       QUERY PLAN                                        
+-----------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_ad_p2 t2_1
+                     Filter: (c IS NOT NULL)
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p3 t1_1
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_ad_p4 t2_2
+                     Filter: (c IS NOT NULL)
+               ->  Hash
+                     ->  Seq Scan on plt1_ad_p4 t1_2
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+(17 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 AND t2.c IS NOT NULL ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 5 | 0005 | 5 | 0005
+(3 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Seq Scan on plt2_ad_p4 t2
+         ->  Hash
+               ->  Seq Scan on plt1_ad_p4 t1
+                     Filter: ((c IS NULL) AND (b < 10))
+(8 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+ a  | c | a | c 
+----+---+---+---
+ -1 |   |   | 
+(1 row)
+
+DROP TABLE plt1_ad;
+DROP TABLE plt2_ad;
+-- Tests for multi-level partitioned tables
+CREATE TABLE alpha (a double precision, b int, c text) PARTITION BY RANGE (a);
+CREATE TABLE alpha_neg PARTITION OF alpha FOR VALUES FROM ('-Infinity') TO (0) PARTITION BY RANGE (b);
+CREATE TABLE alpha_pos PARTITION OF alpha FOR VALUES FROM (0) TO (10.0) PARTITION BY LIST (c);
+CREATE TABLE alpha_neg_p1 PARTITION OF alpha_neg FOR VALUES FROM (100) TO (200);
+CREATE TABLE alpha_neg_p2 PARTITION OF alpha_neg FOR VALUES FROM (200) TO (300);
+CREATE TABLE alpha_neg_p3 PARTITION OF alpha_neg FOR VALUES FROM (300) TO (400);
+CREATE TABLE alpha_pos_p1 PARTITION OF alpha_pos FOR VALUES IN ('0001', '0003');
+CREATE TABLE alpha_pos_p2 PARTITION OF alpha_pos FOR VALUES IN ('0004', '0006');
+CREATE TABLE alpha_pos_p3 PARTITION OF alpha_pos FOR VALUES IN ('0008', '0009');
+INSERT INTO alpha_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+INSERT INTO alpha_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+ANALYZE alpha;
+CREATE TABLE beta (a double precision, b int, c text) PARTITION BY RANGE (a);
+CREATE TABLE beta_neg PARTITION OF beta FOR VALUES FROM (-10.0) TO (0) PARTITION BY RANGE (b);
+CREATE TABLE beta_pos PARTITION OF beta FOR VALUES FROM (0) TO ('Infinity') PARTITION BY LIST (c);
+CREATE TABLE beta_neg_p1 PARTITION OF beta_neg FOR VALUES FROM (100) TO (150);
+CREATE TABLE beta_neg_p2 PARTITION OF beta_neg FOR VALUES FROM (200) TO (300);
+CREATE TABLE beta_neg_p3 PARTITION OF beta_neg FOR VALUES FROM (350) TO (500);
+CREATE TABLE beta_pos_p1 PARTITION OF beta_pos FOR VALUES IN ('0002', '0003');
+CREATE TABLE beta_pos_p2 PARTITION OF beta_pos FOR VALUES IN ('0004', '0006');
+CREATE TABLE beta_pos_p3 PARTITION OF beta_pos FOR VALUES IN ('0007', '0009');
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+ANALYZE beta;
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.b
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.b = t2_1.b))
+               ->  Seq Scan on alpha_neg_p1 t1_1
+                     Filter: ((b >= 125) AND (b < 225))
+               ->  Hash
+                     ->  Seq Scan on beta_neg_p1 t2_1
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.b = t1_2.b))
+               ->  Seq Scan on beta_neg_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on alpha_neg_p2 t1_2
+                           Filter: ((b >= 125) AND (b < 225))
+         ->  Hash Join
+               Hash Cond: ((t2_4.a = t1_4.a) AND (t2_4.b = t1_4.b))
+               ->  Append
+                     ->  Seq Scan on beta_pos_p1 t2_4
+                     ->  Seq Scan on beta_pos_p2 t2_5
+                     ->  Seq Scan on beta_pos_p3 t2_6
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on alpha_pos_p1 t1_4
+                                 Filter: ((b >= 125) AND (b < 225))
+                           ->  Seq Scan on alpha_pos_p2 t1_5
+                                 Filter: ((b >= 125) AND (b < 225))
+                           ->  Seq Scan on alpha_pos_p3 t1_6
+                                 Filter: ((b >= 125) AND (b < 225))
+(29 rows)
+
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b;
+ a  |  b  |  c   | a  |  b  |  c   
+----+-----+------+----+-----+------
+ -1 | 126 | 0006 | -1 | 126 | 0006
+ -1 | 129 | 0009 | -1 | 129 | 0009
+ -1 | 133 | 0003 | -1 | 133 | 0003
+ -1 | 134 | 0004 | -1 | 134 | 0004
+ -1 | 136 | 0006 | -1 | 136 | 0006
+ -1 | 139 | 0009 | -1 | 139 | 0009
+ -1 | 143 | 0003 | -1 | 143 | 0003
+ -1 | 144 | 0004 | -1 | 144 | 0004
+ -1 | 146 | 0006 | -1 | 146 | 0006
+ -1 | 149 | 0009 | -1 | 149 | 0009
+ -1 | 203 | 0003 | -1 | 203 | 0003
+ -1 | 204 | 0004 | -1 | 204 | 0004
+ -1 | 206 | 0006 | -1 | 206 | 0006
+ -1 | 209 | 0009 | -1 | 209 | 0009
+ -1 | 213 | 0003 | -1 | 213 | 0003
+ -1 | 214 | 0004 | -1 | 214 | 0004
+ -1 | 216 | 0006 | -1 | 216 | 0006
+ -1 | 219 | 0009 | -1 | 219 | 0009
+ -1 | 223 | 0003 | -1 | 223 | 0003
+ -1 | 224 | 0004 | -1 | 224 | 0004
+  1 | 126 | 0006 |  1 | 126 | 0006
+  1 | 129 | 0009 |  1 | 129 | 0009
+  1 | 133 | 0003 |  1 | 133 | 0003
+  1 | 134 | 0004 |  1 | 134 | 0004
+  1 | 136 | 0006 |  1 | 136 | 0006
+  1 | 139 | 0009 |  1 | 139 | 0009
+  1 | 143 | 0003 |  1 | 143 | 0003
+  1 | 144 | 0004 |  1 | 144 | 0004
+  1 | 146 | 0006 |  1 | 146 | 0006
+  1 | 149 | 0009 |  1 | 149 | 0009
+  1 | 203 | 0003 |  1 | 203 | 0003
+  1 | 204 | 0004 |  1 | 204 | 0004
+  1 | 206 | 0006 |  1 | 206 | 0006
+  1 | 209 | 0009 |  1 | 209 | 0009
+  1 | 213 | 0003 |  1 | 213 | 0003
+  1 | 214 | 0004 |  1 | 214 | 0004
+  1 | 216 | 0006 |  1 | 216 | 0006
+  1 | 219 | 0009 |  1 | 219 | 0009
+  1 | 223 | 0003 |  1 | 223 | 0003
+  1 | 224 | 0004 |  1 | 224 | 0004
+(40 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b >= 125 AND t1.b < 225 AND t2.b >= 125 AND t2.b < 225 AND t1.c IN ('0003') ORDER BY t1.a, t1.b, t2.a, t2.b;
+                                        QUERY PLAN                                         
+-------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.b, t2.b
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t1_2.a = t2_2.a)
+               ->  Append
+                     ->  Seq Scan on alpha_neg_p1 t1_2
+                           Filter: ((b >= 125) AND (b < 225) AND (c = '0003'::text))
+                     ->  Seq Scan on alpha_neg_p2 t1_3
+                           Filter: ((b >= 125) AND (b < 225) AND (c = '0003'::text))
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on beta_neg_p1 t2_2
+                                 Filter: ((b >= 125) AND (b < 225) AND (c = '0003'::text))
+                           ->  Seq Scan on beta_neg_p2 t2_3
+                                 Filter: ((b >= 125) AND (b < 225) AND (c = '0003'::text))
+         ->  Hash Join
+               Hash Cond: (t1_4.a = t2_4.a)
+               ->  Seq Scan on alpha_pos_p1 t1_4
+                     Filter: ((b >= 125) AND (b < 225) AND (c = '0003'::text))
+               ->  Hash
+                     ->  Seq Scan on beta_pos_p1 t2_4
+                           Filter: ((b >= 125) AND (b < 225) AND (c = '0003'::text))
+(23 rows)
+
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b >= 125 AND t1.b < 225 AND t2.b >= 125 AND t2.b < 225 AND t1.c IN ('0003') ORDER BY t1.a, t1.b, t2.a, t2.b;
+ a  |  b  |  c   | a  |  b  |  c   
+----+-----+------+----+-----+------
+ -1 | 133 | 0003 | -1 | 133 | 0003
+ -1 | 133 | 0003 | -1 | 143 | 0003
+ -1 | 133 | 0003 | -1 | 203 | 0003
+ -1 | 133 | 0003 | -1 | 213 | 0003
+ -1 | 133 | 0003 | -1 | 223 | 0003
+ -1 | 143 | 0003 | -1 | 133 | 0003
+ -1 | 143 | 0003 | -1 | 143 | 0003
+ -1 | 143 | 0003 | -1 | 203 | 0003
+ -1 | 143 | 0003 | -1 | 213 | 0003
+ -1 | 143 | 0003 | -1 | 223 | 0003
+ -1 | 153 | 0003 | -1 | 133 | 0003
+ -1 | 153 | 0003 | -1 | 143 | 0003
+ -1 | 153 | 0003 | -1 | 203 | 0003
+ -1 | 153 | 0003 | -1 | 213 | 0003
+ -1 | 153 | 0003 | -1 | 223 | 0003
+ -1 | 163 | 0003 | -1 | 133 | 0003
+ -1 | 163 | 0003 | -1 | 143 | 0003
+ -1 | 163 | 0003 | -1 | 203 | 0003
+ -1 | 163 | 0003 | -1 | 213 | 0003
+ -1 | 163 | 0003 | -1 | 223 | 0003
+ -1 | 173 | 0003 | -1 | 133 | 0003
+ -1 | 173 | 0003 | -1 | 143 | 0003
+ -1 | 173 | 0003 | -1 | 203 | 0003
+ -1 | 173 | 0003 | -1 | 213 | 0003
+ -1 | 173 | 0003 | -1 | 223 | 0003
+ -1 | 183 | 0003 | -1 | 133 | 0003
+ -1 | 183 | 0003 | -1 | 143 | 0003
+ -1 | 183 | 0003 | -1 | 203 | 0003
+ -1 | 183 | 0003 | -1 | 213 | 0003
+ -1 | 183 | 0003 | -1 | 223 | 0003
+ -1 | 193 | 0003 | -1 | 133 | 0003
+ -1 | 193 | 0003 | -1 | 143 | 0003
+ -1 | 193 | 0003 | -1 | 203 | 0003
+ -1 | 193 | 0003 | -1 | 213 | 0003
+ -1 | 193 | 0003 | -1 | 223 | 0003
+ -1 | 203 | 0003 | -1 | 133 | 0003
+ -1 | 203 | 0003 | -1 | 143 | 0003
+ -1 | 203 | 0003 | -1 | 203 | 0003
+ -1 | 203 | 0003 | -1 | 213 | 0003
+ -1 | 203 | 0003 | -1 | 223 | 0003
+ -1 | 213 | 0003 | -1 | 133 | 0003
+ -1 | 213 | 0003 | -1 | 143 | 0003
+ -1 | 213 | 0003 | -1 | 203 | 0003
+ -1 | 213 | 0003 | -1 | 213 | 0003
+ -1 | 213 | 0003 | -1 | 223 | 0003
+ -1 | 223 | 0003 | -1 | 133 | 0003
+ -1 | 223 | 0003 | -1 | 143 | 0003
+ -1 | 223 | 0003 | -1 | 203 | 0003
+ -1 | 223 | 0003 | -1 | 213 | 0003
+ -1 | 223 | 0003 | -1 | 223 | 0003
+  1 | 133 | 0003 |  1 | 133 | 0003
+  1 | 133 | 0003 |  1 | 143 | 0003
+  1 | 133 | 0003 |  1 | 203 | 0003
+  1 | 133 | 0003 |  1 | 213 | 0003
+  1 | 133 | 0003 |  1 | 223 | 0003
+  1 | 143 | 0003 |  1 | 133 | 0003
+  1 | 143 | 0003 |  1 | 143 | 0003
+  1 | 143 | 0003 |  1 | 203 | 0003
+  1 | 143 | 0003 |  1 | 213 | 0003
+  1 | 143 | 0003 |  1 | 223 | 0003
+  1 | 153 | 0003 |  1 | 133 | 0003
+  1 | 153 | 0003 |  1 | 143 | 0003
+  1 | 153 | 0003 |  1 | 203 | 0003
+  1 | 153 | 0003 |  1 | 213 | 0003
+  1 | 153 | 0003 |  1 | 223 | 0003
+  1 | 163 | 0003 |  1 | 133 | 0003
+  1 | 163 | 0003 |  1 | 143 | 0003
+  1 | 163 | 0003 |  1 | 203 | 0003
+  1 | 163 | 0003 |  1 | 213 | 0003
+  1 | 163 | 0003 |  1 | 223 | 0003
+  1 | 173 | 0003 |  1 | 133 | 0003
+  1 | 173 | 0003 |  1 | 143 | 0003
+  1 | 173 | 0003 |  1 | 203 | 0003
+  1 | 173 | 0003 |  1 | 213 | 0003
+  1 | 173 | 0003 |  1 | 223 | 0003
+  1 | 183 | 0003 |  1 | 133 | 0003
+  1 | 183 | 0003 |  1 | 143 | 0003
+  1 | 183 | 0003 |  1 | 203 | 0003
+  1 | 183 | 0003 |  1 | 213 | 0003
+  1 | 183 | 0003 |  1 | 223 | 0003
+  1 | 193 | 0003 |  1 | 133 | 0003
+  1 | 193 | 0003 |  1 | 143 | 0003
+  1 | 193 | 0003 |  1 | 203 | 0003
+  1 | 193 | 0003 |  1 | 213 | 0003
+  1 | 193 | 0003 |  1 | 223 | 0003
+  1 | 203 | 0003 |  1 | 133 | 0003
+  1 | 203 | 0003 |  1 | 143 | 0003
+  1 | 203 | 0003 |  1 | 203 | 0003
+  1 | 203 | 0003 |  1 | 213 | 0003
+  1 | 203 | 0003 |  1 | 223 | 0003
+  1 | 213 | 0003 |  1 | 133 | 0003
+  1 | 213 | 0003 |  1 | 143 | 0003
+  1 | 213 | 0003 |  1 | 203 | 0003
+  1 | 213 | 0003 |  1 | 213 | 0003
+  1 | 213 | 0003 |  1 | 223 | 0003
+  1 | 223 | 0003 |  1 | 133 | 0003
+  1 | 223 | 0003 |  1 | 143 | 0003
+  1 | 223 | 0003 |  1 | 203 | 0003
+  1 | 223 | 0003 |  1 | 213 | 0003
+  1 | 223 | 0003 |  1 | 223 | 0003
+(100 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE t1.b > 300 AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;
+                                        QUERY PLAN                                        
+------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.b
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.b = t1_1.b) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on beta_neg_p3 t2_1
+               ->  Hash
+                     ->  Seq Scan on alpha_neg_p3 t1_1
+                           Filter: ((b > 300) AND (c = ANY ('{0004,0009}'::text[])))
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.b = t1_2.b) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on beta_pos_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on alpha_pos_p2 t1_2
+                           Filter: ((b > 300) AND (c = ANY ('{0004,0009}'::text[])))
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.b = t1_3.b) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on beta_pos_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on alpha_pos_p3 t1_3
+                           Filter: ((b > 300) AND (c = ANY ('{0004,0009}'::text[])))
+(21 rows)
+
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE t1.b < 300 AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;
+ a  |  b  |  c   | a  |  b  |  c   
+----+-----+------+----+-----+------
+ -1 | 104 | 0004 | -1 | 104 | 0004
+ -1 | 109 | 0009 | -1 | 109 | 0009
+ -1 | 114 | 0004 | -1 | 114 | 0004
+ -1 | 119 | 0009 | -1 | 119 | 0009
+ -1 | 124 | 0004 | -1 | 124 | 0004
+ -1 | 129 | 0009 | -1 | 129 | 0009
+ -1 | 134 | 0004 | -1 | 134 | 0004
+ -1 | 139 | 0009 | -1 | 139 | 0009
+ -1 | 144 | 0004 | -1 | 144 | 0004
+ -1 | 149 | 0009 | -1 | 149 | 0009
+ -1 | 204 | 0004 | -1 | 204 | 0004
+ -1 | 209 | 0009 | -1 | 209 | 0009
+ -1 | 214 | 0004 | -1 | 214 | 0004
+ -1 | 219 | 0009 | -1 | 219 | 0009
+ -1 | 224 | 0004 | -1 | 224 | 0004
+ -1 | 229 | 0009 | -1 | 229 | 0009
+ -1 | 234 | 0004 | -1 | 234 | 0004
+ -1 | 239 | 0009 | -1 | 239 | 0009
+ -1 | 244 | 0004 | -1 | 244 | 0004
+ -1 | 249 | 0009 | -1 | 249 | 0009
+ -1 | 254 | 0004 | -1 | 254 | 0004
+ -1 | 259 | 0009 | -1 | 259 | 0009
+ -1 | 264 | 0004 | -1 | 264 | 0004
+ -1 | 269 | 0009 | -1 | 269 | 0009
+ -1 | 274 | 0004 | -1 | 274 | 0004
+ -1 | 279 | 0009 | -1 | 279 | 0009
+ -1 | 284 | 0004 | -1 | 284 | 0004
+ -1 | 289 | 0009 | -1 | 289 | 0009
+ -1 | 294 | 0004 | -1 | 294 | 0004
+ -1 | 299 | 0009 | -1 | 299 | 0009
+  1 | 104 | 0004 |  1 | 104 | 0004
+  1 | 109 | 0009 |  1 | 109 | 0009
+  1 | 114 | 0004 |  1 | 114 | 0004
+  1 | 119 | 0009 |  1 | 119 | 0009
+  1 | 124 | 0004 |  1 | 124 | 0004
+  1 | 129 | 0009 |  1 | 129 | 0009
+  1 | 134 | 0004 |  1 | 134 | 0004
+  1 | 139 | 0009 |  1 | 139 | 0009
+  1 | 144 | 0004 |  1 | 144 | 0004
+  1 | 149 | 0009 |  1 | 149 | 0009
+  1 | 204 | 0004 |  1 | 204 | 0004
+  1 | 209 | 0009 |  1 | 209 | 0009
+  1 | 214 | 0004 |  1 | 214 | 0004
+  1 | 219 | 0009 |  1 | 219 | 0009
+  1 | 224 | 0004 |  1 | 224 | 0004
+  1 | 229 | 0009 |  1 | 229 | 0009
+  1 | 234 | 0004 |  1 | 234 | 0004
+  1 | 239 | 0009 |  1 | 239 | 0009
+  1 | 244 | 0004 |  1 | 244 | 0004
+  1 | 249 | 0009 |  1 | 249 | 0009
+  1 | 254 | 0004 |  1 | 254 | 0004
+  1 | 259 | 0009 |  1 | 259 | 0009
+  1 | 264 | 0004 |  1 | 264 | 0004
+  1 | 269 | 0009 |  1 | 269 | 0009
+  1 | 274 | 0004 |  1 | 274 | 0004
+  1 | 279 | 0009 |  1 | 279 | 0009
+  1 | 284 | 0004 |  1 | 284 | 0004
+  1 | 289 | 0009 |  1 | 289 | 0009
+  1 | 294 | 0004 |  1 | 294 | 0004
+  1 | 299 | 0009 |  1 | 299 | 0009
+(60 rows)
 
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 575ba7b8d4..31dcd6e77c 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -445,11 +445,643 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
 
--- partitionwise join can not be applied if only one of joining tables has
--- default partition
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
-ANALYZE prt2;
 
+--
+-- Test advanced partition-matching algorithm for partitioned join
+--
+
+-- Tests for range-partitioned tables
+CREATE TABLE prt1_ad (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_ad_p1 PARTITION OF prt1_ad FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_ad_p2 PARTITION OF prt1_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_ad_p3 PARTITION OF prt1_ad FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_ad_a_idx on prt1_ad (a);
+INSERT INTO prt1_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_ad;
+
+CREATE TABLE prt2_ad (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_ad_p1 PARTITION OF prt2_ad FOR VALUES FROM (100) TO (150);
+CREATE TABLE prt2_ad_p2 PARTITION OF prt2_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt2_ad_p3 PARTITION OF prt2_ad FOR VALUES FROM (350) TO (500);
+CREATE INDEX prt2_ad_b_idx on prt2_ad (b);
+INSERT INTO prt2_ad_p1 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 149) i;
+INSERT INTO prt2_ad_p2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(200, 299) i;
+INSERT INTO prt2_ad_p3 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_ad;
+
+-- inner join
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+-- Test cases where one side has an extra partition
+
+-- Add an extra partition to prt2_ad
+CREATE TABLE prt2_ad_extra PARTITION OF prt2_ad FOR VALUES FROM (500) TO (MAXVALUE);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(500, 599) i;
+ANALYZE prt2_ad;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c FROM prt2_ad t1 LEFT JOIN prt1_ad t2 ON (t1.b = t2.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt1_ad t2 WHERE t1.b = t2.a) AND t1.a = 0;
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+-- 3-way join where not every pair of relations can do partitioned join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1_ad t1 RIGHT JOIN prt2_ad t2 ON (t1.a = t2.b) INNER JOIN prt1_ad t3 ON (t2.b = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.a, t3.c;
+SELECT t1.a, t2.a, t3.c FROM prt1_ad t1 RIGHT JOIN prt2_ad t2 ON (t1.a = t2.b) INNER JOIN prt1_ad t3 ON (t2.b = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.a, t3.c;
+
+DROP TABLE prt2_ad_extra;
+
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in those cases
+
+-- Split prt2_ad_p3 into two partitions so that prt1_ad_p3 matches both
+-- partitions
+ALTER TABLE prt2_ad DETACH PARTITION prt2_ad_p3;
+CREATE TABLE prt2_ad_p3_350_375 PARTITION OF prt2_ad FOR VALUES FROM (350) TO (375);
+CREATE TABLE prt2_ad_p3_375_500 PARTITION OF prt2_ad FOR VALUES FROM (375) TO (500);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_ad;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+DROP TABLE prt2_ad_p3_350_375;
+DROP TABLE prt2_ad_p3_375_500;
+
+-- Test default partitions
+
+-- Change prt1_ad_p1 to the default partition
+ALTER TABLE prt1_ad DETACH PARTITION prt1_ad_p1;
+ALTER TABLE prt1_ad ATTACH PARTITION prt1_ad_p1 DEFAULT;
+ANALYZE prt1_ad;
+
+CREATE TABLE prt2_ad_p3_300_400 PARTITION OF prt2_ad FOR VALUES FROM (300) TO (400);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(300, 399) i;
+ANALYZE prt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt2_ad_p3_300_400;
+
+-- Restore prt2_ad_p3
+ALTER TABLE prt2_ad ATTACH PARTITION prt2_ad_p3 FOR VALUES FROM (350) TO (500);
+ANALYZE prt2_ad;
+
+-- Partitioned join can't be applied because the default partition of prt1_ad
+-- prt1_ad_p1 matches prt2_ad_p1 and prt2_ad_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- Change prt2_ad_p3 to the default partition
+ALTER TABLE prt2_ad DETACH PARTITION prt2_ad_p3;
+ALTER TABLE prt2_ad ATTACH PARTITION prt2_ad_p3 DEFAULT;
+ANALYZE prt2_ad;
+
+-- Partitioned join can't be applied because the default partition of prt1_ad
+-- prt1_ad_p1 matches prt2_ad_p1 and prt2_ad_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt1_ad_p3;
+ANALYZE prt1_ad;
+
+DROP TABLE prt2_ad_p3;
+ANALYZE prt2_ad;
+
+CREATE TABLE prt3_ad (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt3_ad_p1 PARTITION OF prt3_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt3_ad_p2 PARTITION OF prt3_ad FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt3_ad_a_idx on prt3_ad (a);
+INSERT INTO prt3_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(200, 399) i;
+ANALYZE prt3_ad;
+
+-- 3-way join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) LEFT JOIN prt3_ad t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) LEFT JOIN prt3_ad t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+
+DROP TABLE prt1_ad;
+DROP TABLE prt2_ad;
+DROP TABLE prt3_ad;
+
+-- Test interaction of partitioned join with partition pruning
+
+CREATE TABLE prt1_ad (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_ad_p1 PARTITION OF prt1_ad FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_ad_p2 PARTITION OF prt1_ad FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_ad_p3 PARTITION OF prt1_ad FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_ad_a_idx on prt1_ad (a);
+INSERT INTO prt1_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_ad;
+
+CREATE TABLE prt2_ad (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_ad_p1 PARTITION OF prt2_ad FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt2_ad_p2 PARTITION OF prt2_ad FOR VALUES FROM (200) TO (400);
+CREATE INDEX prt2_ad_b_idx on prt2_ad (b);
+INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt1_ad_p3;
+CREATE TABLE prt1_ad_default PARTITION OF prt1_ad DEFAULT;
+ANALYZE prt1_ad;
+
+CREATE TABLE prt2_ad_default PARTITION OF prt2_ad DEFAULT;
+ANALYZE prt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt1_ad;
+DROP TABLE prt2_ad;
+
+-- Tests for list-partitioned tables
+CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0001', '0003');
+CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt1_ad_p3 PARTITION OF plt1_ad FOR VALUES IN ('0008', '0009');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 NOT IN (0, 2, 5, 7);
+ANALYZE plt1_ad;
+
+CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt2_ad_p3 PARTITION OF plt2_ad FOR VALUES IN ('0007', '0009');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 NOT IN (0, 1, 5, 8);
+ANALYZE plt2_ad;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+-- Test cases where one side has an extra partition
+
+-- Add an extra partition to plt2_ad
+CREATE TABLE plt2_ad_extra PARTITION OF plt2_ad FOR VALUES IN ('0000');
+INSERT INTO plt2_ad_extra VALUES (0, 0, '0000');
+ANALYZE plt2_ad;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2_ad t1 LEFT JOIN plt1_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt2_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt1_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt2_ad_extra;
+
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in those cases
+
+-- Split plt2_ad_p2 into two partitions so that prt1_ad_p2 matches both
+-- partitions
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2;
+CREATE TABLE plt2_ad_p2_0004 PARTITION OF plt2_ad FOR VALUES IN ('0004');
+CREATE TABLE plt2_ad_p2_0006 PARTITION OF plt2_ad FOR VALUES IN ('0006');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6);
+ANALYZE plt2_ad;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt2_ad_p2_0004;
+DROP TABLE plt2_ad_p2_0006;
+
+-- Restore plt2_ad_p2
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_ad;
+
+-- Test NULL partitions
+
+-- Change plt1_ad_p1 to the NULL partition
+ALTER TABLE plt1_ad DETACH PARTITION plt1_ad_p1;
+CREATE TABLE plt1_ad_p1_null PARTITION OF plt1_ad FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt1_ad VALUES (-1, -1, NULL);
+ANALYZE plt1_ad;
+
+-- Change plt2_ad_p3 to the NULL partition
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p3;
+CREATE TABLE plt2_ad_p3_null PARTITION OF plt2_ad FOR VALUES IN (NULL, '0007', '0009');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (7, 9);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt1_ad_p1_null;
+DROP TABLE plt2_ad_p3_null;
+
+-- Restore plt1_ad_p1
+ALTER TABLE plt1_ad ATTACH PARTITION plt1_ad_p1 FOR VALUES IN ('0001', '0003');
+
+-- Add to plt1_ad the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt1_ad_extra PARTITION OF plt1_ad FOR VALUES IN (NULL);
+INSERT INTO plt1_ad VALUES (-1, -1, NULL);
+ANALYZE plt1_ad;
+
+-- Restore plt2_ad_p3
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p3 FOR VALUES IN ('0007', '0009');
+ANALYZE plt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Partitioned join can't be applied because there isn't any partition on the
+-- plt2_ad side that matches the NULL partition of plt1_ad plt1_ad_extra
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Partitioned join can't be applied because there isn't any partition on the
+-- plt2_ad side that matches the NULL partition of plt1_ad plt1_ad_extra
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+-- Add to plt2_ad the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt2_ad_extra PARTITION OF plt2_ad FOR VALUES IN (NULL);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt1_ad_extra;
+DROP TABLE plt2_ad_extra;
+
+-- Test default partitions
+
+-- Change plt1_ad_p1 to the default partition
+ALTER TABLE plt1_ad DETACH PARTITION plt1_ad_p1;
+ALTER TABLE plt1_ad ATTACH PARTITION plt1_ad_p1 DEFAULT;
+DROP TABLE plt1_ad_p3;
+ANALYZE plt1_ad;
+
+DROP TABLE plt2_ad_p3;
+ANALYZE plt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Change plt2_ad_p2 to contain '0005' in addition to '0004' and '0006' as
+-- the key values
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2;
+CREATE TABLE plt2_ad_p2_ext PARTITION OF plt2_ad FOR VALUES IN ('0004', '0005', '0006');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 5, 6);
+ANALYZE plt2_ad;
+
+-- Partitioned join can't be applied because the default partition of plt1_ad
+-- plt1_ad_p1 matches plt2_ad_p1 and plt2_ad_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Change plt2_ad_p2_ext to the default partition
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2_ext;
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2_ext DEFAULT;
+ANALYZE plt2_ad;
+
+-- Partitioned join can't be applied because the default partition of plt1_ad
+-- plt1_ad_p1 matches plt2_ad_p1 and plt2_ad_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt2_ad_p2_ext;
+
+-- Restore plt2_ad_p2
+ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_ad;
+
+CREATE TABLE plt3_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_ad_p1 PARTITION OF plt3_ad FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt3_ad_p2 PARTITION OF plt3_ad FOR VALUES IN ('0007', '0009');
+INSERT INTO plt3_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6, 7, 9);
+ANALYZE plt3_ad;
+
+-- 3-way join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_ad t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_ad t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Test cases where one side has the default partition while the other side
+-- has the NULL partition
+
+-- Change plt2_ad_p1 to the NULL partition
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p1;
+CREATE TABLE plt2_ad_p1_null PARTITION OF plt2_ad FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Change the NULL partition to contain only NULL values as the key values
+ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p1_null;
+CREATE TABLE plt2_ad_p1_nullonly PARTITION OF plt2_ad FOR VALUES IN (NULL);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt2_ad_p1_nullonly;
+DROP TABLE plt2_ad_p1_null;
+DROP TABLE plt2_ad_p1;
+
+DROP TABLE plt1_ad;
+DROP TABLE plt2_ad;
+DROP TABLE plt3_ad;
+
+
+CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1_ad;
+
+CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0002');
+CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2_ad;
+
+CREATE TABLE plt3_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_ad_p1 PARTITION OF plt3_ad FOR VALUES IN ('0001');
+CREATE TABLE plt3_ad_p2 PARTITION OF plt3_ad FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.c = t2.c)) FULL JOIN plt3_ad t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.c = t2.c)) FULL JOIN plt3_ad t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+
+DROP TABLE plt1_ad;
+DROP TABLE plt2_ad;
+DROP TABLE plt3_ad;
+
+-- Test interaction of partitioned join with partition pruning
+
+CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0001');
+CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0002');
+CREATE TABLE plt1_ad_p3 PARTITION OF plt1_ad FOR VALUES IN ('0003');
+CREATE TABLE plt1_ad_p4 PARTITION OF plt1_ad FOR VALUES IN (NULL, '0004', '0005');
+INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt1_ad VALUES (-1, -1, NULL);
+ANALYZE plt1_ad;
+
+CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0003');
+CREATE TABLE plt2_ad_p3 PARTITION OF plt2_ad FOR VALUES IN (NULL);
+CREATE TABLE plt2_ad_p4 PARTITION OF plt2_ad FOR VALUES IN ('0004', '0005');
+INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt2_ad VALUES (-1, -1, NULL);
+ANALYZE plt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 AND t2.c IS NOT NULL ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 AND t2.c IS NOT NULL ORDER BY t1.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+
+CREATE TABLE plt1_ad_default PARTITION OF plt1_ad DEFAULT;
+ANALYZE plt1_ad;
+
+CREATE TABLE plt2_ad_default PARTITION OF plt2_ad DEFAULT;
+ANALYZE plt2_ad;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 AND t2.c IS NOT NULL ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 AND t2.c IS NOT NULL ORDER BY t1.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt1_ad;
+DROP TABLE plt2_ad;
+
+-- Tests for multi-level partitioned tables
+
+CREATE TABLE alpha (a double precision, b int, c text) PARTITION BY RANGE (a);
+CREATE TABLE alpha_neg PARTITION OF alpha FOR VALUES FROM ('-Infinity') TO (0) PARTITION BY RANGE (b);
+CREATE TABLE alpha_pos PARTITION OF alpha FOR VALUES FROM (0) TO (10.0) PARTITION BY LIST (c);
+CREATE TABLE alpha_neg_p1 PARTITION OF alpha_neg FOR VALUES FROM (100) TO (200);
+CREATE TABLE alpha_neg_p2 PARTITION OF alpha_neg FOR VALUES FROM (200) TO (300);
+CREATE TABLE alpha_neg_p3 PARTITION OF alpha_neg FOR VALUES FROM (300) TO (400);
+CREATE TABLE alpha_pos_p1 PARTITION OF alpha_pos FOR VALUES IN ('0001', '0003');
+CREATE TABLE alpha_pos_p2 PARTITION OF alpha_pos FOR VALUES IN ('0004', '0006');
+CREATE TABLE alpha_pos_p3 PARTITION OF alpha_pos FOR VALUES IN ('0008', '0009');
+INSERT INTO alpha_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+INSERT INTO alpha_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+ANALYZE alpha;
+
+CREATE TABLE beta (a double precision, b int, c text) PARTITION BY RANGE (a);
+CREATE TABLE beta_neg PARTITION OF beta FOR VALUES FROM (-10.0) TO (0) PARTITION BY RANGE (b);
+CREATE TABLE beta_pos PARTITION OF beta FOR VALUES FROM (0) TO ('Infinity') PARTITION BY LIST (c);
+CREATE TABLE beta_neg_p1 PARTITION OF beta_neg FOR VALUES FROM (100) TO (150);
+CREATE TABLE beta_neg_p2 PARTITION OF beta_neg FOR VALUES FROM (200) TO (300);
+CREATE TABLE beta_neg_p3 PARTITION OF beta_neg FOR VALUES FROM (350) TO (500);
+CREATE TABLE beta_pos_p1 PARTITION OF beta_pos FOR VALUES IN ('0002', '0003');
+CREATE TABLE beta_pos_p2 PARTITION OF beta_pos FOR VALUES IN ('0004', '0006');
+CREATE TABLE beta_pos_p3 PARTITION OF beta_pos FOR VALUES IN ('0007', '0009');
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+ANALYZE beta;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b;
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b >= 125 AND t1.b < 225 AND t2.b >= 125 AND t2.b < 225 AND t1.c IN ('0003') ORDER BY t1.a, t1.b, t2.a, t2.b;
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b >= 125 AND t1.b < 225 AND t2.b >= 125 AND t2.b < 225 AND t1.c IN ('0003') ORDER BY t1.a, t1.b, t2.a, t2.b;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE t1.b > 300 AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE t1.b < 300 AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;
#121Ashutosh Bapat
ashutosh.bapat.oss@gmail.com
In reply to: Etsuro Fujita (#120)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join
Hi Fujita-san,
I reviewed the patch. Except for the logic of matching the pairs of
partitions from already merged partitions, I think the code changes are
good. But there are several places where it needs further changes to
comments. The attached patch has those. I have described some of them below.
+ * We can not perform partition-wise join if either of the joining
+ * relations is not partitioned.

We are consistently using partitionwise instead of partition-wise.

+ /*
+ * See if the partition bounds for inputs are exactly the same, in
+ * which case we don't need to work hard: partitions with the same
+ * partition indexes will form join pairs, and the join rel will have
+ * the same partition bounds as inputs; otherwise try to merge the
+ * partition bounds along with generating join pairs.

Phrase "joining relations" is better than "inputs", IMO. Updated in the
attached patch.

+ /*
+ * If the partition bounds for the join rel are not merged ones,
+ * inputs are guaranteed to have the same partition bounds, so
+ * partitions with the same partition indexes will form join pairs;
+ * else let get_matching_part_pairs() do the work.
+ */
+ if (joinrel->merged)
+ {

This condition in the comment is opposite to the condition being checked in
code, so looks confusing. BTW this comment is also repeated around line
1405.
See attached patch for correction.

+ /*
+ * If this segment of the join is empty, it means that this segment

"partition of the join" looks consistent with other usages than "segment of
the
join". Modified in the attached patch.

+ /*
+ * Get a relids set of partition(s) involved in this join segment that
+ * are from the rel1 side.
+ */
+ child_relids1 = bms_intersect(child_joinrel->relids,
+  rel1->all_partrels);

The partition bounds are sorted by their values. Even for a list partitioned
table, the bounds are sorted by the least partition value. We do not allow
multiple paritions from one side to be joined with one partition on the
other
and vice versa. All this together means that the partitions of the join
relation are formed by joining partitions from either side in the order of
their bounds. This means that the matching pairs of partitions can be found
by
matching relids of partitions of join with those of the joining relation by
traversing partitions from all the three relations only once in the order
they
appears in partition bounds of corresponding relations. If we use this
algorithm, we don't need all_partrels to be collected and also don't need to
search base or join relation. That, I think, will reduce the time and space
complexity of this logic. Am I missing something? As a side effect it won't
require any special handling for base and join relation.

+ /*
+ * Get a child rel for rel1 with the relids.  Note that we should have
+ * the child rel even if rel1 is a join rel, because in that case the
+ * partitions specified in the relids would have matching/overlapping
+ * boundaries, so those partitions should be considered as ones to be
+ * joined even when planning partitionwise joins of rel1, meaning that
+ * the child rel would have been built by the time we get here.
+ */
+ if (rel1_is_simple)

This variable is used only in one place. So instead we should the expression
assigning the value to it. Changed in the attached patch.

- rel->nparts = 0;
+ rel->nparts = -1;

I think we need to add comments around various values that nparts can take.
How
about like something attached.

+ case PARTITION_STRATEGY_HASH:
+ merged_bounds = NULL;

I think, we need to explain why we aren't merging hash partition bounds.
AFAIU,
the reason is thus: When the modulo of used for partition mapping (i.e.
maximum
number of has partitions) is same, their partition bounds are same and do
not
need merging. If the maximum number of partitions is different for both the
joining relations, there's high probability that one partition on one side
will
join with multiple partitions on the other side. So exact partition bounds
match will work in most of the cases. The cases otherwise are not so common
to
spend the effort in coding and planning.

I have added this explanation in the patch. Don't know if it's there written
somewhere already.

+ if (part_index == -1)
+ return -1;
+ } while (is_dummy_partition(rel, part_index));

I understand why we are skipping NULL positions. I am not sure why are we
are
skipping over RelOptInfos which exist but are marked as dummy; we can still
create a join pair with those partitions.

+/*
+ * get_merged_range_bounds
+ * Given the bounds of range partitions to be join, determine the range

s/join/joined/

There are more changes to comments, where I thought that the comments are
required or existing comments need more clarification. Please review the
attached patch. This patch is created on top of
v32-0001-Improve-partition-matching-for-partitionwise-join.

On Mon, Feb 10, 2020 at 5:14 PM Etsuro Fujita <etsuro.fujita@gmail.com>
wrote:

On Fri, Feb 7, 2020 at 9:57 PM Etsuro Fujita <etsuro.fujita@gmail.com>
wrote:

On Thu, Feb 6, 2020 at 3:55 AM Mark Dilger <mark.dilger@enterprisedb.com>

wrote:

The patches apply and pass all tests. A review of the patch vs.

master looks reasonable.

I've merged the patches. Attached is a new version of the patch.

The partition_join.sql test has multiple levels of partitioning, but

when your patch extends that test with “advanced partition-wise join”, none
of the tables for the new section have multiple levels. I spent a little
while reviewing the code and inventing multiple level partitioning tests
for advanced partition-wise join and did not encounter any problems. I
don’t care whether you use this particular example, but do you want to have
multiple level partitioning in the new test section?

Yes, I do.

CREATE TABLE alpha (a double precision, b double precision) PARTITION

BY RANGE (a);

CREATE TABLE alpha_neg PARTITION OF alpha FOR VALUES FROM

('-Infinity') TO (0) PARTITION BY RANGE (b);

CREATE TABLE alpha_pos PARTITION OF alpha FOR VALUES FROM (0) TO

('Infinity') PARTITION BY RANGE (b);

CREATE TABLE alpha_nan PARTITION OF alpha FOR VALUES FROM ('Infinity')

TO ('NaN');

CREATE TABLE alpha_neg_neg PARTITION OF alpha_neg FOR VALUES FROM

('-Infinity') TO (0);

CREATE TABLE alpha_neg_pos PARTITION OF alpha_neg FOR VALUES FROM (0)

TO ('Infinity');

CREATE TABLE alpha_neg_nan PARTITION OF alpha_neg FOR VALUES FROM

('Infinity') TO ('NaN');

CREATE TABLE alpha_pos_neg PARTITION OF alpha_pos FOR VALUES FROM

('-Infinity') TO (0);

CREATE TABLE alpha_pos_pos PARTITION OF alpha_pos FOR VALUES FROM (0)

TO ('Infinity');

CREATE TABLE alpha_pos_nan PARTITION OF alpha_pos FOR VALUES FROM

('Infinity') TO ('NaN');

INSERT INTO alpha (a, b)
(SELECT * FROM
(VALUES (-1.0::float8), (0.0::float8), (1.0::float8),

('Infinity'::float8)) a,

(VALUES (-1.0::float8), (0.0::float8), (1.0::float8),

('Infinity'::float8)) b

);
ANALYZE alpha;
ANALYZE alpha_neg;
ANALYZE alpha_pos;
ANALYZE alpha_nan;
ANALYZE alpha_neg_neg;
ANALYZE alpha_neg_pos;
ANALYZE alpha_neg_nan;
ANALYZE alpha_pos_neg;
ANALYZE alpha_pos_pos;
ANALYZE alpha_pos_nan;
CREATE TABLE beta (a double precision, b double precision) PARTITION

BY RANGE (a, b);

CREATE TABLE beta_lo PARTITION OF beta FOR VALUES FROM (-5, -5) TO (0,

0);

CREATE TABLE beta_me PARTITION OF beta FOR VALUES FROM (0, 0) TO (0,

5);

CREATE TABLE beta_hi PARTITION OF beta FOR VALUES FROM (0, 5) TO (5,

5);

INSERT INTO beta (a, b)
(SELECT * FROM
(VALUES (-1.0::float8), (0.0::float8), (1.0::float8)) a,
(VALUES (-1.0::float8), (0.0::float8), (1.0::float8)) b
);
ANALYZE beta;
ANALYZE beta_lo;
ANALYZE beta_me;
ANALYZE beta_hi;
EXPLAIN SELECT * FROM alpha INNER JOIN beta ON (alpha.a = beta.a AND

alpha.b = beta.b) WHERE alpha.a = 1 AND beta.b = 1;

QUERY PLAN

-------------------------------------------------------------------------------

Nested Loop (cost=0.00..2.11 rows=1 width=32)
-> Seq Scan on alpha_pos_pos alpha (cost=0.00..1.06 rows=1

width=16)

Filter: ((b = '1'::double precision) AND (a = '1'::double

precision))

-> Seq Scan on beta_hi beta (cost=0.00..1.04 rows=1 width=16)
Filter: ((b = '1'::double precision) AND (a = '1'::double

precision))

(5 rows)

Hmm, I'm not sure this is a good test case for that, because this
result would be due to partition pruning applied to each side of the
join before considering partition-wise join; you could get the same
result even with enable_partitionwise_join=off. I think it's
important that the partition-wise join logic doesn't break this query,
though.

I think this would be beyond the scope of the patch, so I added
different test cases that I think would be better as ones for
multi-level partitioning.

Thanks!

Best regards,
Etsuro Fujita

--
--
Best Wishes,
Ashutosh Bapat

Attachments:

changes_on_top_of_v32_0001.patchtext/x-patch; charset=US-ASCII; name=changes_on_top_of_v32_0001.patchDownload
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 77e6ff5376..e14a2e51c2 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1378,7 +1378,7 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * We can not perform partition-wise join if either of the joining
+	 * We can not perform partitionwise join if either of the joining
 	 * relations is not partitioned.
 	 */
 	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
@@ -1399,8 +1399,9 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 
 	/*
 	 * If we don't have the partition bounds for the join rel yet, try to
-	 * create it along with pairs of partitions to be joined; else generate
-	 * those using the partitioning info for the join rel we already have.
+	 * compute those along with pairs of partitions to be joined; else generate
+	 * the pairs using the partitioning info of the join relation we already
+	 * have.
 	 */
 	if (joinrel->nparts == -1)
 	{
@@ -1412,16 +1413,17 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		Assert(joinrel->part_rels == NULL);
 
 		/*
-		 * See if the partition bounds for inputs are exactly the same, in
-		 * which case we don't need to work hard: partitions with the same
-		 * partition indexes will form join pairs, and the join rel will have
-		 * the same partition bounds as inputs; otherwise try to merge the
-		 * partition bounds along with generating join pairs.
+		 * See if the partition bounds of the joining relations are exactly the
+		 * same, in which case we don't need to work hard: partitions with the
+		 * same partition indexes will form join pairs, and the join relation
+		 * will have the same partition bounds as the joining relations;
+		 * otherwise try to merge the partition bounds along with generating
+		 * join pairs.
 		 *
-		 * Even if one or both inputs have merged partition bounds, it'd be
-		 * possible for the partition bounds to be exactly the same, but it
-		 * seems unlikely to be worth the cycles to check; do this check only
-		 * if both inputs have non-merged partition bounds.
+		 * Even if one or both the joining relations have merged partition
+		 * bounds, it'd be possible for the partition bounds to be exactly the
+		 * same, but it seems unlikely to be worth the cycles to check; do this
+		 * check only if both inputs have non-merged partition bounds.
 		 */
 		if (!rel1->merged &&
 			!rel2->merged &&
@@ -1467,10 +1469,11 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		Assert(joinrel->part_rels);
 
 		/*
-		 * If the partition bounds for the join rel are not merged ones,
-		 * inputs are guaranteed to have the same partition bounds, so
-		 * partitions with the same partition indexes will form join pairs;
-		 * else let get_matching_part_pairs() do the work.
+		 * If the partition bounds of the join relation were computed by
+		 * merging the bounds of the joining relations, generate the pairs of
+		 * joining partitions by matching their relids. Nothing to do otherwise
+		 * since the partitions at same cardinal positions form the joining
+		 * pairs.
 		 */
 		if (joinrel->merged)
 		{
@@ -1847,8 +1850,6 @@ get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
 						RelOptInfo *rel1, RelOptInfo *rel2,
 						List **parts1, List **parts2)
 {
-	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
-	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
 	int 		cnt_parts;
 
 	*parts1 = NIL;
@@ -1863,7 +1864,7 @@ get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
 		Relids		child_relids2;
 
 		/*
-		 * If this segment of the join is empty, it means that this segment
+		 * If the current partition of the join is empty, it means that this segment
 		 * was ignored when previously creating child-join paths for it in
 		 * try_partitionwise_join() as it would not contribute to the join
 		 * result, due to one or both inputs being empty; add NULL to each of
@@ -1893,7 +1894,7 @@ get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
 		 * joined even when planning partitionwise joins of rel1, meaning that
 		 * the child rel would have been built by the time we get here.
 		 */
-		if (rel1_is_simple)
+		if (IS_SIMPLE_REL(rel1))
 		{
 			int			varno = bms_singleton_member(child_relids1);
 
@@ -1914,7 +1915,7 @@ get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
 		/*
 		 * Get a child rel for rel2 with the relids.  See above comments.
 		 */
-		if (rel2_is_simple)
+		if (IS_SIMPLE_REL(rel2))
 		{
 			int			varno = bms_singleton_member(child_relids2);
 
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 3c7b6030c0..f639e2c69b 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -3156,6 +3156,16 @@ partition_bounds_merge(int partnatts,
 	switch (strategy)
 	{
 		case PARTITION_STRATEGY_HASH:
+			/*
+			 * When the modulo of used for partition mapping (i.e. maximum
+			 * number of has partitions) is same, their partition bounds are
+			 * same and do not need merging. If the maximum number of
+			 * partitions is different for both the joining relations, there's
+			 * high probability that one partition on one side will join with
+			 * multiple partitions on the other side. So exact partition bounds
+			 * match will work in most of the cases. The cases otherwise are
+			 * not so common to spend the effort in coding and planning.
+			 */
 			merged_bounds = NULL;
 
 			break;
@@ -3324,7 +3334,7 @@ compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
 
 /*
  * get_merged_range_bounds
- *		Given the bounds of range partitions to be join, determine the range
+ *		Given the bounds of range partitions to be joined, determine the range
  *		bounds of the merged partition produced from the range partitions
  *
  * *merged_lb and *merged_ub are set to the lower and upper bounds of the
@@ -3534,7 +3544,7 @@ partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs,
 		 * the side which finishes earlier has an extra partition with lower
 		 * and upper bounds higher than any other partition of the unfinished
 		 * side. That way we advance the partitions on that side till all of
-		 * them are  exhausted.
+		 * them are exhausted.
 		 */
 		if (outer_part == -1)
 		{
@@ -4093,8 +4103,13 @@ map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
 		if (!outer_merged && !inner_merged)
 		{
 			/*
-			 * Note that we will fix the larger index that have been added to
-			 * the merged_indexes list so far in fix_merged_indexes().
+			 * Both the inner and outer partitions have an empty partition on
+			 * the other side as their joining partner. But now that each of
+			 * them has found a non-empty joining partner we should re-map
+			 * those to a single partition in the join. We use lower of the
+			 * two indexes to avoid any holes being created by re-mapping.
+			 * Also, it keeps the partition index array in partition bounds
+			 * roughly sorted.
 			 */
 			if (outer_merged_index < inner_merged_index)
 			{
@@ -4165,6 +4180,12 @@ map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
 /*
  * merge_partition_with_dummy
  *
+ * The caller thinks that the partition at the given index does not have a
+ * partition in the other relation or the joining partition is empty. In such a
+ * case we assign a temporary index (indicated by merged flag in the map) for
+ * the resulting partition in the join. In case the given partition finds a
+ * non-empty partner latter we will adjust the mapping again.
+ *
  * *next_index is incremented.
  */
 static int
@@ -4206,8 +4227,8 @@ process_outer_partition(PartitionMap *outer_map,
 
 	/*
 	 * If the inner side has the default partition, the outer partition has to
-	 * be joined with the default partition; try merging them.  Otherwise, we
-	 * should in an outer join, in which case the outer partition has to be
+	 * be joined with the default partition; try merging them.  Otherwise
+	 * it's an outer join, in which case the outer partition has to be
 	 * scanned all the way anyway; if the outer partition is already mapped to
 	 * a merged partition, get it, otherwise create a new merged partition by
 	 * merging the outer partition with a dummy partition.
@@ -4217,9 +4238,11 @@ process_outer_partition(PartitionMap *outer_map,
 		Assert(inner_default >= 0);
 
 		/*
-		 * If the outer side has the default partition as well, we need to
-		 * merge the default partitions (see merge_default_partitions()); give
-		 * up on it.
+		 * If the outer side has the default partition as well, the default
+		 * partition from inner side will have two matching partitions on the
+		 * outer side: the default partition on the outer side and the given
+		 * outer partition. Partitionwise join doesn't handle this scenario
+		 * yet.
 		 */
 		if (outer_has_default)
 			return -1;
@@ -4231,9 +4254,13 @@ process_outer_partition(PartitionMap *outer_map,
 			return -1;
 
 		/*
-		 * If this is a FULL join, the merged partition would act as the
-		 * default partition of the join; record the index in *default_index
-		 * if not done yet.
+		 * If this is a FULL join, both the sides act as outer side. Since the
+		 * inner partition is a default partition, it will have partition key
+		 * values which are not covered by any other partition. In join result
+		 * as well, the resulting partition will hold partition key values that
+		 * no other partition holds. Thus the merged partition would act as the
+		 * default partition of the join; record the index in *default_index if
+		 * not done yet.
 		 */
 		if (jointype == JOIN_FULL)
 		{
@@ -4293,8 +4320,11 @@ process_inner_partition(PartitionMap *outer_map,
 
 		/*
 		 * If the inner side has the default partition as well, we need to
-		 * merge the default partitions (see merge_default_partitions()); give
-		 * up on it.
+		 * merge the default partitions (see merge_default_partitions()). So
+		 * there will two inner partitions, given inner partition and the
+		 * default inner partition, that will map to the default outer
+		 * partition. Partitionwise join does not support this case, so give up
+		 * on it.
 		 */
 		if (inner_has_default)
 			return -1;
@@ -4590,6 +4620,13 @@ merge_default_partitions(PartitionMap *outer_map,
 	}
 	else
 	{
+		/*
+		 * We should have already given up if we found that both the inner and
+		 * outer relations have default partitions and either of them had a
+		 * partition without a matching non-default partition on the other
+		 * side. See process_outer_partition() and process_inner_partition()
+		 * for details.
+		 */
 		Assert(outer_has_default && inner_has_default);
 		Assert(outer_merged_index == -1);
 		Assert(inner_merged_index == -1);
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index 1545877d8c..bdf8ac4bce 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -720,7 +720,12 @@ typedef struct RelOptInfo
 
 	/* used for partitioned relations */
 	PartitionScheme part_scheme;	/* Partitioning scheme. */
-	int			nparts;			/* number of partitions */
+	int			nparts;			/* number of partitions.
+								 * 0 for a relation with no partitions,
+								 * > 0 indicates actual number of partitions
+								 * -1 for a relation whose number of partitions
+								 *    is not yet known.
+								 */
 	struct PartitionBoundInfoData *boundinfo;	/* Partition bounds */
 	bool		merged;			/* true if partition bounds were created by
 								 * partition_bounds_merge() */
#122Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Ashutosh Bapat (#121)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Hi Ashutosh,

On Wed, Mar 4, 2020 at 1:48 AM Ashutosh Bapat
<ashutosh.bapat.oss@gmail.com> wrote:

I reviewed the patch. Except for the logic of matching the pairs of partitions from already merged partitions, I think the code changes are good. But there are several places where it needs further changes to comments. The attached patch has those. I have described some of them below.

Thanks for reviewing!

+ * We can not perform partition-wise join if either of the joining
+ * relations is not partitioned.

We are consistently using partitionwise instead of partition-wise.

Will fix.

+ /*
+ * See if the partition bounds for inputs are exactly the same, in
+ * which case we don't need to work hard: partitions with the same
+ * partition indexes will form join pairs, and the join rel will have
+ * the same partition bounds as inputs; otherwise try to merge the
+ * partition bounds along with generating join pairs.

Phrase "joining relations" is better than "inputs", IMO. Updated in the
attached patch.

"inputs" is used in many places in the planner performing join
planning, so I'm not sure "joining relations" is better than "inputs".

+ /*
+ * If the partition bounds for the join rel are not merged ones,
+ * inputs are guaranteed to have the same partition bounds, so
+ * partitions with the same partition indexes will form join pairs;
+ * else let get_matching_part_pairs() do the work.
+ */
+ if (joinrel->merged)
+ {

This condition in the comment is opposite to the condition being checked in
code, so looks confusing. BTW this comment is also repeated around line 1405.
See attached patch for correction.

OK, I'll revise the comments as proposed.

+ /*
+ * If this segment of the join is empty, it means that this segment

"partition of the join" looks consistent with other usages than "segment of the
join".

Actually, "segment" is used in the existing comments in the caller
function try_partitionwise_join(), so I think "segment" is better here
for consistency.

+ /*
+ * Get a relids set of partition(s) involved in this join segment that
+ * are from the rel1 side.
+ */
+ child_relids1 = bms_intersect(child_joinrel->relids,
+  rel1->all_partrels);

The partition bounds are sorted by their values. Even for a list partitioned
table, the bounds are sorted by the least partition value. We do not allow
multiple paritions from one side to be joined with one partition on the other
and vice versa. All this together means that the partitions of the join
relation are formed by joining partitions from either side in the order of
their bounds. This means that the matching pairs of partitions can be found by
matching relids of partitions of join with those of the joining relation by
traversing partitions from all the three relations only once in the order they
appears in partition bounds of corresponding relations.

Consider this 2-way join for list partitioned tables:

CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0001', '0003');
CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0002');
INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM
generate_series(1, 100) i WHERE i % 10 in (1, 2, 3);
ANALYZE plt1_ad;
CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c);
CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0002', '0004');
CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0003');
INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM
generate_series(1, 100) i WHERE i % 10 IN (2, 3, 4);
ANALYZE plt2_ad;

EXPLAIN (COSTS OFF)
SELECT t1.c, t1.a, t2.a FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.c
= t2.c) WHERE t1.a < 10 ORDER BY t1.c, t1.a, t2.a;
QUERY PLAN
-----------------------------------------------------
Sort
Sort Key: t1.c, t1.a, t2.a
-> Append
-> Hash Join
Hash Cond: (t2_1.c = t1_2.c)
-> Seq Scan on plt2_ad_p1 t2_1
-> Hash
-> Seq Scan on plt1_ad_p2 t1_2
Filter: (a < 10)
-> Hash Join
Hash Cond: (t2_2.c = t1_1.c)
-> Seq Scan on plt2_ad_p2 t2_2
-> Hash
-> Seq Scan on plt1_ad_p1 t1_1
Filter: (a < 10)
(15 rows)

As you can see from the EXPLAIN result, the first partition on the
outer side matches the second partition on the inner side, and the
second partition on the outer side matches the first partition on the
inner side. How does the algorithm you proposed work e.g., when an
N-way join for list partitioned tables contains this join as its lower
join?

If we use this
algorithm, we don't need all_partrels to be collected and also don't need to
search base or join relation. That, I think, will reduce the time and space
complexity of this logic. Am I missing something?

Relids is used for storing all_partrels, so the time/space cost of
handling it would be small. Also, the cost of searing base relations
would be small. The cost of searching join relations would be a bit
large in some cases, but I thought that would be acceptable, compared
with large overhead of performing other part of partitionwise join
planning.

+ if (rel1_is_simple)

This variable is used only in one place. So instead we should the expression
assigning the value to it. Changed in the attached patch.

I don't think that's a good idea, because this check is done
repeatedly in a for loop.

- rel->nparts = 0;
+ rel->nparts = -1;

I think we need to add comments around various values that nparts can take. How
about like something attached.

+1

+ case PARTITION_STRATEGY_HASH:
+ merged_bounds = NULL;

I think, we need to explain why we aren't merging hash partition bounds. AFAIU,
the reason is thus: When the modulo of used for partition mapping (i.e. maximum
number of has partitions) is same, their partition bounds are same and do not
need merging.

I don't think that's always true; there are cases where the moduli are
the same, but the partition bounds are not, because it's possible to
only define partitions for some of the remainders. See the discussion
in [1]/messages/by-id/CAAJ_b94tJTix3kR8uBjin-ruJ-7ojn-gAWJQRicbLqAttQTe1g@mail.gmail.com.

If the maximum number of partitions is different for both the
joining relations, there's high probability that one partition on one side will
join with multiple partitions on the other side. So exact partition bounds
match will work in most of the cases. The cases otherwise are not so common to
spend the effort in coding and planning.

I have added this explanation in the patch.

I also think it would be great if we can perform generic partitionwise
join for hash partitioned tables, so I'd like to propose to add
something like this, instead: "Currently we support partitionwise join
for hash partitioned tables only when the partition bounds for them
exactly match, but later it might be worth the effort to relax the
restriction."

+ if (part_index == -1)
+ return -1;
+ } while (is_dummy_partition(rel, part_index));

I understand why we are skipping NULL positions. I am not sure why are we are
skipping over RelOptInfos which exist but are marked as dummy; we can still
create a join pair with those partitions.

Yeah, but I think it's safe to skip over those partitions as well,
because such a join pair can be created using
merge_partition_with_dummy().

+/*
+ * get_merged_range_bounds
+ * Given the bounds of range partitions to be join, determine the range

s/join/joined/

Good catch! Will fix.

There are more changes to comments, where I thought that the comments are
required or existing comments need more clarification. Please review the
attached patch. This patch is created on top of
v32-0001-Improve-partition-matching-for-partitionwise-join.

Thanks for the patch! I will review the patch ASAP.

Sorry for the delay.

Best regards,
Etsuro Fujita

[1]: /messages/by-id/CAAJ_b94tJTix3kR8uBjin-ruJ-7ojn-gAWJQRicbLqAttQTe1g@mail.gmail.com

#123Ashutosh Bapat
ashutosh.bapat.oss@gmail.com
In reply to: Etsuro Fujita (#122)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Tue, Mar 17, 2020 at 1:44 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

+ /*
+ * If this segment of the join is empty, it means that this segment

"partition of the join" looks consistent with other usages than "segment of the
join".

Actually, "segment" is used in the existing comments in the caller
function try_partitionwise_join(), so I think "segment" is better here
for consistency.

A segment can be any part of the join relation, not necessarily a
partition. May be we should change the caller.

+ /*
+ * Get a relids set of partition(s) involved in this join segment that
+ * are from the rel1 side.
+ */
+ child_relids1 = bms_intersect(child_joinrel->relids,
+  rel1->all_partrels);

The partition bounds are sorted by their values. Even for a list partitioned
table, the bounds are sorted by the least partition value. We do not allow
multiple paritions from one side to be joined with one partition on the other
and vice versa. All this together means that the partitions of the join
relation are formed by joining partitions from either side in the order of
their bounds. This means that the matching pairs of partitions can be found by
matching relids of partitions of join with those of the joining relation by
traversing partitions from all the three relations only once in the order they
appears in partition bounds of corresponding relations.

Consider this 2-way join for list partitioned tables:

CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c);
CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0001', '0003');
CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0002');
INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM
generate_series(1, 100) i WHERE i % 10 in (1, 2, 3);
ANALYZE plt1_ad;
CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c);
CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0002', '0004');
CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0003');
INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM
generate_series(1, 100) i WHERE i % 10 IN (2, 3, 4);
ANALYZE plt2_ad;

EXPLAIN (COSTS OFF)
SELECT t1.c, t1.a, t2.a FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.c
= t2.c) WHERE t1.a < 10 ORDER BY t1.c, t1.a, t2.a;
QUERY PLAN
-----------------------------------------------------
Sort
Sort Key: t1.c, t1.a, t2.a
-> Append
-> Hash Join
Hash Cond: (t2_1.c = t1_2.c)
-> Seq Scan on plt2_ad_p1 t2_1
-> Hash
-> Seq Scan on plt1_ad_p2 t1_2
Filter: (a < 10)
-> Hash Join
Hash Cond: (t2_2.c = t1_1.c)
-> Seq Scan on plt2_ad_p2 t2_2
-> Hash
-> Seq Scan on plt1_ad_p1 t1_1
Filter: (a < 10)
(15 rows)

As you can see from the EXPLAIN result, the first partition on the
outer side matches the second partition on the inner side, and the
second partition on the outer side matches the first partition on the
inner side. How does the algorithm you proposed work e.g., when an
N-way join for list partitioned tables contains this join as its lower
join?

Hmm, this is a good example. I tried to work out the algorithm based
on the bound ordering. The algorithm worked well when all the bounds
on both the sides were included in the join. But it didn't work well,
when some bounds vanished. In order to detect whether a bound has
vanished, we need to either compare that bound with the bounds of join
(an operation costlier than comparing bitmapset) or compare relids of
all the partitions of the join. Either way it looks costlier than what
you have right now. May be we could improve by keeping track of such
lost bounds and corresponding partitions. But I didn't get time to
work on that part. Anyway, even if such an algorithm exists, we will
have to change just a single function. That could be done later I
think. So we are good here right now. Thanks.

+ if (rel1_is_simple)

This variable is used only in one place. So instead we should the expression
assigning the value to it. Changed in the attached patch.

I don't think that's a good idea, because this check is done
repeatedly in a for loop.

Compiler's optimizer would anyway optimize it away. But anyway, I
won't insist on this.

- rel->nparts = 0;
+ rel->nparts = -1;

I think we need to add comments around various values that nparts can take. How
about like something attached.

+1

+ case PARTITION_STRATEGY_HASH:
+ merged_bounds = NULL;

I think, we need to explain why we aren't merging hash partition bounds. AFAIU,
the reason is thus: When the modulo of used for partition mapping (i.e. maximum
number of has partitions) is same, their partition bounds are same and do not
need merging.

I don't think that's always true; there are cases where the moduli are
the same, but the partition bounds are not, because it's possible to
only define partitions for some of the remainders. See the discussion
in [1].

Hmm, but that case would be rare, IMO. It's an artifact of the way our
hash partitioning syntax is, and does not reflect a real world
scenario.

If the maximum number of partitions is different for both the
joining relations, there's high probability that one partition on one side will
join with multiple partitions on the other side. So exact partition bounds
match will work in most of the cases. The cases otherwise are not so common to
spend the effort in coding and planning.

I have added this explanation in the patch.

I also think it would be great if we can perform generic partitionwise
join for hash partitioned tables, so I'd like to propose to add
something like this, instead: "Currently we support partitionwise join
for hash partitioned tables only when the partition bounds for them
exactly match, but later it might be worth the effort to relax the
restriction."

That's good too. But please include an explanation about the case when
modulo/max no. of partitions itself differs. That case is not likely
to get addressed in nearer future.

--
Best Wishes,
Ashutosh Bapat

#124Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Ashutosh Bapat (#123)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Hi,

On Mon, Mar 23, 2020 at 10:42 PM Ashutosh Bapat
<ashutosh.bapat.oss@gmail.com> wrote:

On Tue, Mar 17, 2020 at 1:44 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

"partition of the join" looks consistent with other usages than "segment of the
join".

Actually, "segment" is used in the existing comments in the caller
function try_partitionwise_join(), so I think "segment" is better here
for consistency.

A segment can be any part of the join relation, not necessarily a
partition. May be we should change the caller.

I don't think so. My first language is not English, though. I don't
think this should be a blocker, so how about leaving this for another
patch?

+ /*
+ * Get a relids set of partition(s) involved in this join segment that
+ * are from the rel1 side.
+ */
+ child_relids1 = bms_intersect(child_joinrel->relids,
+  rel1->all_partrels);

The partition bounds are sorted by their values. Even for a list partitioned
table, the bounds are sorted by the least partition value. We do not allow
multiple paritions from one side to be joined with one partition on the other
and vice versa. All this together means that the partitions of the join
relation are formed by joining partitions from either side in the order of
their bounds. This means that the matching pairs of partitions can be found by
matching relids of partitions of join with those of the joining relation by
traversing partitions from all the three relations only once in the order they
appears in partition bounds of corresponding relations.

Consider this 2-way join for list partitioned tables:

Hmm, this is a good example. I tried to work out the algorithm based
on the bound ordering. The algorithm worked well when all the bounds
on both the sides were included in the join. But it didn't work well,
when some bounds vanished. In order to detect whether a bound has
vanished, we need to either compare that bound with the bounds of join
(an operation costlier than comparing bitmapset) or compare relids of
all the partitions of the join. Either way it looks costlier than what
you have right now. May be we could improve by keeping track of such
lost bounds and corresponding partitions. But I didn't get time to
work on that part. Anyway, even if such an algorithm exists, we will
have to change just a single function. That could be done later I
think. So we are good here right now. Thanks.

OK, there is always room for improvement.

+ if (rel1_is_simple)

This variable is used only in one place. So instead we should the expression
assigning the value to it. Changed in the attached patch.

I don't think that's a good idea, because this check is done
repeatedly in a for loop.

Compiler's optimizer would anyway optimize it away. But anyway, I
won't insist on this.

OK

+ case PARTITION_STRATEGY_HASH:
+ merged_bounds = NULL;

I think, we need to explain why we aren't merging hash partition bounds. AFAIU,
the reason is thus: When the modulo of used for partition mapping (i.e. maximum
number of has partitions) is same, their partition bounds are same and do not
need merging.

I don't think that's always true; there are cases where the moduli are
the same, but the partition bounds are not, because it's possible to
only define partitions for some of the remainders. See the discussion
in [1].

Hmm, but that case would be rare, IMO. It's an artifact of the way our
hash partitioning syntax is, and does not reflect a real world
scenario.

I agree on that point.

If the maximum number of partitions is different for both the
joining relations, there's high probability that one partition on one side will
join with multiple partitions on the other side. So exact partition bounds
match will work in most of the cases. The cases otherwise are not so common to
spend the effort in coding and planning.

I have added this explanation in the patch.

I'd like to propose to add
something like this, instead: "Currently we support partitionwise join
for hash partitioned tables only when the partition bounds for them
exactly match, but later it might be worth the effort to relax the
restriction."

That's good too. But please include an explanation about the case when
modulo/max no. of partitions itself differs. That case is not likely
to get addressed in nearer future.

OK, I added comments, including that explanation. Please find
attached a new version of the patch.

On Tue, Mar 17, 2020 at 5:14 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

On Wed, Mar 4, 2020 at 1:48 AM Ashutosh Bapat
<ashutosh.bapat.oss@gmail.com> wrote:

+ * We can not perform partition-wise join if either of the joining
+ * relations is not partitioned.

We are consistently using partitionwise instead of partition-wise.

Will fix.

Done.

+ /*
+ * If the partition bounds for the join rel are not merged ones,
+ * inputs are guaranteed to have the same partition bounds, so
+ * partitions with the same partition indexes will form join pairs;
+ * else let get_matching_part_pairs() do the work.
+ */
+ if (joinrel->merged)
+ {

This condition in the comment is opposite to the condition being checked in
code, so looks confusing. BTW this comment is also repeated around line 1405.
See attached patch for correction.

OK, I'll revise the comments as proposed.

I updated this based on what you suggested.

- rel->nparts = 0;
+ rel->nparts = -1;

I think we need to add comments around various values that nparts can take. How
about like something attached.

+1

-   int         nparts;         /* number of partitions */
+   int         nparts;         /* number of partitions.
+                                * 0 for a relation with no partitions,
+                                * > 0 indicates actual number of partitions
+                                * -1 for a relation whose number of partitions
+                                *    is not yet known.
+                                */

I don't think the comment "0 for a relation with no partitions" is
correct. I think 0 means the relation is considered unpartitioned, so
I modified the comments as such. I also changed the format to match
other places in pathnodes.h.

+/*
+ * get_merged_range_bounds
+ * Given the bounds of range partitions to be join, determine the range

s/join/joined/

Good catch! Will fix.

Done.

There are more changes to comments, where I thought that the comments are
required or existing comments need more clarification. Please review the
attached patch.

I will review the patch ASAP.

I looked into the patch.

@@ -4093,8 +4103,13 @@ map_and_merge_partitions(PartitionMap
*outer_map, PartitionMap *inner_map,

-            * Note that we will fix the larger index that have been added to
-            * the merged_indexes list so far in fix_merged_indexes().
+            * Both the inner and outer partitions have an empty partition on
+            * the other side as their joining partner. But now that each of
+            * them has found a non-empty joining partner we should re-map
+            * those to a single partition in the join. We use lower of the
+            * two indexes to avoid any holes being created by re-mapping.
+            * Also, it keeps the partition index array in partition bounds
+            * roughly sorted.

OK, but I modified this further.

+ * The caller thinks that the partition at the given index does not have a
+ * partition in the other relation or the joining partition is empty. In such a
+ * case we assign a temporary index (indicated by merged flag in the map) for
+ * the resulting partition in the join. In case the given partition finds a
+ * non-empty partner latter we will adjust the mapping again.

Done. I modified this a bit, though.

@@ -4590,6 +4620,13 @@ merge_default_partitions(PartitionMap *outer_map,

+        * We should have already given up if we found that both the inner and
+        * outer relations have default partitions and either of them had a
+        * partition without a matching non-default partition on the other
+        * side. See process_outer_partition() and process_inner_partition()
+        * for details.

This seems to me a bit difficult to read. How about something simpler
in the attached?

I also updated comments in process_outer_partition() and
process_inner_partition(), almost as proposed in your patch.

I polished the patch further. Changes are:

* I removed the arguments parttyplen and parttypbyval for
partition_bounds_merge(), because we don't need them anymore. Also, I
removed these assertions from that function:

+   Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL));
+
+   Assert(list_length(*outer_parts) == list_length(*inner_parts));

because the first assertion isn't correct as we don't allow non-NULL
merged_bounds to have outer_parts=NIL and inner_parts=NIL, and because
the second assertion is redundant as we have the same in
partition_list_bounds_merge() and partition_range_bounds_merge(). I
modified assertions in these functions a bit, though.

* I renamed partition_list_bounds_merge() and
partition_range_bounds_merge() to merge_list_bounds() and
merge_range_bounds(), analogously to create_list_bounds() and
create_range_bounds() in partition_bounds_create(). Does that make
sense?

* In the old versions of the patch, map_and_merge_partitions() checked
whether it was possible to "map" one of given partitions to the other,
but that function doesn't do that anymore. That function assumes that
given partitions match each other, so I don't think the name is good
anymore. How about renaming that function to
merge_matching_partitions() or something like that?

* I think the names of variables used in functions added to
partbounds.c are inconsistent. For example, the names of the outer
and inner partition indexes in partition_list_bounds_merge() are
"o_index" and "i_index", while those in partition_range_bounds_merge()
are "outer_part" and "inner_part". IMO such inconsistencies would
make the code a bit difficult to read, so I modified the names of
those variables to be more consistent between those functions.

* I modified the code in merge_null_partitions() to match other
functions added to partbounds.c, but no functional changes. Also, I
think the names "outer_null_unmatched" and "inner_null_unmatched" are
a bit misleading, because I think the reader might incorrectly think
both the NULL partitions exist, but that isn't always true, so I
renamed them to what I think would be better.

* I added/tweaked comments a lot, fixing (probably my) typos and
grammatical errors.

* I did some cleanup, including fixing some format issues.

* I sorted functions added to partbounds.c in a more logical order,
and moved them to a more appropriate place in that source file.

* I added/modified some regression tests. I added/modified comments a
bit as well. One thing I should mention is a change to this:

+-- 3-way join where not every pair of relations can do partitioned
join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t2.a, t3.c FROM prt1_ad t1 RIGHT JOIN prt2_ad t2 ON
(t1.a = t2.b) INNER JOIN prt1_ad t3 ON (t2.b = t3.a) WHERE t1.b = 0
ORDER BY t1.a, t2.a, t3.c;

The RIGHT join will be transformed to an INNER join by outer join
reduction IIUC, so this wouldn't be what was originally intended. I
think this is my fault; I modified the original test case incorrectly
when I updated all the regression tests for this feature [1]/messages/by-id/CAPmGK16LsKXX=YYzc-PNiY6aaYApg1Gmkc6A14dnJsrBBmgd0g@mail.gmail.com. Sorry
for that. I fixed this issue.

Best regards,
Etsuro Fujita

[1]: /messages/by-id/CAPmGK16LsKXX=YYzc-PNiY6aaYApg1Gmkc6A14dnJsrBBmgd0g@mail.gmail.com

Attachments:

v33-0001-Improve-partition-matching-for-partitionwise-join.patchapplication/octet-stream; name=v33-0001-Improve-partition-matching-for-partitionwise-join.patchDownload
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index e084c3f069..79c768409c 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -2288,6 +2288,8 @@ _outRelOptInfo(StringInfo str, const RelOptInfo *node)
 	WRITE_BOOL_FIELD(has_eclass_joins);
 	WRITE_BOOL_FIELD(consider_partitionwise_join);
 	WRITE_BITMAPSET_FIELD(top_parent_relids);
+	WRITE_BOOL_FIELD(merged);
+	WRITE_BITMAPSET_FIELD(all_partrels);
 	WRITE_NODE_FIELD(partitioned_child_rels);
 }
 
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index a21c295b99..530ebed245 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -48,6 +48,9 @@ static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 												Relids left_relids, Relids right_relids);
 static int	match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel,
 										 bool strict_op);
+static void get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2);
 
 
 /*
@@ -1357,25 +1360,30 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	bool		merged = false;
+	List	   *parts1 = NIL;
+	List	   *parts2 = NIL;
+	ListCell   *lcr1 = NULL;
+	ListCell   *lcr2 = NULL;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partitionwise join if either of the joining relations
+	 * is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1390,34 +1398,104 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		   joinrel->part_scheme == rel2->part_scheme);
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * If we don't have the partition bounds for the join rel yet, try to
+	 * compute those along with pairs of partitions to be joined.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	if (joinrel->nparts == -1)
+	{
+		PartitionScheme part_scheme = joinrel->part_scheme;
+		PartitionBoundInfo boundinfo = NULL;
+		int			nparts = 0;
+
+		Assert(joinrel->boundinfo == NULL);
+		Assert(joinrel->part_rels == NULL);
+
+		/*
+		 * See if the partition bounds for inputs are exactly the same, in
+		 * which case we don't need to work hard: the join rel have the same
+		 * partition bounds as inputs, and the partitions with the same
+		 * cardinal positions form the pairs.
+		 *
+		 * Note: even in cases where one or both inputs have merged bounds,
+		 * it would be possible for both the bounds to be exactly the same, but
+		 * it seems unlikely to be worth the cycles to check.
+		 */
+		if (!rel1->merged &&
+			!rel2->merged &&
+			rel1->nparts == rel2->nparts &&
+			partition_bounds_equal(part_scheme->partnatts,
+								   part_scheme->parttyplen,
+								   part_scheme->parttypbyval,
+								   rel1->boundinfo, rel2->boundinfo))
+		{
+			boundinfo = rel1->boundinfo;
+			nparts = rel1->nparts;
+		}
+		else
+		{
+			/* Try merging the partition bounds for inputs. */
+			boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											   part_scheme->partsupfunc,
+											   part_scheme->partcollation,
+											   rel1, rel2,
+											   parent_sjinfo->jointype,
+											   &parts1, &parts2);
+			if (boundinfo == NULL)
+			{
+				joinrel->nparts = 0;
+				return;
+			}
+			nparts = list_length(parts1);
+			merged = true;
+		}
+
+		Assert(nparts > 0);
+		joinrel->boundinfo = boundinfo;
+		joinrel->merged = merged;
+		joinrel->nparts = nparts;
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+	}
+	else
+	{
+		Assert(joinrel->nparts > 0);
+		Assert(joinrel->boundinfo);
+		Assert(joinrel->part_rels);
+
+		/*
+		 * If the join rel's merged flag is true, it means inputs are not
+		 * guaranteed to have the same partition bounds, therefore we can't
+		 * assume that the partitions at the same cardinal positions form the
+		 * pairs; let get_matching_part_pairs() generate the pairs.  Otherwise,
+		 * nothing to do since we can assume that.
+		 */
+		if (joinrel->merged)
+		{
+			get_matching_part_pairs(root, joinrel, rel1, rel2,
+									&parts1, &parts2);
+			Assert(list_length(parts1) == joinrel->nparts);
+			Assert(list_length(parts2) == joinrel->nparts);
+			merged = true;
+		}
+	}
 
-	nparts = joinrel->nparts;
+	if (merged)
+	{
+		lcr1 = list_head(parts1);
+		lcr2 = list_head(parts2);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
-		bool		rel1_empty = (child_rel1 == NULL ||
-								  IS_DUMMY_REL(child_rel1));
-		bool		rel2_empty = (child_rel2 == NULL ||
-								  IS_DUMMY_REL(child_rel2));
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		bool		rel1_empty;
+		bool		rel2_empty;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1425,6 +1503,22 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		if (merged)
+		{
+			child_rel1 = lfirst_node(RelOptInfo, lcr1);
+			child_rel2 = lfirst_node(RelOptInfo, lcr2);
+			lcr1 = lnext(parts1, lcr1);
+			lcr2 = lnext(parts2, lcr2);
+		}
+		else
+		{
+			child_rel1 = rel1->part_rels[cnt_parts];
+			child_rel2 = rel2->part_rels[cnt_parts];
+		}
+
+		rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1));
+		rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2));
+
 		/*
 		 * Check for cases where we can prove that this segment of the join
 		 * returns no rows, due to one or both inputs being empty (including
@@ -1522,6 +1616,8 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 												 child_sjinfo,
 												 child_sjinfo->jointype);
 			joinrel->part_rels[cnt_parts] = child_joinrel;
+			joinrel->all_partrels = bms_add_members(joinrel->all_partrels,
+													child_joinrel->relids);
 		}
 
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
@@ -1738,3 +1834,99 @@ match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op)
 
 	return -1;
 }
+
+/*
+ * get_matching_part_pairs
+ *		Generate pairs of partitions to be joined from the two inputs
+ */
+static void
+get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2)
+{
+	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
+	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
+	int 		cnt_parts;
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
+	{
+		RelOptInfo *child_joinrel = joinrel->part_rels[cnt_parts];
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		Relids		child_relids1;
+		Relids		child_relids2;
+
+		/*
+		 * If this segment of the join is empty, it means that this segment
+		 * was ignored when previously creating child-join paths for it in
+		 * try_partitionwise_join() as it would not contribute to the join
+		 * result, due to one or both inputs being empty; add NULL to each of
+		 * the given lists so that this segment will be ignored again in that
+		 * function.
+		 */
+		if (!child_joinrel)
+		{
+			*parts1 = lappend(*parts1, NULL);
+			*parts2 = lappend(*parts2, NULL);
+			continue;
+		}
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel1 side.
+		 */
+		child_relids1 = bms_intersect(child_joinrel->relids,
+									  rel1->all_partrels);
+		Assert(bms_num_members(child_relids1) == bms_num_members(rel1->relids));
+
+		/*
+		 * Get a child rel for rel1 with the relids.  Note that we should have
+		 * the child rel even if rel1 is a join rel, because in that case the
+		 * partitions specified in the relids would have matching/overlapping
+		 * boundaries, so the specified partitions should be considered as ones
+		 * to be joined when planning partitionwise joins of rel1, meaning that
+		 * the child rel would have been built by the time we get here.
+		 */
+		if (rel1_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids1);
+
+			child_rel1 = find_base_rel(root, varno);
+		}
+		else
+			child_rel1 = find_join_rel(root, child_relids1);
+		Assert(child_rel1);
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel2 side.
+		 */
+		child_relids2 = bms_intersect(child_joinrel->relids,
+									  rel2->all_partrels);
+		Assert(bms_num_members(child_relids2) == bms_num_members(rel2->relids));
+
+		/*
+		 * Get a child rel for rel2 with the relids.  See above comments.
+		 */
+		if (rel2_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids2);
+
+			child_rel2 = find_base_rel(root, varno);
+		}
+		else
+			child_rel2 = find_join_rel(root, child_relids2);
+		Assert(child_rel2);
+
+		/*
+		 * The join of rel1 and rel2 is legal, so is the join of the child
+		 * rels obtained above; add them to the given lists as a join pair
+		 * producing this join segment.
+		 */
+		*parts1 = lappend(*parts1, child_rel1);
+		*parts2 = lappend(*parts2, child_rel2);
+	}
+}
diff --git a/src/backend/optimizer/util/inherit.c b/src/backend/optimizer/util/inherit.c
index 7db67fdf34..3132fd35a5 100644
--- a/src/backend/optimizer/util/inherit.c
+++ b/src/backend/optimizer/util/inherit.c
@@ -376,6 +376,8 @@ expand_partitioned_rtentry(PlannerInfo *root, RelOptInfo *relinfo,
 		/* Create the otherrel RelOptInfo too. */
 		childrelinfo = build_simple_rel(root, childRTindex, relinfo);
 		relinfo->part_rels[i] = childrelinfo;
+		relinfo->all_partrels = bms_add_members(relinfo->all_partrels,
+												childrelinfo->relids);
 
 		/* If this child is itself partitioned, recurse */
 		if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 374f93890b..0e4944ac8e 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -240,10 +240,12 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->has_eclass_joins = false;
 	rel->consider_partitionwise_join = false;	/* might get changed later */
 	rel->part_scheme = NULL;
-	rel->nparts = 0;
+	rel->nparts = -1;
 	rel->boundinfo = NULL;
+	rel->merged = false;
 	rel->partition_qual = NIL;
 	rel->part_rels = NULL;
+	rel->all_partrels = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->partitioned_child_rels = NIL;
@@ -653,10 +655,12 @@ build_join_rel(PlannerInfo *root,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -829,10 +833,12 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -1639,7 +1645,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1652,24 +1658,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1680,18 +1668,17 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 		   !joinrel->boundinfo);
 
 	/*
-	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * If the join relation is partitioned, it use the same partitioning scheme
+	 * as the joining relations.
+	 *
+	 * Note: we calculate the partition bounds, number of partitions, and
+	 * child-join relations of the join relation in try_partitionwise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 4c47f54a57..24dbc2c8f3 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -26,6 +26,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/pathnodes.h"
 #include "parser/parse_coerce.h"
 #include "partitioning/partbounds.h"
 #include "partitioning/partdesc.h"
@@ -68,6 +69,25 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+/* Per-partitioned-relation data for merge_list_bounds()/merge_range_bounds() */
+typedef struct PartitionMap
+{
+	int			nparts;			/* number of partitions */
+	int		   *merged_indexes;	/* indexes of merged partitions */
+	bool	   *merged;			/* flags to indicate whether partitions are
+								 * merged with non-dummy partitions */
+	bool		did_remapping;	/* did we re-map partitions? */
+	int		   *old_indexes;	/* old indexes of merged partitions if
+								 * did_remapping */
+} PartitionMap;
+
+/* Macro for comparing two range bounds */
+#define compare_range_bounds(partnatts, partsupfunc, partcollations, \
+							 bound1, bound2) \
+	(partition_rbound_cmp(partnatts, partsupfunc, partcollations, \
+						  (bound1)->datums, (bound1)->kind, (bound1)->lower, \
+						  bound2))
+
 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);
@@ -79,6 +99,116 @@ static PartitionBoundInfo create_list_bounds(PartitionBoundSpec **boundspecs,
 											 int nparts, PartitionKey key, int **mapping);
 static PartitionBoundInfo create_range_bounds(PartitionBoundSpec **boundspecs,
 											  int nparts, PartitionKey key, int **mapping);
+static PartitionBoundInfo merge_list_bounds(FmgrInfo *partsupfunc,
+											Oid *collations,
+											RelOptInfo *outer_rel,
+											RelOptInfo *inner_rel,
+											JoinType jointype,
+											List **outer_parts,
+											List **inner_parts);
+static PartitionBoundInfo merge_range_bounds(int partnatts,
+											 FmgrInfo *partsupfuncs,
+											 Oid *partcollations,
+											 RelOptInfo *outer_rel,
+											 RelOptInfo *inner_rel,
+											 JoinType jointype,
+											 List **outer_parts,
+											 List **inner_parts);
+static void init_partition_map(RelOptInfo *rel, PartitionMap *map);
+static void free_partition_map(PartitionMap *map);
+static bool is_dummy_partition(RelOptInfo *rel, int part_index);
+static int merge_matching_partitions(PartitionMap *outer_map,
+									 PartitionMap *inner_map,
+									 int outer_part,
+									 int inner_part,
+									 int *next_index);
+static int process_outer_partition(PartitionMap *outer_map,
+								   PartitionMap *inner_map,
+								   bool outer_has_default,
+								   bool inner_has_default,
+								   int outer_index,
+								   int inner_default,
+								   JoinType jointype,
+								   int *next_index,
+								   int *default_index);
+static int process_inner_partition(PartitionMap *outer_map,
+								   PartitionMap *inner_map,
+								   bool outer_has_default,
+								   bool inner_has_default,
+								   int inner_index,
+								   int outer_default,
+								   JoinType jointype,
+								   int *next_index,
+								   int *default_index);
+static void merge_null_partitions(PartitionMap *outer_map,
+								  PartitionMap *inner_map,
+								  bool outer_has_null,
+								  bool inner_has_null,
+								  int outer_null,
+								  int inner_null,
+								  JoinType jointype,
+								  int *next_index,
+								  int *null_index);
+static void merge_default_partitions(PartitionMap *outer_map,
+									 PartitionMap *inner_map,
+									 bool outer_has_default,
+									 bool inner_has_default,
+									 int outer_default,
+									 int inner_default,
+									 JoinType jointype,
+									 int *next_index,
+									 int *default_index);
+static int merge_partition_with_dummy(PartitionMap *map, int index,
+									  int *next_index);
+static void fix_merged_indexes(PartitionMap *outer_map,
+							   PartitionMap *inner_map,
+							   int nmerged, List *merged_indexes);
+static void generate_matching_part_pairs(RelOptInfo *outer_rel,
+										 RelOptInfo *inner_rel,
+										 PartitionMap *outer_map,
+										 PartitionMap *inner_map,
+										 int nmerged,
+										 List **outer_parts,
+										 List **inner_parts);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+														List *merged_datums,
+														List *merged_kinds,
+														List *merged_indexes,
+														int null_index,
+														int default_index);
+static int get_range_partition(RelOptInfo *rel,
+							   PartitionBoundInfo bi,
+							   int *lb_pos,
+							   PartitionRangeBound *lb,
+							   PartitionRangeBound *ub);
+static int get_range_partition_internal(PartitionBoundInfo bi,
+										int *lb_pos,
+										PartitionRangeBound *lb,
+										PartitionRangeBound *ub);
+static bool compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+									 Oid *partcollations,
+									 PartitionRangeBound *outer_lb,
+									 PartitionRangeBound *outer_ub,
+									 PartitionRangeBound *inner_lb,
+									 PartitionRangeBound *inner_ub,
+									 int *lb_cmpval, int *ub_cmpval);
+static void get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+									Oid *partcollations, JoinType jointype,
+									PartitionRangeBound *outer_lb,
+									PartitionRangeBound *outer_ub,
+									PartitionRangeBound *inner_lb,
+									PartitionRangeBound *inner_ub,
+									int	lb_cmpval, int ub_cmpval,
+									PartitionRangeBound *merged_lb,
+									PartitionRangeBound *merged_ub);
+static void add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+									Oid *partcollations,
+									PartitionRangeBound *merged_lb,
+									PartitionRangeBound *merged_ub,
+									int merged_index,
+									List **merged_datums,
+									List **merged_kinds,
+									List **merged_indexes);
 static PartitionRangeBound *make_one_partition_rbound(PartitionKey key, int index,
 													  List *datums, bool lower);
 static int32 partition_hbound_cmp(int modulus1, int remainder1, int modulus2,
@@ -866,6 +996,1714 @@ partition_bounds_copy(PartitionBoundInfo src,
 	return dest;
 }
 
+/*
+ * partition_bounds_merge
+ *
+ * This function builds and returns the partition bounds for a join relation
+ * between input relations, creating two lists of partitions, which are
+ * returned to *outer_parts and *inner_parts respectively.  The lists contain
+ * the same number of partitions, and the partitions at the same positions in
+ * the lists indicate join pairs used for partitioned join.
+ *
+ * This function returns NULL, setting *outer_parts and *inner_parts to NIL,
+ * if a partition on one side matches multiple partitions on the other side,
+ * in which case we currently don't support partitioned join.
+ */
+PartitionBoundInfo
+partition_bounds_merge(int partnatts,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype,
+					   List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo outer_binfo = outer_rel->boundinfo;
+	PartitionBoundInfo inner_binfo = inner_rel->boundinfo;
+	char		strategy;
+
+	/*
+	 * Currently, this function is called only from try_partitionwise_join(),
+	 * so the join type should be INNER, LEFT, FULL, SEMI, or ANTI.
+	 */
+	if (jointype != JOIN_INNER && jointype != JOIN_LEFT && 
+		jointype != JOIN_FULL && jointype != JOIN_SEMI &&
+		jointype != JOIN_ANTI)
+		elog(ERROR, "unrecognized join type: %d", (int) jointype);
+
+	/* Bail out if the partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	strategy = outer_binfo->strategy;
+	*outer_parts = *inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+
+			/*
+			 * For hash partitioned tables, we currently support partitioned
+			 * join only when the partition bounds for them exactly match.
+			 *
+			 * XXX: it might be possible to relax the restriction to support
+			 * cases where hash partitioned tables have missing partitions
+			 * and/or different moduli, but it's not clear if it would be
+			 * useful to support the former case since it's unusual to have
+			 * missing partitions.  On the other hand, it would be useful to
+			 * support the latter case, but in that case, there is a high
+			 * probability that a partition on one side will match multiple
+			 * partitions on the other side, which is the scenario the current
+			 * implementation of partitioned join can't handle.
+			 */
+			return NULL;
+
+		case PARTITION_STRATEGY_LIST:
+			return merge_list_bounds(partsupfunc,
+									 partcollation,
+									 outer_rel,
+									 inner_rel,
+									 jointype,
+									 outer_parts,
+									 inner_parts);
+
+		case PARTITION_STRATEGY_RANGE:
+			return merge_range_bounds(partnatts,
+									  partsupfunc,
+									  partcollation,
+									  outer_rel,
+									  inner_rel,
+									  jointype,
+									  outer_parts,
+									  inner_parts);
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", (int) strategy);
+			return NULL;				/* keep compiler quiet */
+	}
+}
+
+/*
+ * merge_list_bounds
+ *		Create the partition bounds for a join relation between list
+ *		partitioned tables, if possible
+ */
+static PartitionBoundInfo
+merge_list_bounds(FmgrInfo *partsupfunc, Oid *partcollation,
+				  RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+				  JoinType jointype,
+				  List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			outer_pos;
+	int			inner_pos;
+	int			next_index = 0;
+	int			null_index = -1;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+	/* List partitioning doesn't require kinds. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * If the default partitions (if any) have been proven empty, deem them
+	 * non-existent.
+	 */
+	if (outer_has_default && is_dummy_partition(outer_rel, outer_default))
+		outer_has_default = false;
+	if (inner_has_default && is_dummy_partition(inner_rel, inner_default))
+		inner_has_default = false;
+
+	/*
+	 * Merge partitions from both sides.  In each iteration we compare a pair
+	 * of list values, one from each side, and decide whether the corresponding
+	 * partitions match or not.  If the two values match exactly, move to the
+	 * next pair of list values, otherwise move to the next list value on the
+	 * side with a smaller list value.
+	 */
+	outer_pos = inner_pos = 0;
+	while (outer_pos < outer_bi->ndatums || inner_pos < inner_bi->ndatums)
+	{
+		int			outer_index = -1;
+		int			inner_index = -1;
+		Datum	   *outer_datums;
+		Datum	   *inner_datums;
+		int			cmpval;
+		Datum	   *merged_datum = NULL;
+		int			merged_index = -1;
+
+		if (outer_pos < outer_bi->ndatums)
+		{
+			/*
+			 * If the partition on the outer side has been proven empty, ignore
+			 * it and move to the next datum on the outer side.
+			 */
+			outer_index = outer_bi->indexes[outer_pos];
+			if (is_dummy_partition(outer_rel, outer_index))
+			{
+				outer_pos++;
+				continue;
+			}
+		}
+		if (inner_pos < inner_bi->ndatums)
+		{
+			/*
+			 * If the partition on the inner side has been proven empty, ignore
+			 * it and move to the next datum on the inner side.
+			 */
+			inner_index = inner_bi->indexes[inner_pos];
+			if (is_dummy_partition(inner_rel, inner_index))
+			{
+				inner_pos++;
+				continue;
+			}
+		}
+
+		/* Get the list values. */
+		outer_datums = outer_pos < outer_bi->ndatums ?
+			outer_bi->datums[outer_pos] : NULL;
+		inner_datums = inner_pos < inner_bi->ndatums ?
+			inner_bi->datums[inner_pos] : NULL;
+
+		/*
+		 * We run this loop till both sides finish.  This allows us to avoid
+		 * duplicating code to handle the remaining values on the side which
+		 * finishes later.  For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra value higher than any other value on the unfinished side.
+		 * That way we advance the values on the unfinished side till all of
+		 * its values are exhausted.
+		 */
+		if (outer_pos >= outer_bi->ndatums)
+			cmpval = 1;
+		else if (inner_pos >= inner_bi->ndatums)
+			cmpval = -1;
+		else
+		{
+			Assert(outer_datums != NULL && inner_datums != NULL);
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 outer_datums[0],
+													 inner_datums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			/* Two list values match exactly. */
+			Assert(outer_pos < outer_bi->ndatums);
+			Assert(inner_pos < inner_bi->ndatums);
+			Assert(outer_index >= 0);
+			Assert(inner_index >= 0);
+
+			/*
+			 * Try merging both paritions.  If successful, add the list value
+			 * and index of the merged partition below.
+			 */
+			merged_index = merge_matching_partitions(&outer_map, &inner_map,
+													 outer_index, inner_index,
+													 &next_index);
+			if (merged_index == -1)
+				goto cleanup;
+
+			merged_datum = outer_datums;
+
+			/* Move to the next pair of list values. */
+			outer_pos++;
+			inner_pos++;
+		}
+		else if (cmpval < 0)
+		{
+			/* A list value missing from the inner side. */
+			Assert(outer_pos < outer_bi->ndatums);
+
+			/*
+			 * If the inner side has the default partition, or this is an outer
+			 * join, try to assign a merged partition to the outer partition
+			 * (see process_outer_partition()).  Otherwise, the outer partition
+			 * will not contribute to the result.
+			 */
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				/* Get the outer partition. */
+				outer_index = outer_bi->indexes[outer_pos];
+				Assert(outer_index >= 0);
+				merged_index = process_outer_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   outer_index,
+													   inner_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+				merged_datum = outer_datums;
+			}
+
+			/* Move to the next list value on the outer side. */
+			outer_pos++;
+		}
+		else
+		{
+			/* A list value missing from the outer side. */
+			Assert(cmpval > 0);
+			Assert(inner_pos < inner_bi->ndatums);
+
+			/*
+			 * If the outer side has the default partition, or this is a FULL
+			 * join, try to assign a merged partition to the inner partition
+			 * (see process_inner_partition()).  Otherwise, the inner partition
+			 * will not contribute to the result.
+			 */
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				/* Get the inner partition. */
+				inner_index = inner_bi->indexes[inner_pos];
+				Assert(inner_index >= 0);
+				merged_index = process_inner_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   inner_index,
+													   outer_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+				merged_datum = inner_datums;
+			}
+
+			/* Move to the next list value on the inner side. */
+			inner_pos++;
+		}
+
+		/*
+		 * If we assigned a merged partition, add the list value and index of
+		 * the merged partition if appropriate.
+		 */
+		if (merged_index >= 0 && merged_index != default_index)
+		{
+			merged_datums = lappend(merged_datums, merged_datum);
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+		}
+	}
+
+	/*
+	 * If the NULL partitions (if any) have been proven empty, deem them
+	 * non-existent.
+	 */
+	if (outer_has_null &&
+		is_dummy_partition(outer_rel, outer_bi->null_index))
+		outer_has_null = false;
+	if (inner_has_null &&
+		is_dummy_partition(inner_rel, inner_bi->null_index))
+		inner_has_null = false;
+
+	/* Merge the NULL partitions if any. */
+	if (outer_has_null || inner_has_null)
+		merge_null_partitions(&outer_map, &inner_map,
+							  outer_has_null, inner_has_null,
+							  outer_bi->null_index, inner_bi->null_index,
+							  jointype, &next_index, &null_index);
+	else
+		Assert(null_index == -1);
+
+	/* Merge the default partitions if any. */
+	if (outer_has_default || inner_has_default)
+		merge_default_partitions(&outer_map, &inner_map,
+								 outer_has_default, inner_has_default,
+								 outer_default, inner_default,
+								 jointype, &next_index, &default_index);
+	else
+		Assert(default_index == -1);
+
+	/* If we have merged partitions, create the partition bounds. */
+	if (next_index > 0)
+	{
+		/* Fix the merged_indexes list if necessary. */
+		if (outer_map.did_remapping || inner_map.did_remapping)
+		{
+			Assert(jointype == JOIN_FULL);
+			fix_merged_indexes(&outer_map, &inner_map,
+							   next_index, merged_indexes);
+		}
+
+		/* Use maps to match partitions from inputs. */
+		generate_matching_part_pairs(outer_rel, inner_rel,
+									 &outer_map, &inner_map,
+									 next_index,
+									 outer_parts, inner_parts);
+		Assert(*outer_parts != NIL);
+		Assert(*inner_parts != NIL);
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) <= next_index);
+
+		/* Make a PartitionBoundInfo struct to return. */
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  NIL,
+													  merged_indexes,
+													  null_index,
+													  default_index);
+		Assert(merged_bounds);
+	}
+
+cleanup:
+	/* Free local memory before returning. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+
+	return merged_bounds;
+}
+
+/*
+ * merge_range_bounds
+ *		Create the partition bounds for a join relation between range
+ *		partitioned tables, if possible
+ */
+static PartitionBoundInfo
+merge_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+				   Oid *partcollations,
+				   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+				   JoinType jointype,
+				   List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			outer_index;
+	int			inner_index;
+	int			outer_lb_pos;
+	int			inner_lb_pos;
+	PartitionRangeBound outer_lb;
+	PartitionRangeBound outer_ub;
+	PartitionRangeBound inner_lb;
+	PartitionRangeBound inner_ub;
+	int			next_index = 0;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_kinds = NIL;
+	List	   *merged_indexes = NIL;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * If the default partitions (if any) have been proven empty, deem them
+	 * non-existent.
+	 */
+	if (outer_has_default && is_dummy_partition(outer_rel, outer_default))
+		outer_has_default = false;
+	if (inner_has_default && is_dummy_partition(inner_rel, inner_default))
+		inner_has_default = false;
+
+	/*
+	 * Merge partitions from both sides.  In each iteration we compare a pair
+	 * of ranges, one from each side, and decide whether the corresponding
+	 * partitions match or not.  If the two ranges overlap, move to the next
+	 * pair of ranges, otherwise move to the next range on the side with a
+	 * lower range.  outer_lb_pos/inner_lb_pos keep track of the positions of
+	 * lower bounds in the datums arrays in the outer/inner PartitionBoundInfos
+	 * respectively.
+	 */
+	outer_lb_pos = inner_lb_pos = 0;
+	outer_index = get_range_partition(outer_rel, outer_bi, &outer_lb_pos,
+									  &outer_lb, &outer_ub);
+	inner_index = get_range_partition(inner_rel, inner_bi, &inner_lb_pos,
+									  &inner_lb, &inner_ub);
+	while (outer_index >= 0 || inner_index >= 0)
+	{
+		bool		overlap;
+		int			ub_cmpval;
+		int			lb_cmpval;
+		PartitionRangeBound merged_lb = {-1, NULL, NULL, true};
+		PartitionRangeBound merged_ub = {-1, NULL, NULL, false};
+		int			merged_index = -1;
+
+		/*
+		 * We run this loop till both sides finish.  This allows us to avoid
+		 * duplicating code to handle the remaining ranges on the side which
+		 * finishes later.  For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra range higher than any other range on the unfinished side.
+		 * That way we advance the ranges on the unfinished side till all of
+		 * its ranges are exhausted.
+		 */
+		if (outer_index == -1)
+		{
+			overlap = false;
+			lb_cmpval = 1;
+			ub_cmpval = 1;
+		}
+		else if (inner_index == -1)
+		{
+			overlap = false;
+			lb_cmpval = -1;
+			ub_cmpval = -1;
+		}
+		else
+			overlap = compare_range_partitions(partnatts, partsupfuncs,
+											   partcollations,
+											   &outer_lb, &outer_ub,
+											   &inner_lb, &inner_ub,
+											   &lb_cmpval, &ub_cmpval);
+
+		if (overlap)
+		{
+			/* Two ranges overlap; form a join pair. */
+
+			PartitionRangeBound save_outer_ub;
+			PartitionRangeBound save_inner_ub;
+
+			/* Both partitions should not have been merged yet. */
+			Assert(outer_index >= 0);
+			Assert(outer_map.merged_indexes[outer_index] == -1 &&
+				   outer_map.merged[outer_index] == false);
+			Assert(inner_index >= 0);
+			Assert(inner_map.merged_indexes[inner_index] == -1 &&
+				   inner_map.merged[inner_index] == false);
+
+			/*
+			 * Get the index of the merged partition.  Both partitions aren't
+			 * merged yet, so the partitions should be merged successfully.
+			 */
+			merged_index = merge_matching_partitions(&outer_map, &inner_map,
+													 outer_index, inner_index,
+													 &next_index);
+			Assert(merged_index >= 0);
+
+			/* Get the range of the merged partition. */
+			get_merged_range_bounds(partnatts, partsupfuncs,
+									partcollations, jointype,
+									&outer_lb, &outer_ub,
+									&inner_lb, &inner_ub,
+									lb_cmpval, ub_cmpval,
+									&merged_lb, &merged_ub);
+
+			/* Save the upper bounds of both partitions for use below. */
+			save_outer_ub = outer_ub;
+			save_inner_ub = inner_ub;
+
+			/* Move to the next pair of ranges. */
+			outer_index = get_range_partition(outer_rel, outer_bi, &outer_lb_pos,
+											  &outer_lb, &outer_ub);
+			inner_index = get_range_partition(inner_rel, inner_bi, &inner_lb_pos,
+											  &inner_lb, &inner_ub);
+
+			/*
+			 * If the range of a partition on one side overlaps the range of
+			 * the next partition on the other side, that will cause the
+			 * partition on one side to match at least two partitions on the
+			 * other side, which is the case that we currently don't support
+			 * partitioned join for; give up.
+			 */
+			if (ub_cmpval > 0 && inner_index >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &save_outer_ub, &inner_lb) > 0)
+				goto cleanup;
+			if (ub_cmpval < 0 && outer_index >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &outer_lb, &save_inner_ub) < 0)
+				goto cleanup;
+
+			/*
+			 * A row from a non-overlapping portion (if any) of a partition
+			 * on one side might find its join partner in the default
+			 * partition (if any) on the other side, causing the same
+			 * situation as above; give up in that case.
+			 */
+			if ((outer_has_default && (lb_cmpval > 0 || ub_cmpval < 0)) ||
+				(inner_has_default && (lb_cmpval < 0 || ub_cmpval > 0)))
+				goto cleanup;
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* A non-overlapping outer range. */
+
+			/* The outer partition should not have been merged yet. */
+			Assert(outer_index >= 0);
+			Assert(outer_map.merged_indexes[outer_index] == -1 &&
+				   outer_map.merged[outer_index] == false);
+
+			/*
+			 * If the inner side has the default partition, or this is an outer
+			 * join, try to assign a merged partition to the outer partition
+			 * (see process_outer_partition()).  Otherwise, the outer partition
+			 * will not contribute to the result.
+			 */
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				merged_index = process_outer_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   outer_index,
+													   inner_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+				merged_lb = outer_lb;
+				merged_ub = outer_ub;
+			}
+
+			/* Move to the next range on the outer side. */
+			outer_index = get_range_partition(outer_rel, outer_bi, &outer_lb_pos,
+											  &outer_lb, &outer_ub);
+		}
+		else
+		{
+			/* A non-overlapping inner range. */
+			Assert(ub_cmpval > 0);
+
+			/* The inner partition should not have been merged yet. */
+			Assert(inner_index >= 0);
+			Assert(inner_map.merged_indexes[inner_index] == -1 &&
+				   inner_map.merged[inner_index] == false);
+
+			/*
+			 * If the outer side has the default partition, or this is a FULL
+			 * join, try to assign a merged partition to the inner partition
+			 * (see process_inner_partition()).  Otherwise, the inner partition
+			 * will not contribute to the result.
+			 */
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				merged_index = process_inner_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   inner_index,
+													   outer_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+				merged_lb = inner_lb;
+				merged_ub = inner_ub;
+			}
+
+			/* Move to the next range on the inner side. */
+			inner_index = get_range_partition(inner_rel, inner_bi, &inner_lb_pos,
+											  &inner_lb, &inner_ub);
+		}
+
+		/*
+		 * If we assigned a merged partition, add the range bounds and index of
+		 * the merged partition if appropriate.
+		 */
+		if (merged_index >= 0 && merged_index != default_index)
+			add_merged_range_bounds(partnatts, partsupfuncs, partcollations,
+									&merged_lb, &merged_ub, merged_index,
+									&merged_datums, &merged_kinds,
+									&merged_indexes);
+	}
+
+	/* Merge the default partitions if any. */
+	if (outer_has_default || inner_has_default)
+		merge_default_partitions(&outer_map, &inner_map,
+								 outer_has_default, inner_has_default,
+								 outer_default, inner_default,
+								 jointype, &next_index, &default_index);
+	else
+		Assert(default_index == -1);
+
+	/* If we have merged partitions, create the partition bounds. */
+	if (next_index > 0)
+	{
+		/*
+		 * Unlike the case of list partitioning, we wouldn't have re-merged
+		 * partitions, so did_remapping should be left alone.
+		 */
+		Assert(!outer_map.did_remapping);
+		Assert(!inner_map.did_remapping);
+
+		/* Use maps to match partitions from inputs. */
+		generate_matching_part_pairs(outer_rel, inner_rel,
+									 &outer_map, &inner_map,
+									 next_index,
+									 outer_parts, inner_parts);
+		Assert(*outer_parts != NIL);
+		Assert(*inner_parts != NIL);
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+
+		/* Make a PartitionBoundInfo struct to return. */
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_kinds,
+													  merged_indexes,
+													  -1,
+													  default_index);
+		Assert(merged_bounds);
+	}
+
+cleanup:
+	/* Free local memory before returning. */
+	list_free(merged_datums);
+	list_free(merged_kinds);
+	list_free(merged_indexes);
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+
+	return merged_bounds;
+}
+
+/*
+ * init_partition_map
+ *		Initialize a PartitionMap struct for given relation
+ */
+static void
+init_partition_map(RelOptInfo *rel, PartitionMap *map)
+{
+	int			nparts = rel->nparts;
+	int			i;
+
+	map->nparts = nparts;
+	map->merged_indexes = (int *) palloc(sizeof(int) * nparts);
+	map->merged = (bool *) palloc(sizeof(bool) * nparts);
+	map->did_remapping = false;
+	map->old_indexes = (int *) palloc(sizeof(int) * nparts);
+	for (i = 0; i < nparts; i++)
+	{
+		map->merged_indexes[i] = map->old_indexes[i] = -1;
+		map->merged[i] = false;
+	}
+}
+
+/*
+ * free_partition_map
+ */
+static void
+free_partition_map(PartitionMap *map)
+{
+	pfree(map->merged_indexes);
+	pfree(map->merged);
+	pfree(map->old_indexes);
+}
+
+/*
+ * is_dummy_partition --- has partition been proven empty?
+ */
+static bool
+is_dummy_partition(RelOptInfo *rel, int part_index)
+{
+	RelOptInfo *part_rel;
+
+	Assert(part_index >= 0);
+	part_rel = rel->part_rels[part_index];
+	if (part_rel == NULL || IS_DUMMY_REL(part_rel))
+		return true;
+	return false;
+}
+
+/*
+ * merge_matching_partitions
+ *		Try to merge given outer/inner partitions, and return the index of a
+ *		merged partition produced from them if successful, -1 otherwise
+ *
+ * If the merged partition is newly created, *next_index is incremented.
+ */
+static int
+merge_matching_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						  int outer_index, int inner_index, int *next_index)
+{
+	int 		outer_merged_index;
+	int 		inner_merged_index;
+	bool 		outer_merged;
+	bool 		inner_merged;
+
+	Assert(outer_index >= 0 && outer_index < outer_map->nparts);
+	outer_merged_index = outer_map->merged_indexes[outer_index];
+	outer_merged = outer_map->merged[outer_index];
+	Assert(inner_index >= 0 && inner_index < inner_map->nparts);
+	inner_merged_index = inner_map->merged_indexes[inner_index];
+	inner_merged = inner_map->merged[inner_index];
+
+	/*
+	 * Handle cases where we have already assigned a merged partition to each
+	 * of the given partitions.
+	 */
+	if (outer_merged_index >= 0 && inner_merged_index >= 0)
+	{
+		/*
+		 * If the mereged partitions are the same, no need to do anything;
+		 * return the index of the merged partitions.  Otherwise, if each of
+		 * the given partitions has been merged with a dummy partition on the
+		 * other side, re-map them to either of the two merged partitions.
+		 * Otherwise, they can't be merged, so return -1.
+		 */
+		if (outer_merged_index == inner_merged_index)
+		{
+			Assert(outer_merged);
+			Assert(inner_merged);
+			return outer_merged_index;
+		}
+		if (!outer_merged && !inner_merged)
+		{
+			/*
+			 * This can only happen for a list-partitioning case.  We re-map
+			 * them to the merged partition with the smaller of the two merged
+			 * indexes to preserve the property that the canonical order of
+			 * list partitions is determined by the indexes assigned to the
+			 * smallest list value of each partition.
+			 */
+			if (outer_merged_index < inner_merged_index)
+			{
+				outer_map->merged[outer_index] = true;
+				inner_map->merged_indexes[inner_index] = outer_merged_index;
+				inner_map->merged[inner_index] = true;
+				inner_map->did_remapping = true;
+				inner_map->old_indexes[inner_index] = inner_merged_index;
+				return outer_merged_index;
+			}
+			else
+			{
+				inner_map->merged[inner_index] = true;
+				outer_map->merged_indexes[outer_index] = inner_merged_index;
+				outer_map->merged[outer_index] = true;
+				outer_map->did_remapping = true;
+				outer_map->old_indexes[outer_index] = outer_merged_index;
+				return inner_merged_index;
+			}
+		}
+		return -1;
+	}
+
+	/* At least one of the given partitions should not have yet been merged. */
+	Assert(outer_merged_index == -1 || inner_merged_index == -1);
+
+	/*
+	 * If neither of them has been merged, merge them.  Otherwise, if one has
+	 * been merged with a dummy relation on the other side (and the other
+	 * hasn't yet been merged with anything), re-merge them.  Otherwise, they
+	 * can't be merged, so return -1.
+	 */
+	if (outer_merged_index == -1 && inner_merged_index == -1)
+	{
+		int 		merged_index = *next_index;
+
+		Assert(!outer_merged);
+		Assert(!inner_merged);
+		outer_map->merged_indexes[outer_index] = merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged_indexes[inner_index] = merged_index;
+		inner_map->merged[inner_index] = true;
+		*next_index = *next_index + 1;
+		return merged_index;
+	}
+	if (outer_merged_index >= 0 && !outer_map->merged[outer_index])
+	{
+		Assert(inner_merged_index == -1);
+		Assert(!inner_merged);
+		inner_map->merged_indexes[inner_index] = outer_merged_index;
+		inner_map->merged[inner_index] = true;
+		outer_map->merged[outer_index] = true;
+		return outer_merged_index;
+	}
+	if (inner_merged_index >= 0 && !inner_map->merged[inner_index])
+	{
+		Assert(outer_merged_index == -1);
+		Assert(!outer_merged);
+		outer_map->merged_indexes[outer_index] = inner_merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged[inner_index] = true;
+		return inner_merged_index;
+	}
+	return -1;
+}
+
+/*
+ * process_outer_partition
+ *		Try to assign given outer partition a merged partition, and return the
+ *		index of the merged partition if successful, -1 otherwise
+ *
+ * If the partition is newly created, *next_index is incremented.  Also, if it
+ * is the default partition of the join relation, *default_partition is set to
+ * the index if not already done.
+ */
+static int
+process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index)
+{
+	int 		merged_index = -1;
+
+	Assert(outer_index >= 0);
+
+	/*
+	 * If the inner side has the default partition, a row from the outer
+	 * partition might find its join partner in the default partition; try
+	 * merging the outer partition with the default partition.  Otherwise, this
+	 * should be an outer join, in which case the outer partition has to be
+	 * scanned all the way anyway; merge the outer partition with a dummy
+	 * partition on the other side.
+	 */
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0);
+
+		/*
+		 * If the outer side has the default partition as well, the default
+		 * partition on the inner side will have two matching partitions on the
+		 * other side: the outer partition and the default partition on the
+		 * outer side.  Partitionwise join doesn't handle this scenario yet.
+		 */
+		if (outer_has_default)
+			return -1;
+
+		merged_index = merge_matching_partitions(outer_map, inner_map,
+												 outer_index, inner_default,
+												 next_index);
+		if (merged_index == -1)
+			return -1;
+
+		/*
+		 * If this is a FULL join, the default partition on the inner side
+		 * has to be scanned all the way anyway, so the resulting partition
+		 * will contain all key values from the default partition, which any
+		 * other partition of the join relation will not contain.  Thus the
+		 * resutling partition will act as the default partition of the join
+		 * relation; record the index in *default_index if not already done.
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (*default_index == -1)
+				*default_index = merged_index;
+			else
+				Assert(*default_index == merged_index);
+		}
+	}
+	else
+	{
+		Assert(IS_OUTER_JOIN(jointype));
+		Assert(jointype != JOIN_RIGHT);
+
+		/* If we have already assigned a partition, no need to do anything. */
+		merged_index = outer_map->merged_indexes[outer_index];
+		if (merged_index == -1)
+			merged_index = merge_partition_with_dummy(outer_map, outer_index,
+													  next_index);
+	}
+	return merged_index;
+}
+
+/*
+ * process_inner_partition
+ *		Try to assign given inner partition a merged partition, and return the
+ *		index of the merged partition if successful, -1 otherwise
+ *
+ * If the partition is newly created, *next_index is incremented.  Also, if it
+ * is the default partition of the join relation, *default_partition is set to
+ * the index if not already done.
+ */
+static int
+process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index)
+{
+	int 		merged_index = -1;
+
+	Assert(inner_index >= 0);
+
+	/*
+	 * If the outer side has the default partition, a row from the inner
+	 * partition might find its join partner in the default partition; try
+	 * merging the inner partition with the default partition.  Otherwise, this
+	 * should be a FULL join, in which case the inner partition has to be
+	 * scanned all the way anyway; merge the inner partition with a dummy
+	 * partition on the other side.
+	 */
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0);
+
+		/*
+		 * If the inner side has the default partition as well, the default
+		 * partition on the outer side will have two matching partitions on the
+		 * other side: the inner partition and the default partition on the
+		 * inner side.  Partitionwise join doesn't handle this scenario yet.
+		 */
+		if (inner_has_default)
+			return -1;
+
+		merged_index = merge_matching_partitions(outer_map, inner_map,
+												 outer_default, inner_index,
+												 next_index);
+		if (merged_index == -1)
+			return -1;
+
+		/*
+		 * If this is an outer join, the default partition on the outer side
+		 * has to be scanned all the way anyway, so the resulting partition
+		 * will contain all key values from the default partition, which any
+		 * other partition of the join relation will not contain.  Thus the
+		 * resutling partition will act as the default partition of the join
+		 * relation; record the index in *default_index if not already done.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			if (*default_index == -1)
+				*default_index = merged_index;
+			else
+				Assert(*default_index == merged_index);
+		}
+	}
+	else
+	{
+		Assert(jointype == JOIN_FULL);
+
+		/* If we have already assigned a partition, no need to do anything. */
+		merged_index = inner_map->merged_indexes[inner_index];
+		if (merged_index == -1)
+			merged_index = merge_partition_with_dummy(inner_map, inner_index,
+													  next_index);
+	}
+	return merged_index;
+}
+
+/*
+ * merge_null_partitions
+ *		Merge the NULL partitions from a join's outer and inner sides.
+ *
+ * If the merged partition produced from them is the NULL partition of the join
+ * relation, *null_index is set to the index of the merged partition.
+ *
+ * Note: We assume here that the join clause for a partitioned join is strict
+ * because have_partkey_equi_join() requires that the corresponding operator
+ * be mergejoinable, and we currently assume that mergejoinable operators are
+ * strict (see MJEvalOuterValues()/MJEvalInnerValues()).
+ */
+static void
+merge_null_partitions(PartitionMap *outer_map,
+					  PartitionMap *inner_map,
+					  bool outer_has_null,
+					  bool inner_has_null,
+					  int outer_null,
+					  int inner_null,
+					  JoinType jointype,
+					  int *next_index,
+					  int *null_index)
+{
+	bool 		consider_outer_null = false;
+	bool 		consider_inner_null = false;
+
+	Assert(outer_has_null || inner_has_null);
+	Assert(*null_index == -1);
+
+	/*
+	 * Check whether the NULL partitions have already been merged and if so,
+	 * set the consider_outer_null/consider_inner_null flags.
+	 */
+	if (outer_has_null)
+	{
+		Assert(outer_null >= 0 && outer_null < outer_map->nparts);
+		if (outer_map->merged_indexes[outer_null] == -1)
+		 	consider_outer_null = true;
+	}
+	if (inner_has_null)
+	{
+		Assert(inner_null >= 0 && inner_null < inner_map->nparts);
+		if (inner_map->merged_indexes[inner_null] == -1)
+		 	consider_inner_null = true;
+	}
+
+	/* If both flags are set false, we don't need to do anything. */
+	if (!consider_outer_null && !consider_inner_null)
+		return;
+
+	if (consider_outer_null && !consider_inner_null)
+	{
+		Assert(outer_has_null);
+
+		/*
+		 * If this is an outer join, the NULL partition on the outer side has
+		 * to be scanned all the way anyway; merge the NULL partition with a
+		 * dummy partition on the other side.  In that case consider_outer_null
+		 * means that the NULL partition only contains NULL values as the key
+		 * values, so the merged partition will do so; treat it as the NULL
+		 * partition of the join relation.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			*null_index = merge_partition_with_dummy(outer_map, outer_null,
+													 next_index);
+		}
+	}
+	else if (!consider_outer_null && consider_inner_null)
+	{
+		Assert(inner_has_null);
+
+		/*
+		 * If this is a FULL join, the NULL partition on the inner side has
+		 * to be scanned all the way anyway; merge the NULL partition with a
+		 * dummy partition on the other side.  In that case consider_inner_null
+		 * means that the NULL partition only contains NULL values as the key
+		 * values, so the merged partition will do so; treat it as the NULL
+		 * partition of the join relation.
+		 */
+		if (jointype == JOIN_FULL)
+			*null_index = merge_partition_with_dummy(inner_map, inner_null,
+													 next_index);
+	}
+	else
+	{
+		Assert(consider_outer_null && consider_inner_null);
+		Assert(outer_has_null);
+		Assert(inner_has_null);
+
+		/*
+		 * If this is an outer join, the NULL partition on the outer side (and
+		 * that on the inner side if this is a FULL join) have to be scanned
+		 * all the way anyway, so merge them.  Note that each of the NULL
+		 * partitions isn't merged yet, so they should be merged successfully.
+		 * Like the above, each of the NULL partitions only contains NULL
+		 * values as the key values, so the merged partition will do so; treat
+		 * it as the NULL partition of the join relation.
+		 *
+		 * Note: if this an INNER/SEMI join, the join clause will never be
+		 * satisfied by two NULL values (see comments above), so both the NULL
+		 * partitions can be eliminated.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			*null_index = merge_matching_partitions(outer_map, inner_map,
+													outer_null, inner_null,
+													next_index);
+			Assert(*null_index >= 0);
+		}
+	}
+}
+
+/*
+ * merge_default_partitions
+ *		Merge the default partitions from a join's outer and inner sides.
+ *
+ * If the merged partition produced from them is the default partition of the
+ * join relation, *default_index is set to the index of the merged partition.
+ */
+static void
+merge_default_partitions(PartitionMap *outer_map,
+						 PartitionMap *inner_map,
+						 bool outer_has_default,
+						 bool inner_has_default,
+						 int outer_default,
+						 int inner_default,
+						 JoinType jointype,
+						 int *next_index,
+						 int *default_index)
+{
+	int 		outer_merged_index = -1;
+	int 		inner_merged_index = -1;
+
+	Assert(outer_has_default || inner_has_default);
+
+	/* Get the merged partition indexes for the default partitions. */
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0 && outer_default < outer_map->nparts);
+		outer_merged_index = outer_map->merged_indexes[outer_default];
+	}
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0 && inner_default < inner_map->nparts);
+		inner_merged_index = inner_map->merged_indexes[inner_default];
+	}
+
+	if (outer_has_default && !inner_has_default)
+	{
+		/*
+		 * If this is an outer join, the default partition on the outer side
+		 * has to be scanned all the way anyway; if we have not yet assigned a
+		 * partition, merge the default partition with a dummy partition on the
+		 * other side.  The merged partition will act as the default partition
+		 * of the join relation (see comments in process_inner_partition()).
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			if (outer_merged_index == -1)
+			{
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(outer_map,
+															outer_default,
+															next_index);
+			}
+			else
+				Assert(*default_index == outer_merged_index);
+		}
+		else
+			Assert(*default_index == -1);
+	}
+	else if (!outer_has_default && inner_has_default)
+	{
+		/*
+		 * If this is a FULL join, the default partition on the inner side
+		 * has to be scanned all the way anyway; if we have not yet assigned a
+		 * partition, merge the default partition with a dummy partition on the
+		 * other side.  The merged partition will act as the default partition
+		 * of the join relation (see comments in process_outer_partition()).
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (inner_merged_index == -1)
+			{
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(inner_map,
+															inner_default,
+															next_index);
+			}
+			else
+				Assert(*default_index == inner_merged_index);
+		}
+		else
+			Assert(*default_index == -1);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		/*
+		 * The default partitions have to be joined with each other, so merge
+		 * them.  Note that each of the default partitions isn't merged yet
+		 * (see, process_outer_partition()/process_innerer_partition()), so
+		 * they should be merged successfully.  The merged partition will act
+		 * as the default partition of the join relation.
+		 */
+		Assert(outer_merged_index == -1);
+		Assert(inner_merged_index == -1);
+		Assert(*default_index == -1);
+		*default_index = merge_matching_partitions(outer_map,
+												   inner_map,
+												   outer_default,
+												   inner_default,
+												   next_index);
+		Assert(*default_index >= 0);
+	}
+}
+
+/*
+ * merge_partition_with_dummy
+ *		Assign given partition a new partition of a join relation
+ *
+ * Note: The caller assumes that the given partition doesn't have a non-dummy
+ * matching partition on the other side, but if the given partition finds the
+ * matchig partition later, we will adjust the assignment.
+ */
+static int
+merge_partition_with_dummy(PartitionMap *map, int index, int *next_index)
+{
+	int 		merged_index = *next_index;
+
+	Assert(index >= 0 && index < map->nparts);
+	Assert(map->merged_indexes[index] == -1);
+	Assert(!map->merged[index]);
+	map->merged_indexes[index] = merged_index;
+	/* Leave the merged flag alone! */
+	*next_index = *next_index + 1;
+	return merged_index;
+}
+
+/*
+ * fix_merged_indexes
+ *		Adjust merged indexes of re-merged partitions
+ */
+static void
+fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes)
+{
+	int		   *new_indexes;
+	int			merged_index;
+	int			i;
+	ListCell   *lc;
+
+	Assert(nmerged > 0);
+
+	new_indexes = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		new_indexes[i] = -1;
+
+	/* Build the mapping of old merged indexes to new merged indexes. */
+	if (outer_map->did_remapping)
+	{
+		for (i = 0; i < outer_map->nparts; i++)
+		{
+			merged_index = outer_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = outer_map->merged_indexes[i];
+		}
+	}
+	if (inner_map->did_remapping)
+	{
+		for (i = 0; i < inner_map->nparts; i++)
+		{
+			merged_index = inner_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = inner_map->merged_indexes[i];
+		}
+	}
+
+	/* Fix the merged_indexes list using the mapping. */
+	foreach(lc, merged_indexes)
+	{
+		merged_index = lfirst_int(lc);
+		Assert(merged_index >= 0);
+		if (new_indexes[merged_index] >= 0)
+			lfirst_int(lc) = new_indexes[merged_index];
+	}
+
+	pfree(new_indexes);
+}
+
+/*
+ * generate_matching_part_pairs
+ *		Generate a pair of lists of partitions that produce merged partitions
+ *
+ * The lists of partitions are built in the order of merged partition indexes,
+ * and returned in *outer_parts and *inner_parts.
+ */
+static void
+generate_matching_part_pairs(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 PartitionMap *outer_map, PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_parts, List **inner_parts)
+{
+	int			outer_nparts = outer_map->nparts;
+	int			inner_nparts = inner_map->nparts;
+	int		   *outer_indexes;
+	int		   *inner_indexes;
+	int			max_nparts;
+	int 		i;
+
+	Assert(nmerged > 0);
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	outer_indexes = (int *) palloc(sizeof(int) * nmerged);
+	inner_indexes = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		outer_indexes[i] = inner_indexes[i] = -1;
+
+	/* Set pairs of matching partitions. */
+	Assert(outer_nparts == outer_rel->nparts);
+	Assert(inner_nparts == inner_rel->nparts);
+	max_nparts = Max(outer_nparts, inner_nparts);
+	for (i = 0; i < max_nparts; i++)
+	{
+		if (i < outer_nparts)
+		{
+			int 		merged_index = outer_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				outer_indexes[merged_index] = i;
+			}
+		}
+		if (i < inner_nparts)
+		{
+			int 		merged_index = inner_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				inner_indexes[merged_index] = i;
+			}
+		}
+	}
+
+	/* Build the list pairs. */
+	for (i = 0; i < nmerged; i++)
+	{
+		int			outer_index = outer_indexes[i];
+		int			inner_index = inner_indexes[i];
+
+		/*
+		 * If both partitions are dummy, it means the merged partition that had
+		 * been assigned to the outer/inner partition was removed when
+		 * re-merging the outer/inner partition in merge_matching_partitions();
+		 * ignore the merged partition.
+		 */
+		if (outer_index == -1 && inner_index == -1)
+			continue;
+
+		*outer_parts = lappend(*outer_parts, outer_index >= 0 ?
+							   outer_rel->part_rels[outer_index] : NULL);
+		*inner_parts = lappend(*inner_parts, inner_index >= 0 ?
+							   inner_rel->part_rels[inner_index] : NULL);
+	}
+
+	pfree(outer_indexes);
+	pfree(inner_indexes);
+}
+
+/*
+ * build_merged_partition_bounds
+ *		Create a PartitionBoundInfo struct from merged partition bounds
+ */
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_kinds, List *merged_indexes,
+							  int null_index, int default_index)
+{
+	PartitionBoundInfo merged_bounds;
+	int			ndatums = list_length(merged_datums);
+	int			pos;
+	ListCell   *lc;
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = ndatums;
+
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) * ndatums);
+	pos = 0;
+	foreach(lc, merged_datums)
+		merged_bounds->datums[pos++] = (Datum *) lfirst(lc);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_kinds) == ndatums);
+		merged_bounds->kind = (PartitionRangeDatumKind **)
+			palloc(sizeof(PartitionRangeDatumKind *) * ndatums);
+		pos = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[pos++] = (PartitionRangeDatumKind *) lfirst(lc);
+
+		/* There are ndatums+1 indexes in the case of range partitioning. */
+		merged_indexes = lappend_int(merged_indexes, -1);
+		ndatums++;
+	}
+	else
+	{
+		Assert(strategy == PARTITION_STRATEGY_LIST);
+		Assert(merged_kinds == NIL);
+		merged_bounds->kind = NULL;
+	}
+
+	Assert(list_length(merged_indexes) == ndatums);
+	merged_bounds->indexes = (int *) palloc(sizeof(int) * ndatums);
+	pos = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[pos++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * get_range_partition
+ *		Get the next non-dummy partition of a range-partitioned relation,
+ *		returning the index of that partition
+ *
+ * *lb and *ub are set to the lower and upper bounds of that partition
+ * respectively, and *lb_pos is advanced to the next lower bound, if any.
+ */
+static int
+get_range_partition(RelOptInfo *rel,
+					PartitionBoundInfo bi,
+					int *lb_pos,
+					PartitionRangeBound *lb,
+					PartitionRangeBound *ub)
+{
+	int			part_index;
+
+	Assert(bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	do {
+		part_index = get_range_partition_internal(bi, lb_pos, lb, ub);
+		if (part_index == -1)
+			return -1;
+	} while (is_dummy_partition(rel, part_index));
+
+	return part_index;
+}
+
+static int
+get_range_partition_internal(PartitionBoundInfo bi,
+							 int *lb_pos,
+							 PartitionRangeBound *lb,
+							 PartitionRangeBound *ub)
+{
+	/* Return the index as -1 if we've exhausted all lower bounds. */
+	if (*lb_pos >= bi->ndatums)
+		return -1;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(*lb_pos + 1 < bi->ndatums);
+
+	/* Set the lower bound. */
+	lb->index = bi->indexes[*lb_pos];
+	lb->datums = bi->datums[*lb_pos];
+	lb->kind = bi->kind[*lb_pos];
+	lb->lower = true;
+	/* Set the upper bound. */
+	ub->index = bi->indexes[*lb_pos + 1];
+	ub->datums = bi->datums[*lb_pos + 1];
+	ub->kind = bi->kind[*lb_pos + 1];
+	ub->lower = false;
+
+	/* The index assigned to an upper bound should be valid. */
+	Assert(ub->index >= 0);
+
+	/*
+	 * Advance the position to the next lower bound.  If there are no bounds
+	 * left beyond the upper bound, we have reached the last lower bound.
+	 */
+	if (*lb_pos + 2 >= bi->ndatums)
+		*lb_pos = bi->ndatums;
+	else
+	{
+		/*
+		 * If the index assigned to the bound next to the upper bound isn't
+		 * valid, that is the next lower bound; else, the upper bound is also
+		 * the lower bound of the next range partition.
+		 */
+		if (bi->indexes[*lb_pos + 2] < 0)
+			*lb_pos = *lb_pos + 2;
+		else
+			*lb_pos = *lb_pos + 1;
+	}
+
+	return ub->index;
+}
+
+/*
+ * compare_range_partitions
+ *		Compare the bounds of two range partitions, and return true if the
+ *		two partitions overlap, false otherwise
+ *
+ * *lb_cmpval is set to -1, 0, or 1 if the outer partition's lower bound is
+ * lower than, equal to, or higher than the inner partition's lower bound
+ * respectively.  Likewise, *ub_cmpval is set to -1, 0, or 1 if the outer
+ * partition's upper bound is lower than, equal to, or higher than the inner
+ * partition's upper bound respectively.
+ */
+static bool
+compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+						 Oid *partcollations,
+						 PartitionRangeBound *outer_lb,
+						 PartitionRangeBound *outer_ub,
+						 PartitionRangeBound *inner_lb,
+						 PartitionRangeBound *inner_ub,
+						 int *lb_cmpval, int *ub_cmpval)
+{
+	/*
+	 * Check if the outer partition's upper bound is lower than the inner
+	 * partition's lower bound; if so the partitions aren't overlapping.
+	 */
+	if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+							 outer_ub, inner_lb) < 0)
+	{
+		*lb_cmpval = -1;
+		*ub_cmpval = -1;
+		return false;
+	}
+
+	/*
+	 * Check if the outer partition's lower bound is higher than the inner
+	 * partition's upper bound; if so the partitions aren't overlapping.
+	 */
+	if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+							 outer_lb, inner_ub) > 0)
+	{
+		*lb_cmpval = 1;
+		*ub_cmpval = 1;
+		return false;
+	}
+
+	/* All other cases indicate overlapping partitions. */
+	*lb_cmpval = compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									  outer_lb, inner_lb);
+	*ub_cmpval = compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									  outer_ub, inner_ub);
+	return true;
+}
+
+/*
+ * get_merged_range_bounds
+ *		Given the bounds of range partitions to be joined, determine the bounds
+ *		of a merged partition produced from the range partitions
+ *
+ * *merged_lb and *merged_ub are set to the lower and upper bounds of the
+ * merged partition.
+ */
+static void
+get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations, JoinType jointype,
+						PartitionRangeBound *outer_lb,
+						PartitionRangeBound *outer_ub,
+						PartitionRangeBound *inner_lb,
+						PartitionRangeBound *inner_ub,
+						int	lb_cmpval, int ub_cmpval,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub)
+{
+	Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								outer_lb, inner_lb) == lb_cmpval);
+	Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								outer_ub, inner_ub) == ub_cmpval);
+
+	switch (jointype)
+	{
+		case JOIN_INNER:
+		case JOIN_SEMI:
+
+			/*
+			 * An INNER/SEMI join will have the rows that fit both sides, so
+			 * the lower bound of the merged partition will be the higher of
+			 * the two lower bounds, and the upper bound of the merged
+			 * partition will be the lower of the two upper bounds.
+			 */
+			*merged_lb = (lb_cmpval > 0) ? *outer_lb : *inner_lb;
+			*merged_ub = (ub_cmpval < 0) ? *outer_ub : *inner_ub;
+			break;
+
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+
+			/*
+			 * A LEFT/ANTI join will have all the rows from the outer side, so
+			 * the bounds of the merged partition will be the same as the outer
+			 * bounds.
+			 */
+			*merged_lb = *outer_lb;
+			*merged_ub = *outer_ub;
+			break;
+
+		case JOIN_FULL:
+
+			/*
+			 * A FULL join will have all the rows from both sides, so the lower
+			 * bound of the merged partition will be the lower of the two lower
+			 * bounds, and the upper bound of the merged partition will be the
+			 * higher of the two upper bounds.
+			 */
+			*merged_lb = (lb_cmpval < 0) ? *outer_lb : *inner_lb;
+			*merged_ub = (ub_cmpval > 0) ? *outer_ub : *inner_ub;
+			break;
+
+		default:
+			elog(ERROR, "unrecognized join type: %d", (int) jointype);
+	}
+}
+
+/*
+ * add_merged_range_bounds
+ *		Add the bounds of a merged partition to the lists of range bounds
+ */
+static void
+add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub,
+						int merged_index,
+						List **merged_datums,
+						List **merged_kinds,
+						List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		/* First merged partition */
+		Assert(!*merged_kinds);
+		Assert(!*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		Assert(*merged_datums);
+		Assert(*merged_kinds);
+		Assert(*merged_indexes);
+
+		/* Get the last upper bound. */
+		prev_ub.index = llast_int(*merged_indexes);
+		prev_ub.datums = (Datum *) llast(*merged_datums);
+		prev_ub.kind = (PartitionRangeDatumKind *) llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		/*
+		 * We pass to partition_rbound_cmp() lower1 as false to prevent it
+		 * from considering the last upper bound to be smaller than the lower
+		 * bound of the merged partition when the values of the two range
+		 * bounds compare equal.
+		 */
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  merged_lb->datums, merged_lb->kind,
+									  false, &prev_ub);
+		Assert(cmpval >= 0);
+	}
+
+	/*
+	 * If the lower bound is higher than the last upper bound, add the lower
+	 * bound with the index as -1 indicating that that is a lower bound; else,
+	 * the last upper bound will be reused as the lower bound of the merged
+	 * partition, so skip this.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, merged_lb->datums);
+		*merged_kinds = lappend(*merged_kinds, merged_lb->kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	/* Add the upper bound and index of the merged partition. */
+	*merged_datums = lappend(*merged_datums, merged_ub->datums);
+	*merged_kinds = lappend(*merged_kinds, merged_ub->kind);
+	*merged_indexes = lappend_int(*merged_indexes, merged_index);
+}
+
 /*
  * partitions_are_ordered
  *		Determine whether the partitions described by 'boundinfo' are ordered,
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index 0ceb809644..622ea2bf63 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -579,8 +579,10 @@ typedef struct PartitionSchemeData *PartitionScheme;
  *		part_scheme - Partitioning scheme of the relation
  *		nparts - Number of partitions
  *		boundinfo - Partition bounds
+ *		merged - true if partition bounds are merged ones
  *		partition_qual - Partition constraint if not the root
  *		part_rels - RelOptInfos for each partition
+ *		all_partrels - Relids set of all partition relids
  *		partexprs, nullable_partexprs - Partition key expressions
  *		partitioned_child_rels - RT indexes of unpruned partitions of
  *								 this relation that are partitioned tables
@@ -718,11 +720,15 @@ typedef struct RelOptInfo
 
 	/* used for partitioned relations */
 	PartitionScheme part_scheme;	/* Partitioning scheme. */
-	int			nparts;			/* number of partitions */
+	int			nparts;			/* number of partitions; 0 = not partitioned;
+								 * -1 = not yet set */
 	struct PartitionBoundInfoData *boundinfo;	/* Partition bounds */
+	bool		merged;			/* true if partition bounds were created by
+								 * partition_bounds_merge() */
 	List	   *partition_qual; /* partition constraint */
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of partitions,
 									 * stored in the same order of bounds */
+	Relids		all_partrels;	/* Relids set of all partition relids */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 	List	   *partitioned_child_rels; /* List of RT indexes. */
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index be940b5192..dfc720720b 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -16,6 +16,7 @@
 #include "nodes/pg_list.h"
 #include "partitioning/partdefs.h"
 #include "utils/relcache.h"
+struct RelOptInfo;				/* avoid including pathnodes.h here */
 
 
 /*
@@ -87,6 +88,14 @@ extern bool partition_bounds_equal(int partnatts, int16 *parttyplen,
 								   PartitionBoundInfo b2);
 extern PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src,
 												PartitionKey key);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+												 FmgrInfo *partsupfunc,
+												 Oid *partcollation,
+												 struct RelOptInfo *outer_rel,
+												 struct RelOptInfo *inner_rel,
+												 JoinType jointype,
+												 List **outer_parts,
+												 List **inner_parts);
 extern bool partitions_are_ordered(PartitionBoundInfo boundinfo, int nparts);
 extern void check_new_partition_bound(char *relname, Relation parent,
 									  PartitionBoundSpec *spec);
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index b3fbe47bde..27588c883e 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -2055,30 +2055,2606 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
                ->  Seq Scan on prt1_n_p2 t1_2
 (10 rows)
 
--- partitionwise join can not be applied if only one of joining tables has
--- default partition
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
-ANALYZE prt2;
+--
+-- Test advanced partition-matching algorithm for partitioned join
+--
+-- Tests for range-partitioned tables
+CREATE TABLE prt1_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_adv_p1 PARTITION OF prt1_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_adv_p2 PARTITION OF prt1_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_adv_p3 PARTITION OF prt1_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_adv_a_idx ON prt1_adv (a);
+INSERT INTO prt1_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_adv;
+CREATE TABLE prt2_adv (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_adv_p1 PARTITION OF prt2_adv FOR VALUES FROM (100) TO (150);
+CREATE TABLE prt2_adv_p2 PARTITION OF prt2_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt2_adv_p3 PARTITION OF prt2_adv FOR VALUES FROM (350) TO (500);
+CREATE INDEX prt2_adv_b_idx ON prt2_adv (b);
+INSERT INTO prt2_adv_p1 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 149) i;
+INSERT INTO prt2_adv_p2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(200, 299) i;
+INSERT INTO prt2_adv_p3 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_adv;
+-- inner join
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
  Sort
    Sort Key: t1.a
-   ->  Hash Join
-         Hash Cond: (t2.b = t1.a)
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 100 | 0 | 0100
+ 125 | 0 | 0125
+ 200 | 0 | 0200
+ 225 | 0 | 0225
+ 250 | 0 | 0250
+ 275 | 0 | 0275
+ 350 | 0 | 0350
+ 375 | 0 | 0375
+(8 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 |     | 
+ 175 | 0175 |     | 
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 300 | 0300 |     | 
+ 325 | 0325 |     | 
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(12 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 150 | 0 | 0150
+ 175 | 0 | 0175
+ 300 | 0 | 0300
+ 325 | 0 | 0325
+(4 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                                QUERY PLAN                                
+--------------------------------------------------------------------------
+ Sort
+   Sort Key: prt1_adv.a, prt2_adv.b
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_adv_1.a = prt2_adv_1.b)
+               Filter: (((175) = prt1_adv_1.a) OR ((425) = prt2_adv_1.b))
+               ->  Seq Scan on prt1_adv_p1 prt1_adv_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 prt2_adv_1
+                           Filter: (a = 0)
+         ->  Hash Full Join
+               Hash Cond: (prt1_adv_2.a = prt2_adv_2.b)
+               Filter: (((175) = prt1_adv_2.a) OR ((425) = prt2_adv_2.b))
+               ->  Seq Scan on prt1_adv_p2 prt1_adv_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 prt2_adv_2
+                           Filter: (a = 0)
+         ->  Hash Full Join
+               Hash Cond: (prt2_adv_3.b = prt1_adv_3.a)
+               Filter: (((175) = prt1_adv_3.a) OR ((425) = prt2_adv_3.b))
+               ->  Seq Scan on prt2_adv_p3 prt2_adv_3
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 prt1_adv_3
+                           Filter: (b = 0)
+(27 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 175 | 0175 |     | 
+     |      | 425 | 0425
+(2 rows)
+
+-- Test cases where one side has an extra partition
+CREATE TABLE prt2_adv_extra PARTITION OF prt2_adv FOR VALUES FROM (500) TO (MAXVALUE);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(500, 599) i;
+ANALYZE prt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 100 | 0 | 0100
+ 125 | 0 | 0125
+ 200 | 0 | 0200
+ 225 | 0 | 0225
+ 250 | 0 | 0250
+ 275 | 0 | 0275
+ 350 | 0 | 0350
+ 375 | 0 | 0375
+(8 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 |     | 
+ 175 | 0175 |     | 
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 300 | 0300 |     | 
+ 325 | 0325 |     | 
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(12 rows)
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t2.a
+   ->  Hash Right Join
+         Hash Cond: (t2.a = t1.b)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2_1
-               ->  Seq Scan on prt2_p2 t2_2
-               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt1_adv_p1 t2_1
+               ->  Seq Scan on prt1_adv_p2 t2_2
+               ->  Seq Scan on prt1_adv_p3 t2_3
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt2_adv_p1 t1_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_adv_p2 t1_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_adv_p3 t1_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_adv_extra t1_4
+                           Filter: (a = 0)
+(18 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 150 | 0 | 0150
+ 175 | 0 | 0175
+ 300 | 0 | 0300
+ 325 | 0 | 0325
+(4 rows)
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt1_adv t2 WHERE t1.b = t2.a) AND t1.a = 0 ORDER BY t1.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_extra t1_4
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 t2_1
+                     ->  Seq Scan on prt1_adv_p2 t2_2
+                     ->  Seq Scan on prt1_adv_p3 t2_3
+(18 rows)
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                           QUERY PLAN                           
+----------------------------------------------------------------
+ Sort
+   Sort Key: prt1_adv.a, prt2_adv.b
+   ->  Hash Full Join
+         Hash Cond: (prt2_adv.b = prt1_adv.a)
+         Filter: (((175) = prt1_adv.a) OR ((425) = prt2_adv.b))
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 prt2_adv_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p2 prt2_adv_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p3 prt2_adv_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_extra prt2_adv_4
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 prt1_adv_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_adv_p2 prt1_adv_2
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_adv_p3 prt1_adv_3
                            Filter: (b = 0)
+(22 rows)
+
+-- 3-way join where not every pair of relations can do partitioned join
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c, t3.a, t3.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) INNER JOIN prt1_adv t3 ON (t1.b = t3.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a, t3.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t2.a
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_adv_p1 t1_1
+                           Filter: (a = 0)
+                     ->  Index Scan using prt1_adv_p1_a_idx on prt1_adv_p1 t3_1
+                           Index Cond: (a = t1_1.b)
+               ->  Index Scan using prt1_adv_p1_a_idx on prt1_adv_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.a = t1_2.b)
+               ->  Seq Scan on prt1_adv_p2 t2_2
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_2.a = t1_2.b)
+                           ->  Seq Scan on prt1_adv_p2 t3_2
+                           ->  Hash
+                                 ->  Seq Scan on prt2_adv_p2 t1_2
+                                       Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_3.a = t1_3.b)
+               ->  Seq Scan on prt1_adv_p3 t2_3
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_3.a = t1_3.b)
+                           ->  Seq Scan on prt1_adv_p3 t3_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_adv_p3 t1_3
+                                       Filter: (a = 0)
+(31 rows)
+
+SELECT t1.b, t1.c, t2.a, t2.c, t3.a, t3.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) INNER JOIN prt1_adv t3 ON (t1.b = t3.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a, t3.a;
+  b  |  c   |  a  |  c   |  a  |  c   
+-----+------+-----+------+-----+------
+ 100 | 0100 | 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275 | 275 | 0275
+ 350 | 0350 | 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375 | 375 | 0375
+(8 rows)
+
+DROP TABLE prt2_adv_extra;
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in such cases
+ALTER TABLE prt2_adv DETACH PARTITION prt2_adv_p3;
+-- Split prt2_adv_p3 into two partitions so that prt1_adv_p3 matches both
+CREATE TABLE prt2_adv_p3_1 PARTITION OF prt2_adv FOR VALUES FROM (350) TO (375);
+CREATE TABLE prt2_adv_p3_2 PARTITION OF prt2_adv FOR VALUES FROM (375) TO (500);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Seq Scan on prt2_adv_p3_1 t2_3
+               ->  Seq Scan on prt2_adv_p3_2 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+                     ->  Seq Scan on prt2_adv_p3_1 t2_3
+                     ->  Seq Scan on prt2_adv_p3_2 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Seq Scan on prt2_adv_p3_1 t2_3
+               ->  Seq Scan on prt2_adv_p3_2 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(17 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+                     ->  Seq Scan on prt2_adv_p3_1 t2_3
+                     ->  Seq Scan on prt2_adv_p3_2 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                           QUERY PLAN                           
+----------------------------------------------------------------
+ Sort
+   Sort Key: prt1_adv.a, prt2_adv.b
+   ->  Hash Full Join
+         Hash Cond: (prt2_adv.b = prt1_adv.a)
+         Filter: (((175) = prt1_adv.a) OR ((425) = prt2_adv.b))
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 prt2_adv_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p2 prt2_adv_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p3_1 prt2_adv_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p3_2 prt2_adv_4
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 prt1_adv_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p2 prt1_adv_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 prt1_adv_3
+                           Filter: (b = 0)
+(22 rows)
+
+DROP TABLE prt2_adv_p3_1;
+DROP TABLE prt2_adv_p3_2;
+ANALYZE prt2_adv;
+-- Test default partitions
+ALTER TABLE prt1_adv DETACH PARTITION prt1_adv_p1;
+-- Change prt1_adv_p1 to the default partition
+ALTER TABLE prt1_adv ATTACH PARTITION prt1_adv_p1 DEFAULT;
+ALTER TABLE prt1_adv DETACH PARTITION prt1_adv_p3;
+ANALYZE prt1_adv;
+-- We can do partitioned join even if only one of relations has the default
+-- partition
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_1
+                           Filter: (b = 0)
+(15 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+(6 rows)
+
+-- Restore prt1_adv_p3
+ALTER TABLE prt1_adv ATTACH PARTITION prt1_adv_p3 FOR VALUES FROM (300) TO (400);
+ANALYZE prt1_adv;
+-- Restore prt2_adv_p3
+ALTER TABLE prt2_adv ATTACH PARTITION prt2_adv_p3 FOR VALUES FROM (350) TO (500);
+ANALYZE prt2_adv;
+-- Partitioned join can't be applied because the default partition of prt1_adv
+-- matches prt2_adv_p1 and prt2_adv_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Seq Scan on prt2_adv_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p2 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p1 t1_3
+                           Filter: (b = 0)
+(16 rows)
+
+ALTER TABLE prt2_adv DETACH PARTITION prt2_adv_p3;
+-- Change prt2_adv_p3 to the default partition
+ALTER TABLE prt2_adv ATTACH PARTITION prt2_adv_p3 DEFAULT;
+ANALYZE prt2_adv;
+-- Partitioned join can't be applied because the default partition of prt1_adv
+-- matches prt2_adv_p1 and prt2_adv_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Seq Scan on prt2_adv_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p2 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p1 t1_3
+                           Filter: (b = 0)
+(16 rows)
+
+DROP TABLE prt1_adv_p3;
+ANALYZE prt1_adv;
+DROP TABLE prt2_adv_p3;
+ANALYZE prt2_adv;
+CREATE TABLE prt3_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt3_adv_p1 PARTITION OF prt3_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt3_adv_p2 PARTITION OF prt3_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt3_adv_a_idx ON prt3_adv (a);
+INSERT INTO prt3_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(200, 399) i;
+ANALYZE prt3_adv;
+-- 3-way join to test the default partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) LEFT JOIN prt3_adv t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b, t3.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t3_1.a = t1_1.a)
+               ->  Seq Scan on prt3_adv_p1 t3_1
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_2.b = t1_1.a)
+                           ->  Seq Scan on prt2_adv_p2 t2_2
+                           ->  Hash
+                                 ->  Seq Scan on prt1_adv_p2 t1_1
+                                       Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t3_2.a = t1_2.a)
+               ->  Seq Scan on prt3_adv_p2 t3_2
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_1.b = t1_2.a)
+                           ->  Seq Scan on prt2_adv_p1 t2_1
+                           ->  Hash
+                                 ->  Seq Scan on prt1_adv_p1 t1_2
+                                       Filter: (b = 0)
+(23 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) LEFT JOIN prt3_adv t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+  a  |  c   |  b  |  c   |  a  |  c   
+-----+------+-----+------+-----+------
+ 100 | 0100 | 100 | 0100 |     | 
+ 125 | 0125 | 125 | 0125 |     | 
+ 150 | 0150 |     |      |     | 
+ 175 | 0175 |     |      |     | 
+ 200 | 0200 | 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275 | 275 | 0275
+(8 rows)
+
+DROP TABLE prt1_adv;
+DROP TABLE prt2_adv;
+DROP TABLE prt3_adv;
+-- Test interaction of partitioned join with partition pruning
+CREATE TABLE prt1_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_adv_p1 PARTITION OF prt1_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_adv_p2 PARTITION OF prt1_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_adv_p3 PARTITION OF prt1_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_adv_a_idx ON prt1_adv (a);
+INSERT INTO prt1_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_adv;
+CREATE TABLE prt2_adv (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_adv_p1 PARTITION OF prt2_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt2_adv_p2 PARTITION OF prt2_adv FOR VALUES FROM (200) TO (400);
+CREATE INDEX prt2_adv_b_idx ON prt2_adv (b);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+                        QUERY PLAN                         
+-----------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: ((a < 300) AND (b = 0))
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: ((a < 300) AND (b = 0))
+(15 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 | 150 | 0150
+ 175 | 0175 | 175 | 0175
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+(8 rows)
+
+DROP TABLE prt1_adv_p3;
+CREATE TABLE prt1_adv_default PARTITION OF prt1_adv DEFAULT;
+ANALYZE prt1_adv;
+CREATE TABLE prt2_adv_default PARTITION OF prt2_adv DEFAULT;
+ANALYZE prt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+                                QUERY PLAN                                
+--------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: ((a >= 100) AND (a < 300) AND (b = 0))
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: ((a >= 100) AND (a < 300) AND (b = 0))
+(15 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 | 150 | 0150
+ 175 | 0175 | 175 | 0175
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+(8 rows)
+
+DROP TABLE prt1_adv;
+DROP TABLE prt2_adv;
+-- Tests for list-partitioned tables
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0001', '0003');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt1_adv_p3 PARTITION OF plt1_adv FOR VALUES IN ('0008', '0009');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+ANALYZE plt1_adv;
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt2_adv_p3 PARTITION OF plt2_adv FOR VALUES IN ('0007', '0009');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+(6 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 1 | 1 | 0001
+ 8 | 8 | 0008
+(2 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+   |      | 2 | 0002
+   |      | 7 | 0007
+(8 rows)
+
+-- Test cases where one side has an extra partition
+CREATE TABLE plt2_adv_extra PARTITION OF plt2_adv FOR VALUES IN ('0000');
+INSERT INTO plt2_adv_extra VALUES (0, 0, '0000');
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+(6 rows)
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2_adv t1 LEFT JOIN plt1_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt1_adv_p1 t2_1
+               ->  Seq Scan on plt1_adv_p2 t2_2
+               ->  Seq Scan on plt1_adv_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_adv_extra t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_adv_p1 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_adv_p2 t1_3
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_adv_p3 t1_4
+                           Filter: (b < 10)
+(18 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 1 | 1 | 0001
+ 8 | 8 | 0008
+(2 rows)
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt2_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt1_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Anti Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_extra t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t1_4
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t2_1
+                     ->  Seq Scan on plt1_adv_p2 t2_2
+                     ->  Seq Scan on plt1_adv_p3 t2_3
+(18 rows)
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt2_adv_extra t2_1
+               ->  Seq Scan on plt2_adv_p1 t2_2
+               ->  Seq Scan on plt2_adv_p2 t2_3
+               ->  Seq Scan on plt2_adv_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+(15 rows)
+
+DROP TABLE plt2_adv_extra;
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in such cases
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2;
+-- Split plt2_adv_p2 into two partitions so that plt1_adv_p2 matches both
+CREATE TABLE plt2_adv_p2_1 PARTITION OF plt2_adv FOR VALUES IN ('0004');
+CREATE TABLE plt2_adv_p2_2 PARTITION OF plt2_adv FOR VALUES IN ('0006');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6);
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_1 t2_2
+               ->  Seq Scan on plt2_adv_p2_2 t2_3
+               ->  Seq Scan on plt2_adv_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Semi Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+                     ->  Seq Scan on plt2_adv_p2_1 t2_2
+                     ->  Seq Scan on plt2_adv_p2_2 t2_3
+                     ->  Seq Scan on plt2_adv_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_1 t2_2
+               ->  Seq Scan on plt2_adv_p2_2 t2_3
+               ->  Seq Scan on plt2_adv_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(17 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Anti Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+                     ->  Seq Scan on plt2_adv_p2_1 t2_2
+                     ->  Seq Scan on plt2_adv_p2_2 t2_3
+                     ->  Seq Scan on plt2_adv_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_1 t2_2
+               ->  Seq Scan on plt2_adv_p2_2 t2_3
+               ->  Seq Scan on plt2_adv_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+(15 rows)
+
+DROP TABLE plt2_adv_p2_1;
+DROP TABLE plt2_adv_p2_2;
+-- Restore plt2_adv_p2
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2 FOR VALUES IN ('0004', '0006');
+-- Test NULL partitions
+ALTER TABLE plt1_adv DETACH PARTITION plt1_adv_p1;
+-- Change plt1_adv_p1 to the NULL partition
+CREATE TABLE plt1_adv_p1_null PARTITION OF plt1_adv FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p3;
+-- Change plt2_adv_p3 to the NULL partition
+CREATE TABLE plt2_adv_p3_null PARTITION OF plt2_adv FOR VALUES IN (NULL, '0007', '0009');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (7, 9);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1_null t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1_null t1_1
+                     Filter: (b < 10)
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+(19 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1_null t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a  |  c   | a |  c   
+----+------+---+------
+ -1 |      |   | 
+  1 | 0001 |   | 
+  3 | 0003 | 3 | 0003
+  4 | 0004 | 4 | 0004
+  6 | 0006 | 6 | 0006
+  8 | 0008 |   | 
+  9 | 0009 | 9 | 0009
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1_null t1_1
+                     Filter: (b < 10)
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+(19 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a  | b  |  c   
+----+----+------
+ -1 | -1 | 
+  1 |  1 | 0001
+  8 |  8 | 0008
+(3 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p1_null t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+ -1 |      |    | 
+  1 | 0001 |    | 
+  3 | 0003 |  3 | 0003
+  4 | 0004 |  4 | 0004
+  6 | 0006 |  6 | 0006
+  8 | 0008 |    | 
+  9 | 0009 |  9 | 0009
+    |      | -1 | 
+    |      |  2 | 0002
+    |      |  7 | 0007
+(10 rows)
+
+DROP TABLE plt1_adv_p1_null;
+-- Restore plt1_adv_p1
+ALTER TABLE plt1_adv ATTACH PARTITION plt1_adv_p1 FOR VALUES IN ('0001', '0003');
+-- Add to plt1_adv the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt1_adv_extra PARTITION OF plt1_adv FOR VALUES IN (NULL);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+DROP TABLE plt2_adv_p3_null;
+-- Restore plt2_adv_p3
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p3 FOR VALUES IN ('0007', '0009');
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Seq Scan on plt2_adv_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_extra t1_4
+                           Filter: (b < 10)
+(18 rows)
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt1_adv_p1 t1_1
+               ->  Seq Scan on plt1_adv_p2 t1_2
+               ->  Seq Scan on plt1_adv_p3 t1_3
+               ->  Seq Scan on plt1_adv_extra t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+                     ->  Seq Scan on plt2_adv_p2 t2_2
+                     ->  Seq Scan on plt2_adv_p3 t2_3
+(15 rows)
+
+-- Add to plt2_adv the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt2_adv_extra PARTITION OF plt2_adv FOR VALUES IN (NULL);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+         ->  Nested Loop Left Join
+               Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+               ->  Seq Scan on plt1_adv_extra t1_4
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_extra t2_4
+(26 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a  |  c   | a |  c   
+----+------+---+------
+ -1 |      |   | 
+  1 | 0001 |   | 
+  3 | 0003 | 3 | 0003
+  4 | 0004 | 4 | 0004
+  6 | 0006 | 6 | 0006
+  8 | 0008 |   | 
+  9 | 0009 | 9 | 0009
+(7 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p3 t2_3
+         ->  Hash Full Join
+               Hash Cond: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+               Filter: ((COALESCE(t1_4.b, 0) < 10) AND (COALESCE(t2_4.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_extra t1_4
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_extra t2_4
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+ -1 |      |    | 
+  1 | 0001 |    | 
+  3 | 0003 |  3 | 0003
+  4 | 0004 |  4 | 0004
+  6 | 0006 |  6 | 0006
+  8 | 0008 |    | 
+  9 | 0009 |  9 | 0009
+    |      | -1 | 
+    |      |  2 | 0002
+    |      |  7 | 0007
+(10 rows)
+
+-- 3-way join to test the NULL partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt1_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t3_1.a = t1_1.a) AND (t3_1.c = t1_1.c))
+               ->  Seq Scan on plt1_adv_p1 t3_1
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+                           ->  Seq Scan on plt2_adv_p1 t2_1
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p1 t1_1
+                                       Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t3_2.a = t1_2.a) AND (t3_2.c = t1_2.c))
+               ->  Seq Scan on plt1_adv_p2 t3_2
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+                           ->  Seq Scan on plt2_adv_p2 t2_2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p2 t1_2
+                                       Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t3_3.a = t1_3.a) AND (t3_3.c = t1_3.c))
+               ->  Seq Scan on plt1_adv_p3 t3_3
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+                           ->  Seq Scan on plt2_adv_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p3 t1_3
+                                       Filter: (b < 10)
+         ->  Nested Loop Left Join
+               Join Filter: ((t1_4.a = t3_4.a) AND (t1_4.c = t3_4.c))
+               ->  Nested Loop Left Join
+                     Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+                     ->  Seq Scan on plt1_adv_extra t1_4
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_adv_extra t2_4
+               ->  Seq Scan on plt1_adv_extra t3_4
+(41 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt1_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a  |  c   | a |  c   | a |  c   
+----+------+---+------+---+------
+ -1 |      |   |      |   | 
+  1 | 0001 |   |      | 1 | 0001
+  3 | 0003 | 3 | 0003 | 3 | 0003
+  4 | 0004 | 4 | 0004 | 4 | 0004
+  6 | 0006 | 6 | 0006 | 6 | 0006
+  8 | 0008 |   |      | 8 | 0008
+  9 | 0009 | 9 | 0009 | 9 | 0009
+(7 rows)
+
+DROP TABLE plt1_adv_extra;
+DROP TABLE plt2_adv_extra;
+-- Test default partitions
+ALTER TABLE plt1_adv DETACH PARTITION plt1_adv_p1;
+-- Change plt1_adv_p1 to the default partition
+ALTER TABLE plt1_adv ATTACH PARTITION plt1_adv_p1 DEFAULT;
+DROP TABLE plt1_adv_p3;
+ANALYZE plt1_adv;
+DROP TABLE plt2_adv_p3;
+ANALYZE plt2_adv;
+-- We can do partitioned join even if only one of relations has the default
+-- partition
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_1
+                           Filter: (b < 10)
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(3 rows)
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2;
+-- Change plt2_adv_p2 to contain '0005' in addition to '0004' and '0006' as
+-- the key values
+CREATE TABLE plt2_adv_p2_ext PARTITION OF plt2_adv FOR VALUES IN ('0004', '0005', '0006');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 5, 6);
+ANALYZE plt2_adv;
+-- Partitioned join can't be applied because the default partition of plt1_adv
+-- matches plt2_adv_p1 and plt2_adv_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_ext t2_2
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p2 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p1 t1_2
+                           Filter: (b < 10)
+(13 rows)
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2_ext;
+-- Change plt2_adv_p2_ext to the default partition
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2_ext DEFAULT;
+ANALYZE plt2_adv;
+-- Partitioned join can't be applied because the default partition of plt1_adv
+-- matches plt2_adv_p1 and plt2_adv_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_ext t2_2
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p2 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p1 t1_2
+                           Filter: (b < 10)
+(13 rows)
+
+DROP TABLE plt2_adv_p2_ext;
+-- Restore plt2_adv_p2
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_adv;
+CREATE TABLE plt3_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_adv_p1 PARTITION OF plt3_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt3_adv_p2 PARTITION OF plt3_adv FOR VALUES IN ('0007', '0009');
+INSERT INTO plt3_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6, 7, 9);
+ANALYZE plt3_adv;
+-- 3-way join to test the default partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t3_1.a = t1_1.a) AND (t3_1.c = t1_1.c))
+               ->  Seq Scan on plt3_adv_p1 t3_1
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+                           ->  Seq Scan on plt2_adv_p2 t2_2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p2 t1_1
+                                       Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t3_2.a = t1_2.a) AND (t3_2.c = t1_2.c))
+               ->  Seq Scan on plt3_adv_p2 t3_2
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+                           ->  Seq Scan on plt2_adv_p1 t2_1
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p1 t1_2
+                                       Filter: (b < 10)
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   | a |  c   
+---+------+---+------+---+------
+ 1 | 0001 |   |      |   | 
+ 3 | 0003 | 3 | 0003 |   | 
+ 4 | 0004 | 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006 | 6 | 0006
+(4 rows)
+
+-- Test cases where one side has the default partition while the other side
+-- has the NULL partition
+DROP TABLE plt2_adv_p1;
+-- Add the NULL partition to plt2_adv
+CREATE TABLE plt2_adv_p1_null PARTITION OF plt2_adv FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p1_null t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_1
+                           Filter: (b < 10)
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 | 1 | 0001
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(4 rows)
+
+DROP TABLE plt2_adv_p1_null;
+-- Add the NULL partition that contains only NULL values as the key values
+CREATE TABLE plt2_adv_p1_null PARTITION OF plt2_adv FOR VALUES IN (NULL);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Seq Scan on plt2_adv_p2 t2
+         ->  Hash
+               ->  Seq Scan on plt1_adv_p2 t1
+                     Filter: (b < 10)
+(8 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(2 rows)
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+DROP TABLE plt3_adv;
+-- Test interaction of partitioned join with partition pruning
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0001');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0002');
+CREATE TABLE plt1_adv_p3 PARTITION OF plt1_adv FOR VALUES IN ('0003');
+CREATE TABLE plt1_adv_p4 PARTITION OF plt1_adv FOR VALUES IN (NULL, '0004', '0005');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN (NULL);
+CREATE TABLE plt2_adv_p3 PARTITION OF plt2_adv FOR VALUES IN ('0003');
+CREATE TABLE plt2_adv_p4 PARTITION OF plt2_adv FOR VALUES IN ('0004', '0005');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+                                       QUERY PLAN                                        
+-----------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p3 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_1
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p4 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p4 t1_2
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 5 | 0005 | 5 | 0005
+(3 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Seq Scan on plt2_adv_p4 t2
+         ->  Hash
+               ->  Seq Scan on plt1_adv_p4 t1
+                     Filter: ((c IS NULL) AND (b < 10))
+(8 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+ a  | c | a | c 
+----+---+---+---
+ -1 |   |   | 
+(1 row)
+
+CREATE TABLE plt1_adv_default PARTITION OF plt1_adv DEFAULT;
+ANALYZE plt1_adv;
+CREATE TABLE plt2_adv_default PARTITION OF plt2_adv DEFAULT;
+ANALYZE plt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+                                       QUERY PLAN                                        
+-----------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p3 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_1
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p4 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p4 t1_2
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 5 | 0005 | 5 | 0005
+(3 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Seq Scan on plt2_adv_p4 t2
+         ->  Hash
+               ->  Seq Scan on plt1_adv_p4 t1
+                     Filter: ((c IS NULL) AND (b < 10))
+(8 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+ a  | c | a | c 
+----+---+---+---
+ -1 |   |   | 
+(1 row)
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+-- Test the process_outer_partition() code path
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1_adv;
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0002');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2_adv;
+CREATE TABLE plt3_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_adv_p1 PARTITION OF plt3_adv FOR VALUES IN ('0001');
+CREATE TABLE plt3_adv_p2 PARTITION OF plt3_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3_adv;
+-- This tests that when merging partitions from plt1_adv and plt2_adv in
+-- merge_list_bounds(), process_outer_partition() returns an already-assigned
+-- merged partition when re-called with plt1_adv_p1 for the second list value
+-- '0001' of that partitin
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.c = t2.c)) FULL JOIN plt3_adv t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2.a, t3.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t3_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_1.c = t2_1.c)
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                     ->  Hash
+                           ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt3_adv_p1 t3_1
+         ->  Hash Full Join
+               Hash Cond: (t1_2.c = t3_2.c)
+               Filter: (((COALESCE(t1_2.a, 0) % 5) <> 3) AND ((COALESCE(t1_2.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_2.c = t2_2.c)
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                     ->  Hash
+                           ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt3_adv_p2 t3_2
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.c = t2.c)) FULL JOIN plt3_adv t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+ a  |  c   | a  |  c   | a  |  c   
+----+------+----+------+----+------
+  0 | 0000 |    |      |    | 
+  5 | 0000 |    |      |    | 
+ 10 | 0000 |    |      |    | 
+ 15 | 0000 |    |      |    | 
+ 20 | 0000 |    |      |    | 
+  1 | 0001 |    |      |  1 | 0001
+  1 | 0001 |    |      |  6 | 0001
+  1 | 0001 |    |      | 11 | 0001
+  1 | 0001 |    |      | 16 | 0001
+  1 | 0001 |    |      | 21 | 0001
+  6 | 0001 |    |      |  1 | 0001
+  6 | 0001 |    |      |  6 | 0001
+  6 | 0001 |    |      | 11 | 0001
+  6 | 0001 |    |      | 16 | 0001
+  6 | 0001 |    |      | 21 | 0001
+ 11 | 0001 |    |      |  1 | 0001
+ 11 | 0001 |    |      |  6 | 0001
+ 11 | 0001 |    |      | 11 | 0001
+ 11 | 0001 |    |      | 16 | 0001
+ 11 | 0001 |    |      | 21 | 0001
+ 16 | 0001 |    |      |  1 | 0001
+ 16 | 0001 |    |      |  6 | 0001
+ 16 | 0001 |    |      | 11 | 0001
+ 16 | 0001 |    |      | 16 | 0001
+ 16 | 0001 |    |      | 21 | 0001
+ 21 | 0001 |    |      |  1 | 0001
+ 21 | 0001 |    |      |  6 | 0001
+ 21 | 0001 |    |      | 11 | 0001
+ 21 | 0001 |    |      | 16 | 0001
+ 21 | 0001 |    |      | 21 | 0001
+  2 | 0002 |  2 | 0002 |    | 
+  2 | 0002 |  7 | 0002 |    | 
+  2 | 0002 | 12 | 0002 |    | 
+  2 | 0002 | 17 | 0002 |    | 
+  2 | 0002 | 22 | 0002 |    | 
+  7 | 0002 |  2 | 0002 |    | 
+  7 | 0002 |  7 | 0002 |    | 
+  7 | 0002 | 12 | 0002 |    | 
+  7 | 0002 | 17 | 0002 |    | 
+  7 | 0002 | 22 | 0002 |    | 
+ 12 | 0002 |  2 | 0002 |    | 
+ 12 | 0002 |  7 | 0002 |    | 
+ 12 | 0002 | 12 | 0002 |    | 
+ 12 | 0002 | 17 | 0002 |    | 
+ 12 | 0002 | 22 | 0002 |    | 
+ 17 | 0002 |  2 | 0002 |    | 
+ 17 | 0002 |  7 | 0002 |    | 
+ 17 | 0002 | 12 | 0002 |    | 
+ 17 | 0002 | 17 | 0002 |    | 
+ 17 | 0002 | 22 | 0002 |    | 
+ 22 | 0002 |  2 | 0002 |    | 
+ 22 | 0002 |  7 | 0002 |    | 
+ 22 | 0002 | 12 | 0002 |    | 
+ 22 | 0002 | 17 | 0002 |    | 
+ 22 | 0002 | 22 | 0002 |    | 
+(55 rows)
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+DROP TABLE plt3_adv;
+-- Tests for multi-level partitioned tables
+CREATE TABLE alpha (a double precision, b int, c text) PARTITION BY RANGE (a);
+CREATE TABLE alpha_neg PARTITION OF alpha FOR VALUES FROM ('-Infinity') TO (0) PARTITION BY RANGE (b);
+CREATE TABLE alpha_pos PARTITION OF alpha FOR VALUES FROM (0) TO (10.0) PARTITION BY LIST (c);
+CREATE TABLE alpha_neg_p1 PARTITION OF alpha_neg FOR VALUES FROM (100) TO (200);
+CREATE TABLE alpha_neg_p2 PARTITION OF alpha_neg FOR VALUES FROM (200) TO (300);
+CREATE TABLE alpha_neg_p3 PARTITION OF alpha_neg FOR VALUES FROM (300) TO (400);
+CREATE TABLE alpha_pos_p1 PARTITION OF alpha_pos FOR VALUES IN ('0001', '0003');
+CREATE TABLE alpha_pos_p2 PARTITION OF alpha_pos FOR VALUES IN ('0004', '0006');
+CREATE TABLE alpha_pos_p3 PARTITION OF alpha_pos FOR VALUES IN ('0008', '0009');
+INSERT INTO alpha_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+INSERT INTO alpha_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+ANALYZE alpha;
+CREATE TABLE beta (a double precision, b int, c text) PARTITION BY RANGE (a);
+CREATE TABLE beta_neg PARTITION OF beta FOR VALUES FROM (-10.0) TO (0) PARTITION BY RANGE (b);
+CREATE TABLE beta_pos PARTITION OF beta FOR VALUES FROM (0) TO ('Infinity') PARTITION BY LIST (c);
+CREATE TABLE beta_neg_p1 PARTITION OF beta_neg FOR VALUES FROM (100) TO (150);
+CREATE TABLE beta_neg_p2 PARTITION OF beta_neg FOR VALUES FROM (200) TO (300);
+CREATE TABLE beta_neg_p3 PARTITION OF beta_neg FOR VALUES FROM (350) TO (500);
+CREATE TABLE beta_pos_p1 PARTITION OF beta_pos FOR VALUES IN ('0002', '0003');
+CREATE TABLE beta_pos_p2 PARTITION OF beta_pos FOR VALUES IN ('0004', '0006');
+CREATE TABLE beta_pos_p3 PARTITION OF beta_pos FOR VALUES IN ('0007', '0009');
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+ANALYZE beta;
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.b
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.b = t2_1.b))
+               ->  Seq Scan on alpha_neg_p1 t1_1
+                     Filter: ((b >= 125) AND (b < 225))
+               ->  Hash
+                     ->  Seq Scan on beta_neg_p1 t2_1
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.b = t1_2.b))
+               ->  Seq Scan on beta_neg_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on alpha_neg_p2 t1_2
+                           Filter: ((b >= 125) AND (b < 225))
+         ->  Hash Join
+               Hash Cond: ((t2_4.a = t1_4.a) AND (t2_4.b = t1_4.b))
+               ->  Append
+                     ->  Seq Scan on beta_pos_p1 t2_4
+                     ->  Seq Scan on beta_pos_p2 t2_5
+                     ->  Seq Scan on beta_pos_p3 t2_6
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on alpha_pos_p1 t1_4
+                                 Filter: ((b >= 125) AND (b < 225))
+                           ->  Seq Scan on alpha_pos_p2 t1_5
+                                 Filter: ((b >= 125) AND (b < 225))
+                           ->  Seq Scan on alpha_pos_p3 t1_6
+                                 Filter: ((b >= 125) AND (b < 225))
+(29 rows)
+
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b;
+ a  |  b  |  c   | a  |  b  |  c   
+----+-----+------+----+-----+------
+ -1 | 126 | 0006 | -1 | 126 | 0006
+ -1 | 129 | 0009 | -1 | 129 | 0009
+ -1 | 133 | 0003 | -1 | 133 | 0003
+ -1 | 134 | 0004 | -1 | 134 | 0004
+ -1 | 136 | 0006 | -1 | 136 | 0006
+ -1 | 139 | 0009 | -1 | 139 | 0009
+ -1 | 143 | 0003 | -1 | 143 | 0003
+ -1 | 144 | 0004 | -1 | 144 | 0004
+ -1 | 146 | 0006 | -1 | 146 | 0006
+ -1 | 149 | 0009 | -1 | 149 | 0009
+ -1 | 203 | 0003 | -1 | 203 | 0003
+ -1 | 204 | 0004 | -1 | 204 | 0004
+ -1 | 206 | 0006 | -1 | 206 | 0006
+ -1 | 209 | 0009 | -1 | 209 | 0009
+ -1 | 213 | 0003 | -1 | 213 | 0003
+ -1 | 214 | 0004 | -1 | 214 | 0004
+ -1 | 216 | 0006 | -1 | 216 | 0006
+ -1 | 219 | 0009 | -1 | 219 | 0009
+ -1 | 223 | 0003 | -1 | 223 | 0003
+ -1 | 224 | 0004 | -1 | 224 | 0004
+  1 | 126 | 0006 |  1 | 126 | 0006
+  1 | 129 | 0009 |  1 | 129 | 0009
+  1 | 133 | 0003 |  1 | 133 | 0003
+  1 | 134 | 0004 |  1 | 134 | 0004
+  1 | 136 | 0006 |  1 | 136 | 0006
+  1 | 139 | 0009 |  1 | 139 | 0009
+  1 | 143 | 0003 |  1 | 143 | 0003
+  1 | 144 | 0004 |  1 | 144 | 0004
+  1 | 146 | 0006 |  1 | 146 | 0006
+  1 | 149 | 0009 |  1 | 149 | 0009
+  1 | 203 | 0003 |  1 | 203 | 0003
+  1 | 204 | 0004 |  1 | 204 | 0004
+  1 | 206 | 0006 |  1 | 206 | 0006
+  1 | 209 | 0009 |  1 | 209 | 0009
+  1 | 213 | 0003 |  1 | 213 | 0003
+  1 | 214 | 0004 |  1 | 214 | 0004
+  1 | 216 | 0006 |  1 | 216 | 0006
+  1 | 219 | 0009 |  1 | 219 | 0009
+  1 | 223 | 0003 |  1 | 223 | 0003
+  1 | 224 | 0004 |  1 | 224 | 0004
+(40 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b, t2.b;
+                                                              QUERY PLAN                                                              
+--------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.b, t2.b
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Append
+                     ->  Seq Scan on alpha_neg_p1 t1_2
+                           Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+                     ->  Seq Scan on alpha_neg_p2 t1_3
+                           Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on beta_neg_p1 t2_2
+                                 Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+                           ->  Seq Scan on beta_neg_p2 t2_3
+                                 Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Nested Loop
+               Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+               ->  Seq Scan on alpha_pos_p2 t1_4
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Seq Scan on beta_pos_p2 t2_4
+                     Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Nested Loop
+               Join Filter: ((t1_5.a = t2_5.a) AND (t1_5.c = t2_5.c))
+               ->  Seq Scan on alpha_pos_p3 t1_5
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Seq Scan on beta_pos_p3 t2_5
+                     Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+(28 rows)
+
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b, t2.b;
+ a  |  b  |  c   | a  |  b  |  c   
+----+-----+------+----+-----+------
+ -1 | 104 | 0004 | -1 | 104 | 0004
+ -1 | 104 | 0004 | -1 | 204 | 0004
+ -1 | 109 | 0009 | -1 | 109 | 0009
+ -1 | 109 | 0009 | -1 | 209 | 0009
+ -1 | 204 | 0004 | -1 | 104 | 0004
+ -1 | 204 | 0004 | -1 | 204 | 0004
+ -1 | 209 | 0009 | -1 | 109 | 0009
+ -1 | 209 | 0009 | -1 | 209 | 0009
+  1 | 104 | 0004 |  1 | 104 | 0004
+  1 | 104 | 0004 |  1 | 204 | 0004
+  1 | 109 | 0009 |  1 | 109 | 0009
+  1 | 109 | 0009 |  1 | 209 | 0009
+  1 | 204 | 0004 |  1 | 104 | 0004
+  1 | 204 | 0004 |  1 | 204 | 0004
+  1 | 209 | 0009 |  1 | 109 | 0009
+  1 | 209 | 0009 |  1 | 209 | 0009
 (16 rows)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;
+                                                           QUERY PLAN                                                           
+--------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.b
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on alpha_neg_p1 t1_1
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Hash
+                     ->  Seq Scan on beta_neg_p1 t2_1
+                           Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Hash Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on alpha_neg_p2 t1_2
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Hash
+                     ->  Seq Scan on beta_neg_p2 t2_2
+                           Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Nested Loop
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.b = t2_3.b) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on alpha_pos_p2 t1_3
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Seq Scan on beta_pos_p2 t2_3
+                     Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Nested Loop
+               Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.b = t2_4.b) AND (t1_4.c = t2_4.c))
+               ->  Seq Scan on alpha_pos_p3 t1_4
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Seq Scan on beta_pos_p3 t2_4
+                     Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+(29 rows)
+
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;
+ a  |  b  |  c   | a  |  b  |  c   
+----+-----+------+----+-----+------
+ -1 | 104 | 0004 | -1 | 104 | 0004
+ -1 | 109 | 0009 | -1 | 109 | 0009
+ -1 | 204 | 0004 | -1 | 204 | 0004
+ -1 | 209 | 0009 | -1 | 209 | 0009
+  1 | 104 | 0004 |  1 | 104 | 0004
+  1 | 109 | 0009 |  1 | 109 | 0009
+  1 | 204 | 0004 |  1 | 204 | 0004
+  1 | 209 | 0009 |  1 | 209 | 0009
+(8 rows)
+
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 575ba7b8d4..df30f851e8 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -445,11 +445,638 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
 
--- partitionwise join can not be applied if only one of joining tables has
--- default partition
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
-ANALYZE prt2;
 
+--
+-- Test advanced partition-matching algorithm for partitioned join
+--
+
+-- Tests for range-partitioned tables
+CREATE TABLE prt1_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_adv_p1 PARTITION OF prt1_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_adv_p2 PARTITION OF prt1_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_adv_p3 PARTITION OF prt1_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_adv_a_idx ON prt1_adv (a);
+INSERT INTO prt1_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_adv;
+
+CREATE TABLE prt2_adv (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_adv_p1 PARTITION OF prt2_adv FOR VALUES FROM (100) TO (150);
+CREATE TABLE prt2_adv_p2 PARTITION OF prt2_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt2_adv_p3 PARTITION OF prt2_adv FOR VALUES FROM (350) TO (500);
+CREATE INDEX prt2_adv_b_idx ON prt2_adv (b);
+INSERT INTO prt2_adv_p1 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 149) i;
+INSERT INTO prt2_adv_p2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(200, 299) i;
+INSERT INTO prt2_adv_p3 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_adv;
+
+-- inner join
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+-- Test cases where one side has an extra partition
+CREATE TABLE prt2_adv_extra PARTITION OF prt2_adv FOR VALUES FROM (500) TO (MAXVALUE);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(500, 599) i;
+ANALYZE prt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt1_adv t2 WHERE t1.b = t2.a) AND t1.a = 0 ORDER BY t1.b;
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+-- 3-way join where not every pair of relations can do partitioned join
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c, t3.a, t3.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) INNER JOIN prt1_adv t3 ON (t1.b = t3.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a, t3.a;
+SELECT t1.b, t1.c, t2.a, t2.c, t3.a, t3.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) INNER JOIN prt1_adv t3 ON (t1.b = t3.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a, t3.a;
+
+DROP TABLE prt2_adv_extra;
+
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in such cases
+ALTER TABLE prt2_adv DETACH PARTITION prt2_adv_p3;
+-- Split prt2_adv_p3 into two partitions so that prt1_adv_p3 matches both
+CREATE TABLE prt2_adv_p3_1 PARTITION OF prt2_adv FOR VALUES FROM (350) TO (375);
+CREATE TABLE prt2_adv_p3_2 PARTITION OF prt2_adv FOR VALUES FROM (375) TO (500);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+DROP TABLE prt2_adv_p3_1;
+DROP TABLE prt2_adv_p3_2;
+ANALYZE prt2_adv;
+
+-- Test default partitions
+ALTER TABLE prt1_adv DETACH PARTITION prt1_adv_p1;
+-- Change prt1_adv_p1 to the default partition
+ALTER TABLE prt1_adv ATTACH PARTITION prt1_adv_p1 DEFAULT;
+ALTER TABLE prt1_adv DETACH PARTITION prt1_adv_p3;
+ANALYZE prt1_adv;
+
+-- We can do partitioned join even if only one of relations has the default
+-- partition
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- Restore prt1_adv_p3
+ALTER TABLE prt1_adv ATTACH PARTITION prt1_adv_p3 FOR VALUES FROM (300) TO (400);
+ANALYZE prt1_adv;
+
+-- Restore prt2_adv_p3
+ALTER TABLE prt2_adv ATTACH PARTITION prt2_adv_p3 FOR VALUES FROM (350) TO (500);
+ANALYZE prt2_adv;
+
+-- Partitioned join can't be applied because the default partition of prt1_adv
+-- matches prt2_adv_p1 and prt2_adv_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+ALTER TABLE prt2_adv DETACH PARTITION prt2_adv_p3;
+-- Change prt2_adv_p3 to the default partition
+ALTER TABLE prt2_adv ATTACH PARTITION prt2_adv_p3 DEFAULT;
+ANALYZE prt2_adv;
+
+-- Partitioned join can't be applied because the default partition of prt1_adv
+-- matches prt2_adv_p1 and prt2_adv_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt1_adv_p3;
+ANALYZE prt1_adv;
+
+DROP TABLE prt2_adv_p3;
+ANALYZE prt2_adv;
+
+CREATE TABLE prt3_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt3_adv_p1 PARTITION OF prt3_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt3_adv_p2 PARTITION OF prt3_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt3_adv_a_idx ON prt3_adv (a);
+INSERT INTO prt3_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(200, 399) i;
+ANALYZE prt3_adv;
+
+-- 3-way join to test the default partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) LEFT JOIN prt3_adv t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) LEFT JOIN prt3_adv t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+
+DROP TABLE prt1_adv;
+DROP TABLE prt2_adv;
+DROP TABLE prt3_adv;
+
+-- Test interaction of partitioned join with partition pruning
+CREATE TABLE prt1_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_adv_p1 PARTITION OF prt1_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_adv_p2 PARTITION OF prt1_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_adv_p3 PARTITION OF prt1_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_adv_a_idx ON prt1_adv (a);
+INSERT INTO prt1_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_adv;
+
+CREATE TABLE prt2_adv (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_adv_p1 PARTITION OF prt2_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt2_adv_p2 PARTITION OF prt2_adv FOR VALUES FROM (200) TO (400);
+CREATE INDEX prt2_adv_b_idx ON prt2_adv (b);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt1_adv_p3;
+CREATE TABLE prt1_adv_default PARTITION OF prt1_adv DEFAULT;
+ANALYZE prt1_adv;
+
+CREATE TABLE prt2_adv_default PARTITION OF prt2_adv DEFAULT;
+ANALYZE prt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt1_adv;
+DROP TABLE prt2_adv;
+
+
+-- Tests for list-partitioned tables
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0001', '0003');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt1_adv_p3 PARTITION OF plt1_adv FOR VALUES IN ('0008', '0009');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+ANALYZE plt1_adv;
+
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt2_adv_p3 PARTITION OF plt2_adv FOR VALUES IN ('0007', '0009');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+-- Test cases where one side has an extra partition
+CREATE TABLE plt2_adv_extra PARTITION OF plt2_adv FOR VALUES IN ('0000');
+INSERT INTO plt2_adv_extra VALUES (0, 0, '0000');
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2_adv t1 LEFT JOIN plt1_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt2_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt1_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt2_adv_extra;
+
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in such cases
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2;
+-- Split plt2_adv_p2 into two partitions so that plt1_adv_p2 matches both
+CREATE TABLE plt2_adv_p2_1 PARTITION OF plt2_adv FOR VALUES IN ('0004');
+CREATE TABLE plt2_adv_p2_2 PARTITION OF plt2_adv FOR VALUES IN ('0006');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6);
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt2_adv_p2_1;
+DROP TABLE plt2_adv_p2_2;
+-- Restore plt2_adv_p2
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2 FOR VALUES IN ('0004', '0006');
+
+-- Test NULL partitions
+ALTER TABLE plt1_adv DETACH PARTITION plt1_adv_p1;
+-- Change plt1_adv_p1 to the NULL partition
+CREATE TABLE plt1_adv_p1_null PARTITION OF plt1_adv FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p3;
+-- Change plt2_adv_p3 to the NULL partition
+CREATE TABLE plt2_adv_p3_null PARTITION OF plt2_adv FOR VALUES IN (NULL, '0007', '0009');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (7, 9);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt1_adv_p1_null;
+-- Restore plt1_adv_p1
+ALTER TABLE plt1_adv ATTACH PARTITION plt1_adv_p1 FOR VALUES IN ('0001', '0003');
+-- Add to plt1_adv the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt1_adv_extra PARTITION OF plt1_adv FOR VALUES IN (NULL);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+
+DROP TABLE plt2_adv_p3_null;
+-- Restore plt2_adv_p3
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p3 FOR VALUES IN ('0007', '0009');
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+-- Add to plt2_adv the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt2_adv_extra PARTITION OF plt2_adv FOR VALUES IN (NULL);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+-- 3-way join to test the NULL partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt1_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt1_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt1_adv_extra;
+DROP TABLE plt2_adv_extra;
+
+-- Test default partitions
+ALTER TABLE plt1_adv DETACH PARTITION plt1_adv_p1;
+-- Change plt1_adv_p1 to the default partition
+ALTER TABLE plt1_adv ATTACH PARTITION plt1_adv_p1 DEFAULT;
+DROP TABLE plt1_adv_p3;
+ANALYZE plt1_adv;
+
+DROP TABLE plt2_adv_p3;
+ANALYZE plt2_adv;
+
+-- We can do partitioned join even if only one of relations has the default
+-- partition
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2;
+-- Change plt2_adv_p2 to contain '0005' in addition to '0004' and '0006' as
+-- the key values
+CREATE TABLE plt2_adv_p2_ext PARTITION OF plt2_adv FOR VALUES IN ('0004', '0005', '0006');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 5, 6);
+ANALYZE plt2_adv;
+
+-- Partitioned join can't be applied because the default partition of plt1_adv
+-- matches plt2_adv_p1 and plt2_adv_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2_ext;
+-- Change plt2_adv_p2_ext to the default partition
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2_ext DEFAULT;
+ANALYZE plt2_adv;
+
+-- Partitioned join can't be applied because the default partition of plt1_adv
+-- matches plt2_adv_p1 and plt2_adv_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt2_adv_p2_ext;
+-- Restore plt2_adv_p2
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_adv;
+
+CREATE TABLE plt3_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_adv_p1 PARTITION OF plt3_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt3_adv_p2 PARTITION OF plt3_adv FOR VALUES IN ('0007', '0009');
+INSERT INTO plt3_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6, 7, 9);
+ANALYZE plt3_adv;
+
+-- 3-way join to test the default partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Test cases where one side has the default partition while the other side
+-- has the NULL partition
+DROP TABLE plt2_adv_p1;
+-- Add the NULL partition to plt2_adv
+CREATE TABLE plt2_adv_p1_null PARTITION OF plt2_adv FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt2_adv_p1_null;
+-- Add the NULL partition that contains only NULL values as the key values
+CREATE TABLE plt2_adv_p1_null PARTITION OF plt2_adv FOR VALUES IN (NULL);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+DROP TABLE plt3_adv;
+
+-- Test interaction of partitioned join with partition pruning
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0001');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0002');
+CREATE TABLE plt1_adv_p3 PARTITION OF plt1_adv FOR VALUES IN ('0003');
+CREATE TABLE plt1_adv_p4 PARTITION OF plt1_adv FOR VALUES IN (NULL, '0004', '0005');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN (NULL);
+CREATE TABLE plt2_adv_p3 PARTITION OF plt2_adv FOR VALUES IN ('0003');
+CREATE TABLE plt2_adv_p4 PARTITION OF plt2_adv FOR VALUES IN ('0004', '0005');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+
+CREATE TABLE plt1_adv_default PARTITION OF plt1_adv DEFAULT;
+ANALYZE plt1_adv;
+
+CREATE TABLE plt2_adv_default PARTITION OF plt2_adv DEFAULT;
+ANALYZE plt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+
+-- Test the process_outer_partition() code path
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1_adv;
+
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0002');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2_adv;
+
+CREATE TABLE plt3_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_adv_p1 PARTITION OF plt3_adv FOR VALUES IN ('0001');
+CREATE TABLE plt3_adv_p2 PARTITION OF plt3_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3_adv;
+
+-- This tests that when merging partitions from plt1_adv and plt2_adv in
+-- merge_list_bounds(), process_outer_partition() returns an already-assigned
+-- merged partition when re-called with plt1_adv_p1 for the second list value
+-- '0001' of that partitin
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.c = t2.c)) FULL JOIN plt3_adv t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.c = t2.c)) FULL JOIN plt3_adv t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+DROP TABLE plt3_adv;
+
+
+-- Tests for multi-level partitioned tables
+CREATE TABLE alpha (a double precision, b int, c text) PARTITION BY RANGE (a);
+CREATE TABLE alpha_neg PARTITION OF alpha FOR VALUES FROM ('-Infinity') TO (0) PARTITION BY RANGE (b);
+CREATE TABLE alpha_pos PARTITION OF alpha FOR VALUES FROM (0) TO (10.0) PARTITION BY LIST (c);
+CREATE TABLE alpha_neg_p1 PARTITION OF alpha_neg FOR VALUES FROM (100) TO (200);
+CREATE TABLE alpha_neg_p2 PARTITION OF alpha_neg FOR VALUES FROM (200) TO (300);
+CREATE TABLE alpha_neg_p3 PARTITION OF alpha_neg FOR VALUES FROM (300) TO (400);
+CREATE TABLE alpha_pos_p1 PARTITION OF alpha_pos FOR VALUES IN ('0001', '0003');
+CREATE TABLE alpha_pos_p2 PARTITION OF alpha_pos FOR VALUES IN ('0004', '0006');
+CREATE TABLE alpha_pos_p3 PARTITION OF alpha_pos FOR VALUES IN ('0008', '0009');
+INSERT INTO alpha_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+INSERT INTO alpha_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+ANALYZE alpha;
+
+CREATE TABLE beta (a double precision, b int, c text) PARTITION BY RANGE (a);
+CREATE TABLE beta_neg PARTITION OF beta FOR VALUES FROM (-10.0) TO (0) PARTITION BY RANGE (b);
+CREATE TABLE beta_pos PARTITION OF beta FOR VALUES FROM (0) TO ('Infinity') PARTITION BY LIST (c);
+CREATE TABLE beta_neg_p1 PARTITION OF beta_neg FOR VALUES FROM (100) TO (150);
+CREATE TABLE beta_neg_p2 PARTITION OF beta_neg FOR VALUES FROM (200) TO (300);
+CREATE TABLE beta_neg_p3 PARTITION OF beta_neg FOR VALUES FROM (350) TO (500);
+CREATE TABLE beta_pos_p1 PARTITION OF beta_pos FOR VALUES IN ('0002', '0003');
+CREATE TABLE beta_pos_p2 PARTITION OF beta_pos FOR VALUES IN ('0004', '0006');
+CREATE TABLE beta_pos_p3 PARTITION OF beta_pos FOR VALUES IN ('0007', '0009');
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+ANALYZE beta;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b;
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b, t2.b;
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b, t2.b;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;
#125Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: Etsuro Fujita (#124)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Hi,

I've started reviewing the patch a couple days ago. I haven't done any
extensive testing, but I do have a bunch of initial comments that I can
share now.

1) I wonder if this needs to update src/backend/optimizer/README, which
does have a section about partitionwise joins. It seems formulated in a
way that that probably covers even this more advanced algorithm, but
maybe it should mention handling of default partitions etc.?

There certainly needs to be some description of the algorithm somewhere,
either in a README or before a suitable function. It doesn't have to be
particularly detailed, a rough outline of the matching would be enough,
so that readers don't have to rebuild the knowledge from pieces
scattered around various comments.

2) Do we need another GUC enabling this more complex algorithm? In PG11
the partitionwise join is disabled by default, on the grounds that it's
expensive and not worth it by default. How much more expensive is this?
Maybe it makes sense to allow enabling only the "simple" approach?

3) This comment in try_partitionwise_join is now incorrect, because the
condition may be true even for partitioned tables with (nparts == 0).

/* Nothing to do, if the join relation is not partitioned. */
if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
return;

Moreover, the condition used to be

if (!IS_PARTITIONED_REL(joinrel))
return;

which is way more readable. I think it's net negative to replace these
"nice" macros with clear meaning with complex conditions. If needed, we
can invent new macros. There are many other places where the patch
replaces macros with less readable conditions.

4) I'm a bit puzzled how we could get here with non-partitioned rels?

/*
* We can not perform partitionwise join if either of the joining relations
* is not partitioned.
*/
if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
return;

5) I find the "merged" flag in RelOptInfo rather unclear, because it
does not clearly indicate what was merged. Maybe something like
partbounds_merged would be better?

6) The try_partitionwise_join function is getting a bit too long and
harder to understand. The whole block in

if (joinrel->nparts == -1)
{
...
}

seems rather well isolated, so I propose to move it to a separate
function responsible only for the merging. We can simply call it on the
joinrel, and make it return right away if (joinrel->nparts == -1).

7) I'd suggest not to reference exact functions in comments unless
abolutely necessary, because it's harder to maintain and it does not
really explain purpose of the struct/code. E.g. consider this:

/* Per-partitioned-relation data for merge_list_bounds()/merge_range_bounds() */
typedef struct PartitionMap
{ ... }

Why does it matter where is the struct used? That's pretty trivial to
find using 'git grep' or something. Instead the comment should explain
the purpose of the struct.

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#126Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: Tomas Vondra (#125)
3 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Hi,

three more comments after eye-balling the code for a bit longer.

1) The patch probably needs to tweak config.sgml which says this about
the enable_partitionwise_join GUC:

.. Partitionwise join currently applies only when the join conditions
include all the partition keys, which must be of the same data type
and have exactly matching sets of child partitions. ..

Which is probably incorrect, because the point of this patch is not to
require exact match of the partitions, right?

2) Do we really need the 'merged' flag in try_partitionwise_join? Can't
we simply use the joinrel->merged flag directly? ISTM the we always
start with joinrel->merged=false, and then only ever set it to true in
some cases. I've tried doing that, per the attached 0002 patch. The
regression tests seem to work fine.

I noticed this because I've tried moving part of the function into a
separate function, and removing the variable makes that simpler.

The patch also does a couple additional minor tweaks.

3) I think the for nparts comment is somewhat misleading:

int nparts; /* number of partitions; 0 = not partitioned;
* -1 = not yet set */

which says that nparts=0 means not partitioned. But then there are
conditions like this:

/* Nothing to do, if the join relation is not partitioned. */
if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
return;

which (ignoring the obsolete comment) seems to say we can have nparts==0
even for partitioned tables, no?

Anyway, attached is the original patch (0001) and two patches with
proposed changes. 0002 removes the "merged" flag as explained in (2),
0003 splits the try_partitionwise_join() function into two parts.

I'm saying these changes have to happen and it's a bit crude (and it
might be a bit of a bikeshedding).

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

Attachments:

0001-v33.patchtext/plain; charset=us-asciiDownload
From 24a86111f31fc3e95c02dc3c75f1be390ebbd09b Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tv@fuzzy.cz>
Date: Wed, 25 Mar 2020 22:43:28 +0100
Subject: [PATCH 1/3] v33

---
 src/backend/nodes/outfuncs.c                 |    2 +
 src/backend/optimizer/path/joinrels.c        |  242 +-
 src/backend/optimizer/util/inherit.c         |    2 +
 src/backend/optimizer/util/relnode.c         |   43 +-
 src/backend/partitioning/partbounds.c        | 1838 ++++++++++++
 src/include/nodes/pathnodes.h                |    8 +-
 src/include/partitioning/partbounds.h        |    9 +
 src/test/regress/expected/partition_join.out | 2608 +++++++++++++++++-
 src/test/regress/sql/partition_join.sql      |  639 ++++-
 9 files changed, 5315 insertions(+), 76 deletions(-)

diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index e084c3f069..79c768409c 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -2288,6 +2288,8 @@ _outRelOptInfo(StringInfo str, const RelOptInfo *node)
 	WRITE_BOOL_FIELD(has_eclass_joins);
 	WRITE_BOOL_FIELD(consider_partitionwise_join);
 	WRITE_BITMAPSET_FIELD(top_parent_relids);
+	WRITE_BOOL_FIELD(merged);
+	WRITE_BITMAPSET_FIELD(all_partrels);
 	WRITE_NODE_FIELD(partitioned_child_rels);
 }
 
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index a21c295b99..530ebed245 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -48,6 +48,9 @@ static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 												Relids left_relids, Relids right_relids);
 static int	match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel,
 										 bool strict_op);
+static void get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2);
 
 
 /*
@@ -1357,25 +1360,30 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	bool		merged = false;
+	List	   *parts1 = NIL;
+	List	   *parts2 = NIL;
+	ListCell   *lcr1 = NULL;
+	ListCell   *lcr2 = NULL;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partitionwise join if either of the joining relations
+	 * is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1390,34 +1398,104 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		   joinrel->part_scheme == rel2->part_scheme);
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * If we don't have the partition bounds for the join rel yet, try to
+	 * compute those along with pairs of partitions to be joined.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	if (joinrel->nparts == -1)
+	{
+		PartitionScheme part_scheme = joinrel->part_scheme;
+		PartitionBoundInfo boundinfo = NULL;
+		int			nparts = 0;
+
+		Assert(joinrel->boundinfo == NULL);
+		Assert(joinrel->part_rels == NULL);
+
+		/*
+		 * See if the partition bounds for inputs are exactly the same, in
+		 * which case we don't need to work hard: the join rel have the same
+		 * partition bounds as inputs, and the partitions with the same
+		 * cardinal positions form the pairs.
+		 *
+		 * Note: even in cases where one or both inputs have merged bounds,
+		 * it would be possible for both the bounds to be exactly the same, but
+		 * it seems unlikely to be worth the cycles to check.
+		 */
+		if (!rel1->merged &&
+			!rel2->merged &&
+			rel1->nparts == rel2->nparts &&
+			partition_bounds_equal(part_scheme->partnatts,
+								   part_scheme->parttyplen,
+								   part_scheme->parttypbyval,
+								   rel1->boundinfo, rel2->boundinfo))
+		{
+			boundinfo = rel1->boundinfo;
+			nparts = rel1->nparts;
+		}
+		else
+		{
+			/* Try merging the partition bounds for inputs. */
+			boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											   part_scheme->partsupfunc,
+											   part_scheme->partcollation,
+											   rel1, rel2,
+											   parent_sjinfo->jointype,
+											   &parts1, &parts2);
+			if (boundinfo == NULL)
+			{
+				joinrel->nparts = 0;
+				return;
+			}
+			nparts = list_length(parts1);
+			merged = true;
+		}
+
+		Assert(nparts > 0);
+		joinrel->boundinfo = boundinfo;
+		joinrel->merged = merged;
+		joinrel->nparts = nparts;
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+	}
+	else
+	{
+		Assert(joinrel->nparts > 0);
+		Assert(joinrel->boundinfo);
+		Assert(joinrel->part_rels);
+
+		/*
+		 * If the join rel's merged flag is true, it means inputs are not
+		 * guaranteed to have the same partition bounds, therefore we can't
+		 * assume that the partitions at the same cardinal positions form the
+		 * pairs; let get_matching_part_pairs() generate the pairs.  Otherwise,
+		 * nothing to do since we can assume that.
+		 */
+		if (joinrel->merged)
+		{
+			get_matching_part_pairs(root, joinrel, rel1, rel2,
+									&parts1, &parts2);
+			Assert(list_length(parts1) == joinrel->nparts);
+			Assert(list_length(parts2) == joinrel->nparts);
+			merged = true;
+		}
+	}
 
-	nparts = joinrel->nparts;
+	if (merged)
+	{
+		lcr1 = list_head(parts1);
+		lcr2 = list_head(parts2);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
-		bool		rel1_empty = (child_rel1 == NULL ||
-								  IS_DUMMY_REL(child_rel1));
-		bool		rel2_empty = (child_rel2 == NULL ||
-								  IS_DUMMY_REL(child_rel2));
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		bool		rel1_empty;
+		bool		rel2_empty;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1425,6 +1503,22 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		if (merged)
+		{
+			child_rel1 = lfirst_node(RelOptInfo, lcr1);
+			child_rel2 = lfirst_node(RelOptInfo, lcr2);
+			lcr1 = lnext(parts1, lcr1);
+			lcr2 = lnext(parts2, lcr2);
+		}
+		else
+		{
+			child_rel1 = rel1->part_rels[cnt_parts];
+			child_rel2 = rel2->part_rels[cnt_parts];
+		}
+
+		rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1));
+		rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2));
+
 		/*
 		 * Check for cases where we can prove that this segment of the join
 		 * returns no rows, due to one or both inputs being empty (including
@@ -1522,6 +1616,8 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 												 child_sjinfo,
 												 child_sjinfo->jointype);
 			joinrel->part_rels[cnt_parts] = child_joinrel;
+			joinrel->all_partrels = bms_add_members(joinrel->all_partrels,
+													child_joinrel->relids);
 		}
 
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
@@ -1738,3 +1834,99 @@ match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op)
 
 	return -1;
 }
+
+/*
+ * get_matching_part_pairs
+ *		Generate pairs of partitions to be joined from the two inputs
+ */
+static void
+get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2)
+{
+	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
+	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
+	int 		cnt_parts;
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
+	{
+		RelOptInfo *child_joinrel = joinrel->part_rels[cnt_parts];
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		Relids		child_relids1;
+		Relids		child_relids2;
+
+		/*
+		 * If this segment of the join is empty, it means that this segment
+		 * was ignored when previously creating child-join paths for it in
+		 * try_partitionwise_join() as it would not contribute to the join
+		 * result, due to one or both inputs being empty; add NULL to each of
+		 * the given lists so that this segment will be ignored again in that
+		 * function.
+		 */
+		if (!child_joinrel)
+		{
+			*parts1 = lappend(*parts1, NULL);
+			*parts2 = lappend(*parts2, NULL);
+			continue;
+		}
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel1 side.
+		 */
+		child_relids1 = bms_intersect(child_joinrel->relids,
+									  rel1->all_partrels);
+		Assert(bms_num_members(child_relids1) == bms_num_members(rel1->relids));
+
+		/*
+		 * Get a child rel for rel1 with the relids.  Note that we should have
+		 * the child rel even if rel1 is a join rel, because in that case the
+		 * partitions specified in the relids would have matching/overlapping
+		 * boundaries, so the specified partitions should be considered as ones
+		 * to be joined when planning partitionwise joins of rel1, meaning that
+		 * the child rel would have been built by the time we get here.
+		 */
+		if (rel1_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids1);
+
+			child_rel1 = find_base_rel(root, varno);
+		}
+		else
+			child_rel1 = find_join_rel(root, child_relids1);
+		Assert(child_rel1);
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel2 side.
+		 */
+		child_relids2 = bms_intersect(child_joinrel->relids,
+									  rel2->all_partrels);
+		Assert(bms_num_members(child_relids2) == bms_num_members(rel2->relids));
+
+		/*
+		 * Get a child rel for rel2 with the relids.  See above comments.
+		 */
+		if (rel2_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids2);
+
+			child_rel2 = find_base_rel(root, varno);
+		}
+		else
+			child_rel2 = find_join_rel(root, child_relids2);
+		Assert(child_rel2);
+
+		/*
+		 * The join of rel1 and rel2 is legal, so is the join of the child
+		 * rels obtained above; add them to the given lists as a join pair
+		 * producing this join segment.
+		 */
+		*parts1 = lappend(*parts1, child_rel1);
+		*parts2 = lappend(*parts2, child_rel2);
+	}
+}
diff --git a/src/backend/optimizer/util/inherit.c b/src/backend/optimizer/util/inherit.c
index 7db67fdf34..3132fd35a5 100644
--- a/src/backend/optimizer/util/inherit.c
+++ b/src/backend/optimizer/util/inherit.c
@@ -376,6 +376,8 @@ expand_partitioned_rtentry(PlannerInfo *root, RelOptInfo *relinfo,
 		/* Create the otherrel RelOptInfo too. */
 		childrelinfo = build_simple_rel(root, childRTindex, relinfo);
 		relinfo->part_rels[i] = childrelinfo;
+		relinfo->all_partrels = bms_add_members(relinfo->all_partrels,
+												childrelinfo->relids);
 
 		/* If this child is itself partitioned, recurse */
 		if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 374f93890b..0e4944ac8e 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -240,10 +240,12 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->has_eclass_joins = false;
 	rel->consider_partitionwise_join = false;	/* might get changed later */
 	rel->part_scheme = NULL;
-	rel->nparts = 0;
+	rel->nparts = -1;
 	rel->boundinfo = NULL;
+	rel->merged = false;
 	rel->partition_qual = NIL;
 	rel->part_rels = NULL;
+	rel->all_partrels = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->partitioned_child_rels = NIL;
@@ -653,10 +655,12 @@ build_join_rel(PlannerInfo *root,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -829,10 +833,12 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -1639,7 +1645,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1652,24 +1658,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1680,18 +1668,17 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 		   !joinrel->boundinfo);
 
 	/*
-	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * If the join relation is partitioned, it use the same partitioning scheme
+	 * as the joining relations.
+	 *
+	 * Note: we calculate the partition bounds, number of partitions, and
+	 * child-join relations of the join relation in try_partitionwise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 4c47f54a57..24dbc2c8f3 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -26,6 +26,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/pathnodes.h"
 #include "parser/parse_coerce.h"
 #include "partitioning/partbounds.h"
 #include "partitioning/partdesc.h"
@@ -68,6 +69,25 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+/* Per-partitioned-relation data for merge_list_bounds()/merge_range_bounds() */
+typedef struct PartitionMap
+{
+	int			nparts;			/* number of partitions */
+	int		   *merged_indexes;	/* indexes of merged partitions */
+	bool	   *merged;			/* flags to indicate whether partitions are
+								 * merged with non-dummy partitions */
+	bool		did_remapping;	/* did we re-map partitions? */
+	int		   *old_indexes;	/* old indexes of merged partitions if
+								 * did_remapping */
+} PartitionMap;
+
+/* Macro for comparing two range bounds */
+#define compare_range_bounds(partnatts, partsupfunc, partcollations, \
+							 bound1, bound2) \
+	(partition_rbound_cmp(partnatts, partsupfunc, partcollations, \
+						  (bound1)->datums, (bound1)->kind, (bound1)->lower, \
+						  bound2))
+
 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);
@@ -79,6 +99,116 @@ static PartitionBoundInfo create_list_bounds(PartitionBoundSpec **boundspecs,
 											 int nparts, PartitionKey key, int **mapping);
 static PartitionBoundInfo create_range_bounds(PartitionBoundSpec **boundspecs,
 											  int nparts, PartitionKey key, int **mapping);
+static PartitionBoundInfo merge_list_bounds(FmgrInfo *partsupfunc,
+											Oid *collations,
+											RelOptInfo *outer_rel,
+											RelOptInfo *inner_rel,
+											JoinType jointype,
+											List **outer_parts,
+											List **inner_parts);
+static PartitionBoundInfo merge_range_bounds(int partnatts,
+											 FmgrInfo *partsupfuncs,
+											 Oid *partcollations,
+											 RelOptInfo *outer_rel,
+											 RelOptInfo *inner_rel,
+											 JoinType jointype,
+											 List **outer_parts,
+											 List **inner_parts);
+static void init_partition_map(RelOptInfo *rel, PartitionMap *map);
+static void free_partition_map(PartitionMap *map);
+static bool is_dummy_partition(RelOptInfo *rel, int part_index);
+static int merge_matching_partitions(PartitionMap *outer_map,
+									 PartitionMap *inner_map,
+									 int outer_part,
+									 int inner_part,
+									 int *next_index);
+static int process_outer_partition(PartitionMap *outer_map,
+								   PartitionMap *inner_map,
+								   bool outer_has_default,
+								   bool inner_has_default,
+								   int outer_index,
+								   int inner_default,
+								   JoinType jointype,
+								   int *next_index,
+								   int *default_index);
+static int process_inner_partition(PartitionMap *outer_map,
+								   PartitionMap *inner_map,
+								   bool outer_has_default,
+								   bool inner_has_default,
+								   int inner_index,
+								   int outer_default,
+								   JoinType jointype,
+								   int *next_index,
+								   int *default_index);
+static void merge_null_partitions(PartitionMap *outer_map,
+								  PartitionMap *inner_map,
+								  bool outer_has_null,
+								  bool inner_has_null,
+								  int outer_null,
+								  int inner_null,
+								  JoinType jointype,
+								  int *next_index,
+								  int *null_index);
+static void merge_default_partitions(PartitionMap *outer_map,
+									 PartitionMap *inner_map,
+									 bool outer_has_default,
+									 bool inner_has_default,
+									 int outer_default,
+									 int inner_default,
+									 JoinType jointype,
+									 int *next_index,
+									 int *default_index);
+static int merge_partition_with_dummy(PartitionMap *map, int index,
+									  int *next_index);
+static void fix_merged_indexes(PartitionMap *outer_map,
+							   PartitionMap *inner_map,
+							   int nmerged, List *merged_indexes);
+static void generate_matching_part_pairs(RelOptInfo *outer_rel,
+										 RelOptInfo *inner_rel,
+										 PartitionMap *outer_map,
+										 PartitionMap *inner_map,
+										 int nmerged,
+										 List **outer_parts,
+										 List **inner_parts);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+														List *merged_datums,
+														List *merged_kinds,
+														List *merged_indexes,
+														int null_index,
+														int default_index);
+static int get_range_partition(RelOptInfo *rel,
+							   PartitionBoundInfo bi,
+							   int *lb_pos,
+							   PartitionRangeBound *lb,
+							   PartitionRangeBound *ub);
+static int get_range_partition_internal(PartitionBoundInfo bi,
+										int *lb_pos,
+										PartitionRangeBound *lb,
+										PartitionRangeBound *ub);
+static bool compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+									 Oid *partcollations,
+									 PartitionRangeBound *outer_lb,
+									 PartitionRangeBound *outer_ub,
+									 PartitionRangeBound *inner_lb,
+									 PartitionRangeBound *inner_ub,
+									 int *lb_cmpval, int *ub_cmpval);
+static void get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+									Oid *partcollations, JoinType jointype,
+									PartitionRangeBound *outer_lb,
+									PartitionRangeBound *outer_ub,
+									PartitionRangeBound *inner_lb,
+									PartitionRangeBound *inner_ub,
+									int	lb_cmpval, int ub_cmpval,
+									PartitionRangeBound *merged_lb,
+									PartitionRangeBound *merged_ub);
+static void add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+									Oid *partcollations,
+									PartitionRangeBound *merged_lb,
+									PartitionRangeBound *merged_ub,
+									int merged_index,
+									List **merged_datums,
+									List **merged_kinds,
+									List **merged_indexes);
 static PartitionRangeBound *make_one_partition_rbound(PartitionKey key, int index,
 													  List *datums, bool lower);
 static int32 partition_hbound_cmp(int modulus1, int remainder1, int modulus2,
@@ -866,6 +996,1714 @@ partition_bounds_copy(PartitionBoundInfo src,
 	return dest;
 }
 
+/*
+ * partition_bounds_merge
+ *
+ * This function builds and returns the partition bounds for a join relation
+ * between input relations, creating two lists of partitions, which are
+ * returned to *outer_parts and *inner_parts respectively.  The lists contain
+ * the same number of partitions, and the partitions at the same positions in
+ * the lists indicate join pairs used for partitioned join.
+ *
+ * This function returns NULL, setting *outer_parts and *inner_parts to NIL,
+ * if a partition on one side matches multiple partitions on the other side,
+ * in which case we currently don't support partitioned join.
+ */
+PartitionBoundInfo
+partition_bounds_merge(int partnatts,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype,
+					   List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo outer_binfo = outer_rel->boundinfo;
+	PartitionBoundInfo inner_binfo = inner_rel->boundinfo;
+	char		strategy;
+
+	/*
+	 * Currently, this function is called only from try_partitionwise_join(),
+	 * so the join type should be INNER, LEFT, FULL, SEMI, or ANTI.
+	 */
+	if (jointype != JOIN_INNER && jointype != JOIN_LEFT && 
+		jointype != JOIN_FULL && jointype != JOIN_SEMI &&
+		jointype != JOIN_ANTI)
+		elog(ERROR, "unrecognized join type: %d", (int) jointype);
+
+	/* Bail out if the partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	strategy = outer_binfo->strategy;
+	*outer_parts = *inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+
+			/*
+			 * For hash partitioned tables, we currently support partitioned
+			 * join only when the partition bounds for them exactly match.
+			 *
+			 * XXX: it might be possible to relax the restriction to support
+			 * cases where hash partitioned tables have missing partitions
+			 * and/or different moduli, but it's not clear if it would be
+			 * useful to support the former case since it's unusual to have
+			 * missing partitions.  On the other hand, it would be useful to
+			 * support the latter case, but in that case, there is a high
+			 * probability that a partition on one side will match multiple
+			 * partitions on the other side, which is the scenario the current
+			 * implementation of partitioned join can't handle.
+			 */
+			return NULL;
+
+		case PARTITION_STRATEGY_LIST:
+			return merge_list_bounds(partsupfunc,
+									 partcollation,
+									 outer_rel,
+									 inner_rel,
+									 jointype,
+									 outer_parts,
+									 inner_parts);
+
+		case PARTITION_STRATEGY_RANGE:
+			return merge_range_bounds(partnatts,
+									  partsupfunc,
+									  partcollation,
+									  outer_rel,
+									  inner_rel,
+									  jointype,
+									  outer_parts,
+									  inner_parts);
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", (int) strategy);
+			return NULL;				/* keep compiler quiet */
+	}
+}
+
+/*
+ * merge_list_bounds
+ *		Create the partition bounds for a join relation between list
+ *		partitioned tables, if possible
+ */
+static PartitionBoundInfo
+merge_list_bounds(FmgrInfo *partsupfunc, Oid *partcollation,
+				  RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+				  JoinType jointype,
+				  List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			outer_pos;
+	int			inner_pos;
+	int			next_index = 0;
+	int			null_index = -1;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+	/* List partitioning doesn't require kinds. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * If the default partitions (if any) have been proven empty, deem them
+	 * non-existent.
+	 */
+	if (outer_has_default && is_dummy_partition(outer_rel, outer_default))
+		outer_has_default = false;
+	if (inner_has_default && is_dummy_partition(inner_rel, inner_default))
+		inner_has_default = false;
+
+	/*
+	 * Merge partitions from both sides.  In each iteration we compare a pair
+	 * of list values, one from each side, and decide whether the corresponding
+	 * partitions match or not.  If the two values match exactly, move to the
+	 * next pair of list values, otherwise move to the next list value on the
+	 * side with a smaller list value.
+	 */
+	outer_pos = inner_pos = 0;
+	while (outer_pos < outer_bi->ndatums || inner_pos < inner_bi->ndatums)
+	{
+		int			outer_index = -1;
+		int			inner_index = -1;
+		Datum	   *outer_datums;
+		Datum	   *inner_datums;
+		int			cmpval;
+		Datum	   *merged_datum = NULL;
+		int			merged_index = -1;
+
+		if (outer_pos < outer_bi->ndatums)
+		{
+			/*
+			 * If the partition on the outer side has been proven empty, ignore
+			 * it and move to the next datum on the outer side.
+			 */
+			outer_index = outer_bi->indexes[outer_pos];
+			if (is_dummy_partition(outer_rel, outer_index))
+			{
+				outer_pos++;
+				continue;
+			}
+		}
+		if (inner_pos < inner_bi->ndatums)
+		{
+			/*
+			 * If the partition on the inner side has been proven empty, ignore
+			 * it and move to the next datum on the inner side.
+			 */
+			inner_index = inner_bi->indexes[inner_pos];
+			if (is_dummy_partition(inner_rel, inner_index))
+			{
+				inner_pos++;
+				continue;
+			}
+		}
+
+		/* Get the list values. */
+		outer_datums = outer_pos < outer_bi->ndatums ?
+			outer_bi->datums[outer_pos] : NULL;
+		inner_datums = inner_pos < inner_bi->ndatums ?
+			inner_bi->datums[inner_pos] : NULL;
+
+		/*
+		 * We run this loop till both sides finish.  This allows us to avoid
+		 * duplicating code to handle the remaining values on the side which
+		 * finishes later.  For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra value higher than any other value on the unfinished side.
+		 * That way we advance the values on the unfinished side till all of
+		 * its values are exhausted.
+		 */
+		if (outer_pos >= outer_bi->ndatums)
+			cmpval = 1;
+		else if (inner_pos >= inner_bi->ndatums)
+			cmpval = -1;
+		else
+		{
+			Assert(outer_datums != NULL && inner_datums != NULL);
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 outer_datums[0],
+													 inner_datums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			/* Two list values match exactly. */
+			Assert(outer_pos < outer_bi->ndatums);
+			Assert(inner_pos < inner_bi->ndatums);
+			Assert(outer_index >= 0);
+			Assert(inner_index >= 0);
+
+			/*
+			 * Try merging both paritions.  If successful, add the list value
+			 * and index of the merged partition below.
+			 */
+			merged_index = merge_matching_partitions(&outer_map, &inner_map,
+													 outer_index, inner_index,
+													 &next_index);
+			if (merged_index == -1)
+				goto cleanup;
+
+			merged_datum = outer_datums;
+
+			/* Move to the next pair of list values. */
+			outer_pos++;
+			inner_pos++;
+		}
+		else if (cmpval < 0)
+		{
+			/* A list value missing from the inner side. */
+			Assert(outer_pos < outer_bi->ndatums);
+
+			/*
+			 * If the inner side has the default partition, or this is an outer
+			 * join, try to assign a merged partition to the outer partition
+			 * (see process_outer_partition()).  Otherwise, the outer partition
+			 * will not contribute to the result.
+			 */
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				/* Get the outer partition. */
+				outer_index = outer_bi->indexes[outer_pos];
+				Assert(outer_index >= 0);
+				merged_index = process_outer_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   outer_index,
+													   inner_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+				merged_datum = outer_datums;
+			}
+
+			/* Move to the next list value on the outer side. */
+			outer_pos++;
+		}
+		else
+		{
+			/* A list value missing from the outer side. */
+			Assert(cmpval > 0);
+			Assert(inner_pos < inner_bi->ndatums);
+
+			/*
+			 * If the outer side has the default partition, or this is a FULL
+			 * join, try to assign a merged partition to the inner partition
+			 * (see process_inner_partition()).  Otherwise, the inner partition
+			 * will not contribute to the result.
+			 */
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				/* Get the inner partition. */
+				inner_index = inner_bi->indexes[inner_pos];
+				Assert(inner_index >= 0);
+				merged_index = process_inner_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   inner_index,
+													   outer_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+				merged_datum = inner_datums;
+			}
+
+			/* Move to the next list value on the inner side. */
+			inner_pos++;
+		}
+
+		/*
+		 * If we assigned a merged partition, add the list value and index of
+		 * the merged partition if appropriate.
+		 */
+		if (merged_index >= 0 && merged_index != default_index)
+		{
+			merged_datums = lappend(merged_datums, merged_datum);
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+		}
+	}
+
+	/*
+	 * If the NULL partitions (if any) have been proven empty, deem them
+	 * non-existent.
+	 */
+	if (outer_has_null &&
+		is_dummy_partition(outer_rel, outer_bi->null_index))
+		outer_has_null = false;
+	if (inner_has_null &&
+		is_dummy_partition(inner_rel, inner_bi->null_index))
+		inner_has_null = false;
+
+	/* Merge the NULL partitions if any. */
+	if (outer_has_null || inner_has_null)
+		merge_null_partitions(&outer_map, &inner_map,
+							  outer_has_null, inner_has_null,
+							  outer_bi->null_index, inner_bi->null_index,
+							  jointype, &next_index, &null_index);
+	else
+		Assert(null_index == -1);
+
+	/* Merge the default partitions if any. */
+	if (outer_has_default || inner_has_default)
+		merge_default_partitions(&outer_map, &inner_map,
+								 outer_has_default, inner_has_default,
+								 outer_default, inner_default,
+								 jointype, &next_index, &default_index);
+	else
+		Assert(default_index == -1);
+
+	/* If we have merged partitions, create the partition bounds. */
+	if (next_index > 0)
+	{
+		/* Fix the merged_indexes list if necessary. */
+		if (outer_map.did_remapping || inner_map.did_remapping)
+		{
+			Assert(jointype == JOIN_FULL);
+			fix_merged_indexes(&outer_map, &inner_map,
+							   next_index, merged_indexes);
+		}
+
+		/* Use maps to match partitions from inputs. */
+		generate_matching_part_pairs(outer_rel, inner_rel,
+									 &outer_map, &inner_map,
+									 next_index,
+									 outer_parts, inner_parts);
+		Assert(*outer_parts != NIL);
+		Assert(*inner_parts != NIL);
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) <= next_index);
+
+		/* Make a PartitionBoundInfo struct to return. */
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  NIL,
+													  merged_indexes,
+													  null_index,
+													  default_index);
+		Assert(merged_bounds);
+	}
+
+cleanup:
+	/* Free local memory before returning. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+
+	return merged_bounds;
+}
+
+/*
+ * merge_range_bounds
+ *		Create the partition bounds for a join relation between range
+ *		partitioned tables, if possible
+ */
+static PartitionBoundInfo
+merge_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+				   Oid *partcollations,
+				   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+				   JoinType jointype,
+				   List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			outer_index;
+	int			inner_index;
+	int			outer_lb_pos;
+	int			inner_lb_pos;
+	PartitionRangeBound outer_lb;
+	PartitionRangeBound outer_ub;
+	PartitionRangeBound inner_lb;
+	PartitionRangeBound inner_ub;
+	int			next_index = 0;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_kinds = NIL;
+	List	   *merged_indexes = NIL;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * If the default partitions (if any) have been proven empty, deem them
+	 * non-existent.
+	 */
+	if (outer_has_default && is_dummy_partition(outer_rel, outer_default))
+		outer_has_default = false;
+	if (inner_has_default && is_dummy_partition(inner_rel, inner_default))
+		inner_has_default = false;
+
+	/*
+	 * Merge partitions from both sides.  In each iteration we compare a pair
+	 * of ranges, one from each side, and decide whether the corresponding
+	 * partitions match or not.  If the two ranges overlap, move to the next
+	 * pair of ranges, otherwise move to the next range on the side with a
+	 * lower range.  outer_lb_pos/inner_lb_pos keep track of the positions of
+	 * lower bounds in the datums arrays in the outer/inner PartitionBoundInfos
+	 * respectively.
+	 */
+	outer_lb_pos = inner_lb_pos = 0;
+	outer_index = get_range_partition(outer_rel, outer_bi, &outer_lb_pos,
+									  &outer_lb, &outer_ub);
+	inner_index = get_range_partition(inner_rel, inner_bi, &inner_lb_pos,
+									  &inner_lb, &inner_ub);
+	while (outer_index >= 0 || inner_index >= 0)
+	{
+		bool		overlap;
+		int			ub_cmpval;
+		int			lb_cmpval;
+		PartitionRangeBound merged_lb = {-1, NULL, NULL, true};
+		PartitionRangeBound merged_ub = {-1, NULL, NULL, false};
+		int			merged_index = -1;
+
+		/*
+		 * We run this loop till both sides finish.  This allows us to avoid
+		 * duplicating code to handle the remaining ranges on the side which
+		 * finishes later.  For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra range higher than any other range on the unfinished side.
+		 * That way we advance the ranges on the unfinished side till all of
+		 * its ranges are exhausted.
+		 */
+		if (outer_index == -1)
+		{
+			overlap = false;
+			lb_cmpval = 1;
+			ub_cmpval = 1;
+		}
+		else if (inner_index == -1)
+		{
+			overlap = false;
+			lb_cmpval = -1;
+			ub_cmpval = -1;
+		}
+		else
+			overlap = compare_range_partitions(partnatts, partsupfuncs,
+											   partcollations,
+											   &outer_lb, &outer_ub,
+											   &inner_lb, &inner_ub,
+											   &lb_cmpval, &ub_cmpval);
+
+		if (overlap)
+		{
+			/* Two ranges overlap; form a join pair. */
+
+			PartitionRangeBound save_outer_ub;
+			PartitionRangeBound save_inner_ub;
+
+			/* Both partitions should not have been merged yet. */
+			Assert(outer_index >= 0);
+			Assert(outer_map.merged_indexes[outer_index] == -1 &&
+				   outer_map.merged[outer_index] == false);
+			Assert(inner_index >= 0);
+			Assert(inner_map.merged_indexes[inner_index] == -1 &&
+				   inner_map.merged[inner_index] == false);
+
+			/*
+			 * Get the index of the merged partition.  Both partitions aren't
+			 * merged yet, so the partitions should be merged successfully.
+			 */
+			merged_index = merge_matching_partitions(&outer_map, &inner_map,
+													 outer_index, inner_index,
+													 &next_index);
+			Assert(merged_index >= 0);
+
+			/* Get the range of the merged partition. */
+			get_merged_range_bounds(partnatts, partsupfuncs,
+									partcollations, jointype,
+									&outer_lb, &outer_ub,
+									&inner_lb, &inner_ub,
+									lb_cmpval, ub_cmpval,
+									&merged_lb, &merged_ub);
+
+			/* Save the upper bounds of both partitions for use below. */
+			save_outer_ub = outer_ub;
+			save_inner_ub = inner_ub;
+
+			/* Move to the next pair of ranges. */
+			outer_index = get_range_partition(outer_rel, outer_bi, &outer_lb_pos,
+											  &outer_lb, &outer_ub);
+			inner_index = get_range_partition(inner_rel, inner_bi, &inner_lb_pos,
+											  &inner_lb, &inner_ub);
+
+			/*
+			 * If the range of a partition on one side overlaps the range of
+			 * the next partition on the other side, that will cause the
+			 * partition on one side to match at least two partitions on the
+			 * other side, which is the case that we currently don't support
+			 * partitioned join for; give up.
+			 */
+			if (ub_cmpval > 0 && inner_index >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &save_outer_ub, &inner_lb) > 0)
+				goto cleanup;
+			if (ub_cmpval < 0 && outer_index >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &outer_lb, &save_inner_ub) < 0)
+				goto cleanup;
+
+			/*
+			 * A row from a non-overlapping portion (if any) of a partition
+			 * on one side might find its join partner in the default
+			 * partition (if any) on the other side, causing the same
+			 * situation as above; give up in that case.
+			 */
+			if ((outer_has_default && (lb_cmpval > 0 || ub_cmpval < 0)) ||
+				(inner_has_default && (lb_cmpval < 0 || ub_cmpval > 0)))
+				goto cleanup;
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* A non-overlapping outer range. */
+
+			/* The outer partition should not have been merged yet. */
+			Assert(outer_index >= 0);
+			Assert(outer_map.merged_indexes[outer_index] == -1 &&
+				   outer_map.merged[outer_index] == false);
+
+			/*
+			 * If the inner side has the default partition, or this is an outer
+			 * join, try to assign a merged partition to the outer partition
+			 * (see process_outer_partition()).  Otherwise, the outer partition
+			 * will not contribute to the result.
+			 */
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				merged_index = process_outer_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   outer_index,
+													   inner_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+				merged_lb = outer_lb;
+				merged_ub = outer_ub;
+			}
+
+			/* Move to the next range on the outer side. */
+			outer_index = get_range_partition(outer_rel, outer_bi, &outer_lb_pos,
+											  &outer_lb, &outer_ub);
+		}
+		else
+		{
+			/* A non-overlapping inner range. */
+			Assert(ub_cmpval > 0);
+
+			/* The inner partition should not have been merged yet. */
+			Assert(inner_index >= 0);
+			Assert(inner_map.merged_indexes[inner_index] == -1 &&
+				   inner_map.merged[inner_index] == false);
+
+			/*
+			 * If the outer side has the default partition, or this is a FULL
+			 * join, try to assign a merged partition to the inner partition
+			 * (see process_inner_partition()).  Otherwise, the inner partition
+			 * will not contribute to the result.
+			 */
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				merged_index = process_inner_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   inner_index,
+													   outer_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+				merged_lb = inner_lb;
+				merged_ub = inner_ub;
+			}
+
+			/* Move to the next range on the inner side. */
+			inner_index = get_range_partition(inner_rel, inner_bi, &inner_lb_pos,
+											  &inner_lb, &inner_ub);
+		}
+
+		/*
+		 * If we assigned a merged partition, add the range bounds and index of
+		 * the merged partition if appropriate.
+		 */
+		if (merged_index >= 0 && merged_index != default_index)
+			add_merged_range_bounds(partnatts, partsupfuncs, partcollations,
+									&merged_lb, &merged_ub, merged_index,
+									&merged_datums, &merged_kinds,
+									&merged_indexes);
+	}
+
+	/* Merge the default partitions if any. */
+	if (outer_has_default || inner_has_default)
+		merge_default_partitions(&outer_map, &inner_map,
+								 outer_has_default, inner_has_default,
+								 outer_default, inner_default,
+								 jointype, &next_index, &default_index);
+	else
+		Assert(default_index == -1);
+
+	/* If we have merged partitions, create the partition bounds. */
+	if (next_index > 0)
+	{
+		/*
+		 * Unlike the case of list partitioning, we wouldn't have re-merged
+		 * partitions, so did_remapping should be left alone.
+		 */
+		Assert(!outer_map.did_remapping);
+		Assert(!inner_map.did_remapping);
+
+		/* Use maps to match partitions from inputs. */
+		generate_matching_part_pairs(outer_rel, inner_rel,
+									 &outer_map, &inner_map,
+									 next_index,
+									 outer_parts, inner_parts);
+		Assert(*outer_parts != NIL);
+		Assert(*inner_parts != NIL);
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+
+		/* Make a PartitionBoundInfo struct to return. */
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_kinds,
+													  merged_indexes,
+													  -1,
+													  default_index);
+		Assert(merged_bounds);
+	}
+
+cleanup:
+	/* Free local memory before returning. */
+	list_free(merged_datums);
+	list_free(merged_kinds);
+	list_free(merged_indexes);
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+
+	return merged_bounds;
+}
+
+/*
+ * init_partition_map
+ *		Initialize a PartitionMap struct for given relation
+ */
+static void
+init_partition_map(RelOptInfo *rel, PartitionMap *map)
+{
+	int			nparts = rel->nparts;
+	int			i;
+
+	map->nparts = nparts;
+	map->merged_indexes = (int *) palloc(sizeof(int) * nparts);
+	map->merged = (bool *) palloc(sizeof(bool) * nparts);
+	map->did_remapping = false;
+	map->old_indexes = (int *) palloc(sizeof(int) * nparts);
+	for (i = 0; i < nparts; i++)
+	{
+		map->merged_indexes[i] = map->old_indexes[i] = -1;
+		map->merged[i] = false;
+	}
+}
+
+/*
+ * free_partition_map
+ */
+static void
+free_partition_map(PartitionMap *map)
+{
+	pfree(map->merged_indexes);
+	pfree(map->merged);
+	pfree(map->old_indexes);
+}
+
+/*
+ * is_dummy_partition --- has partition been proven empty?
+ */
+static bool
+is_dummy_partition(RelOptInfo *rel, int part_index)
+{
+	RelOptInfo *part_rel;
+
+	Assert(part_index >= 0);
+	part_rel = rel->part_rels[part_index];
+	if (part_rel == NULL || IS_DUMMY_REL(part_rel))
+		return true;
+	return false;
+}
+
+/*
+ * merge_matching_partitions
+ *		Try to merge given outer/inner partitions, and return the index of a
+ *		merged partition produced from them if successful, -1 otherwise
+ *
+ * If the merged partition is newly created, *next_index is incremented.
+ */
+static int
+merge_matching_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						  int outer_index, int inner_index, int *next_index)
+{
+	int 		outer_merged_index;
+	int 		inner_merged_index;
+	bool 		outer_merged;
+	bool 		inner_merged;
+
+	Assert(outer_index >= 0 && outer_index < outer_map->nparts);
+	outer_merged_index = outer_map->merged_indexes[outer_index];
+	outer_merged = outer_map->merged[outer_index];
+	Assert(inner_index >= 0 && inner_index < inner_map->nparts);
+	inner_merged_index = inner_map->merged_indexes[inner_index];
+	inner_merged = inner_map->merged[inner_index];
+
+	/*
+	 * Handle cases where we have already assigned a merged partition to each
+	 * of the given partitions.
+	 */
+	if (outer_merged_index >= 0 && inner_merged_index >= 0)
+	{
+		/*
+		 * If the mereged partitions are the same, no need to do anything;
+		 * return the index of the merged partitions.  Otherwise, if each of
+		 * the given partitions has been merged with a dummy partition on the
+		 * other side, re-map them to either of the two merged partitions.
+		 * Otherwise, they can't be merged, so return -1.
+		 */
+		if (outer_merged_index == inner_merged_index)
+		{
+			Assert(outer_merged);
+			Assert(inner_merged);
+			return outer_merged_index;
+		}
+		if (!outer_merged && !inner_merged)
+		{
+			/*
+			 * This can only happen for a list-partitioning case.  We re-map
+			 * them to the merged partition with the smaller of the two merged
+			 * indexes to preserve the property that the canonical order of
+			 * list partitions is determined by the indexes assigned to the
+			 * smallest list value of each partition.
+			 */
+			if (outer_merged_index < inner_merged_index)
+			{
+				outer_map->merged[outer_index] = true;
+				inner_map->merged_indexes[inner_index] = outer_merged_index;
+				inner_map->merged[inner_index] = true;
+				inner_map->did_remapping = true;
+				inner_map->old_indexes[inner_index] = inner_merged_index;
+				return outer_merged_index;
+			}
+			else
+			{
+				inner_map->merged[inner_index] = true;
+				outer_map->merged_indexes[outer_index] = inner_merged_index;
+				outer_map->merged[outer_index] = true;
+				outer_map->did_remapping = true;
+				outer_map->old_indexes[outer_index] = outer_merged_index;
+				return inner_merged_index;
+			}
+		}
+		return -1;
+	}
+
+	/* At least one of the given partitions should not have yet been merged. */
+	Assert(outer_merged_index == -1 || inner_merged_index == -1);
+
+	/*
+	 * If neither of them has been merged, merge them.  Otherwise, if one has
+	 * been merged with a dummy relation on the other side (and the other
+	 * hasn't yet been merged with anything), re-merge them.  Otherwise, they
+	 * can't be merged, so return -1.
+	 */
+	if (outer_merged_index == -1 && inner_merged_index == -1)
+	{
+		int 		merged_index = *next_index;
+
+		Assert(!outer_merged);
+		Assert(!inner_merged);
+		outer_map->merged_indexes[outer_index] = merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged_indexes[inner_index] = merged_index;
+		inner_map->merged[inner_index] = true;
+		*next_index = *next_index + 1;
+		return merged_index;
+	}
+	if (outer_merged_index >= 0 && !outer_map->merged[outer_index])
+	{
+		Assert(inner_merged_index == -1);
+		Assert(!inner_merged);
+		inner_map->merged_indexes[inner_index] = outer_merged_index;
+		inner_map->merged[inner_index] = true;
+		outer_map->merged[outer_index] = true;
+		return outer_merged_index;
+	}
+	if (inner_merged_index >= 0 && !inner_map->merged[inner_index])
+	{
+		Assert(outer_merged_index == -1);
+		Assert(!outer_merged);
+		outer_map->merged_indexes[outer_index] = inner_merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged[inner_index] = true;
+		return inner_merged_index;
+	}
+	return -1;
+}
+
+/*
+ * process_outer_partition
+ *		Try to assign given outer partition a merged partition, and return the
+ *		index of the merged partition if successful, -1 otherwise
+ *
+ * If the partition is newly created, *next_index is incremented.  Also, if it
+ * is the default partition of the join relation, *default_partition is set to
+ * the index if not already done.
+ */
+static int
+process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index)
+{
+	int 		merged_index = -1;
+
+	Assert(outer_index >= 0);
+
+	/*
+	 * If the inner side has the default partition, a row from the outer
+	 * partition might find its join partner in the default partition; try
+	 * merging the outer partition with the default partition.  Otherwise, this
+	 * should be an outer join, in which case the outer partition has to be
+	 * scanned all the way anyway; merge the outer partition with a dummy
+	 * partition on the other side.
+	 */
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0);
+
+		/*
+		 * If the outer side has the default partition as well, the default
+		 * partition on the inner side will have two matching partitions on the
+		 * other side: the outer partition and the default partition on the
+		 * outer side.  Partitionwise join doesn't handle this scenario yet.
+		 */
+		if (outer_has_default)
+			return -1;
+
+		merged_index = merge_matching_partitions(outer_map, inner_map,
+												 outer_index, inner_default,
+												 next_index);
+		if (merged_index == -1)
+			return -1;
+
+		/*
+		 * If this is a FULL join, the default partition on the inner side
+		 * has to be scanned all the way anyway, so the resulting partition
+		 * will contain all key values from the default partition, which any
+		 * other partition of the join relation will not contain.  Thus the
+		 * resutling partition will act as the default partition of the join
+		 * relation; record the index in *default_index if not already done.
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (*default_index == -1)
+				*default_index = merged_index;
+			else
+				Assert(*default_index == merged_index);
+		}
+	}
+	else
+	{
+		Assert(IS_OUTER_JOIN(jointype));
+		Assert(jointype != JOIN_RIGHT);
+
+		/* If we have already assigned a partition, no need to do anything. */
+		merged_index = outer_map->merged_indexes[outer_index];
+		if (merged_index == -1)
+			merged_index = merge_partition_with_dummy(outer_map, outer_index,
+													  next_index);
+	}
+	return merged_index;
+}
+
+/*
+ * process_inner_partition
+ *		Try to assign given inner partition a merged partition, and return the
+ *		index of the merged partition if successful, -1 otherwise
+ *
+ * If the partition is newly created, *next_index is incremented.  Also, if it
+ * is the default partition of the join relation, *default_partition is set to
+ * the index if not already done.
+ */
+static int
+process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index)
+{
+	int 		merged_index = -1;
+
+	Assert(inner_index >= 0);
+
+	/*
+	 * If the outer side has the default partition, a row from the inner
+	 * partition might find its join partner in the default partition; try
+	 * merging the inner partition with the default partition.  Otherwise, this
+	 * should be a FULL join, in which case the inner partition has to be
+	 * scanned all the way anyway; merge the inner partition with a dummy
+	 * partition on the other side.
+	 */
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0);
+
+		/*
+		 * If the inner side has the default partition as well, the default
+		 * partition on the outer side will have two matching partitions on the
+		 * other side: the inner partition and the default partition on the
+		 * inner side.  Partitionwise join doesn't handle this scenario yet.
+		 */
+		if (inner_has_default)
+			return -1;
+
+		merged_index = merge_matching_partitions(outer_map, inner_map,
+												 outer_default, inner_index,
+												 next_index);
+		if (merged_index == -1)
+			return -1;
+
+		/*
+		 * If this is an outer join, the default partition on the outer side
+		 * has to be scanned all the way anyway, so the resulting partition
+		 * will contain all key values from the default partition, which any
+		 * other partition of the join relation will not contain.  Thus the
+		 * resutling partition will act as the default partition of the join
+		 * relation; record the index in *default_index if not already done.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			if (*default_index == -1)
+				*default_index = merged_index;
+			else
+				Assert(*default_index == merged_index);
+		}
+	}
+	else
+	{
+		Assert(jointype == JOIN_FULL);
+
+		/* If we have already assigned a partition, no need to do anything. */
+		merged_index = inner_map->merged_indexes[inner_index];
+		if (merged_index == -1)
+			merged_index = merge_partition_with_dummy(inner_map, inner_index,
+													  next_index);
+	}
+	return merged_index;
+}
+
+/*
+ * merge_null_partitions
+ *		Merge the NULL partitions from a join's outer and inner sides.
+ *
+ * If the merged partition produced from them is the NULL partition of the join
+ * relation, *null_index is set to the index of the merged partition.
+ *
+ * Note: We assume here that the join clause for a partitioned join is strict
+ * because have_partkey_equi_join() requires that the corresponding operator
+ * be mergejoinable, and we currently assume that mergejoinable operators are
+ * strict (see MJEvalOuterValues()/MJEvalInnerValues()).
+ */
+static void
+merge_null_partitions(PartitionMap *outer_map,
+					  PartitionMap *inner_map,
+					  bool outer_has_null,
+					  bool inner_has_null,
+					  int outer_null,
+					  int inner_null,
+					  JoinType jointype,
+					  int *next_index,
+					  int *null_index)
+{
+	bool 		consider_outer_null = false;
+	bool 		consider_inner_null = false;
+
+	Assert(outer_has_null || inner_has_null);
+	Assert(*null_index == -1);
+
+	/*
+	 * Check whether the NULL partitions have already been merged and if so,
+	 * set the consider_outer_null/consider_inner_null flags.
+	 */
+	if (outer_has_null)
+	{
+		Assert(outer_null >= 0 && outer_null < outer_map->nparts);
+		if (outer_map->merged_indexes[outer_null] == -1)
+		 	consider_outer_null = true;
+	}
+	if (inner_has_null)
+	{
+		Assert(inner_null >= 0 && inner_null < inner_map->nparts);
+		if (inner_map->merged_indexes[inner_null] == -1)
+		 	consider_inner_null = true;
+	}
+
+	/* If both flags are set false, we don't need to do anything. */
+	if (!consider_outer_null && !consider_inner_null)
+		return;
+
+	if (consider_outer_null && !consider_inner_null)
+	{
+		Assert(outer_has_null);
+
+		/*
+		 * If this is an outer join, the NULL partition on the outer side has
+		 * to be scanned all the way anyway; merge the NULL partition with a
+		 * dummy partition on the other side.  In that case consider_outer_null
+		 * means that the NULL partition only contains NULL values as the key
+		 * values, so the merged partition will do so; treat it as the NULL
+		 * partition of the join relation.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			*null_index = merge_partition_with_dummy(outer_map, outer_null,
+													 next_index);
+		}
+	}
+	else if (!consider_outer_null && consider_inner_null)
+	{
+		Assert(inner_has_null);
+
+		/*
+		 * If this is a FULL join, the NULL partition on the inner side has
+		 * to be scanned all the way anyway; merge the NULL partition with a
+		 * dummy partition on the other side.  In that case consider_inner_null
+		 * means that the NULL partition only contains NULL values as the key
+		 * values, so the merged partition will do so; treat it as the NULL
+		 * partition of the join relation.
+		 */
+		if (jointype == JOIN_FULL)
+			*null_index = merge_partition_with_dummy(inner_map, inner_null,
+													 next_index);
+	}
+	else
+	{
+		Assert(consider_outer_null && consider_inner_null);
+		Assert(outer_has_null);
+		Assert(inner_has_null);
+
+		/*
+		 * If this is an outer join, the NULL partition on the outer side (and
+		 * that on the inner side if this is a FULL join) have to be scanned
+		 * all the way anyway, so merge them.  Note that each of the NULL
+		 * partitions isn't merged yet, so they should be merged successfully.
+		 * Like the above, each of the NULL partitions only contains NULL
+		 * values as the key values, so the merged partition will do so; treat
+		 * it as the NULL partition of the join relation.
+		 *
+		 * Note: if this an INNER/SEMI join, the join clause will never be
+		 * satisfied by two NULL values (see comments above), so both the NULL
+		 * partitions can be eliminated.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			*null_index = merge_matching_partitions(outer_map, inner_map,
+													outer_null, inner_null,
+													next_index);
+			Assert(*null_index >= 0);
+		}
+	}
+}
+
+/*
+ * merge_default_partitions
+ *		Merge the default partitions from a join's outer and inner sides.
+ *
+ * If the merged partition produced from them is the default partition of the
+ * join relation, *default_index is set to the index of the merged partition.
+ */
+static void
+merge_default_partitions(PartitionMap *outer_map,
+						 PartitionMap *inner_map,
+						 bool outer_has_default,
+						 bool inner_has_default,
+						 int outer_default,
+						 int inner_default,
+						 JoinType jointype,
+						 int *next_index,
+						 int *default_index)
+{
+	int 		outer_merged_index = -1;
+	int 		inner_merged_index = -1;
+
+	Assert(outer_has_default || inner_has_default);
+
+	/* Get the merged partition indexes for the default partitions. */
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0 && outer_default < outer_map->nparts);
+		outer_merged_index = outer_map->merged_indexes[outer_default];
+	}
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0 && inner_default < inner_map->nparts);
+		inner_merged_index = inner_map->merged_indexes[inner_default];
+	}
+
+	if (outer_has_default && !inner_has_default)
+	{
+		/*
+		 * If this is an outer join, the default partition on the outer side
+		 * has to be scanned all the way anyway; if we have not yet assigned a
+		 * partition, merge the default partition with a dummy partition on the
+		 * other side.  The merged partition will act as the default partition
+		 * of the join relation (see comments in process_inner_partition()).
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			if (outer_merged_index == -1)
+			{
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(outer_map,
+															outer_default,
+															next_index);
+			}
+			else
+				Assert(*default_index == outer_merged_index);
+		}
+		else
+			Assert(*default_index == -1);
+	}
+	else if (!outer_has_default && inner_has_default)
+	{
+		/*
+		 * If this is a FULL join, the default partition on the inner side
+		 * has to be scanned all the way anyway; if we have not yet assigned a
+		 * partition, merge the default partition with a dummy partition on the
+		 * other side.  The merged partition will act as the default partition
+		 * of the join relation (see comments in process_outer_partition()).
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (inner_merged_index == -1)
+			{
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(inner_map,
+															inner_default,
+															next_index);
+			}
+			else
+				Assert(*default_index == inner_merged_index);
+		}
+		else
+			Assert(*default_index == -1);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		/*
+		 * The default partitions have to be joined with each other, so merge
+		 * them.  Note that each of the default partitions isn't merged yet
+		 * (see, process_outer_partition()/process_innerer_partition()), so
+		 * they should be merged successfully.  The merged partition will act
+		 * as the default partition of the join relation.
+		 */
+		Assert(outer_merged_index == -1);
+		Assert(inner_merged_index == -1);
+		Assert(*default_index == -1);
+		*default_index = merge_matching_partitions(outer_map,
+												   inner_map,
+												   outer_default,
+												   inner_default,
+												   next_index);
+		Assert(*default_index >= 0);
+	}
+}
+
+/*
+ * merge_partition_with_dummy
+ *		Assign given partition a new partition of a join relation
+ *
+ * Note: The caller assumes that the given partition doesn't have a non-dummy
+ * matching partition on the other side, but if the given partition finds the
+ * matchig partition later, we will adjust the assignment.
+ */
+static int
+merge_partition_with_dummy(PartitionMap *map, int index, int *next_index)
+{
+	int 		merged_index = *next_index;
+
+	Assert(index >= 0 && index < map->nparts);
+	Assert(map->merged_indexes[index] == -1);
+	Assert(!map->merged[index]);
+	map->merged_indexes[index] = merged_index;
+	/* Leave the merged flag alone! */
+	*next_index = *next_index + 1;
+	return merged_index;
+}
+
+/*
+ * fix_merged_indexes
+ *		Adjust merged indexes of re-merged partitions
+ */
+static void
+fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes)
+{
+	int		   *new_indexes;
+	int			merged_index;
+	int			i;
+	ListCell   *lc;
+
+	Assert(nmerged > 0);
+
+	new_indexes = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		new_indexes[i] = -1;
+
+	/* Build the mapping of old merged indexes to new merged indexes. */
+	if (outer_map->did_remapping)
+	{
+		for (i = 0; i < outer_map->nparts; i++)
+		{
+			merged_index = outer_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = outer_map->merged_indexes[i];
+		}
+	}
+	if (inner_map->did_remapping)
+	{
+		for (i = 0; i < inner_map->nparts; i++)
+		{
+			merged_index = inner_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = inner_map->merged_indexes[i];
+		}
+	}
+
+	/* Fix the merged_indexes list using the mapping. */
+	foreach(lc, merged_indexes)
+	{
+		merged_index = lfirst_int(lc);
+		Assert(merged_index >= 0);
+		if (new_indexes[merged_index] >= 0)
+			lfirst_int(lc) = new_indexes[merged_index];
+	}
+
+	pfree(new_indexes);
+}
+
+/*
+ * generate_matching_part_pairs
+ *		Generate a pair of lists of partitions that produce merged partitions
+ *
+ * The lists of partitions are built in the order of merged partition indexes,
+ * and returned in *outer_parts and *inner_parts.
+ */
+static void
+generate_matching_part_pairs(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 PartitionMap *outer_map, PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_parts, List **inner_parts)
+{
+	int			outer_nparts = outer_map->nparts;
+	int			inner_nparts = inner_map->nparts;
+	int		   *outer_indexes;
+	int		   *inner_indexes;
+	int			max_nparts;
+	int 		i;
+
+	Assert(nmerged > 0);
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	outer_indexes = (int *) palloc(sizeof(int) * nmerged);
+	inner_indexes = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		outer_indexes[i] = inner_indexes[i] = -1;
+
+	/* Set pairs of matching partitions. */
+	Assert(outer_nparts == outer_rel->nparts);
+	Assert(inner_nparts == inner_rel->nparts);
+	max_nparts = Max(outer_nparts, inner_nparts);
+	for (i = 0; i < max_nparts; i++)
+	{
+		if (i < outer_nparts)
+		{
+			int 		merged_index = outer_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				outer_indexes[merged_index] = i;
+			}
+		}
+		if (i < inner_nparts)
+		{
+			int 		merged_index = inner_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				inner_indexes[merged_index] = i;
+			}
+		}
+	}
+
+	/* Build the list pairs. */
+	for (i = 0; i < nmerged; i++)
+	{
+		int			outer_index = outer_indexes[i];
+		int			inner_index = inner_indexes[i];
+
+		/*
+		 * If both partitions are dummy, it means the merged partition that had
+		 * been assigned to the outer/inner partition was removed when
+		 * re-merging the outer/inner partition in merge_matching_partitions();
+		 * ignore the merged partition.
+		 */
+		if (outer_index == -1 && inner_index == -1)
+			continue;
+
+		*outer_parts = lappend(*outer_parts, outer_index >= 0 ?
+							   outer_rel->part_rels[outer_index] : NULL);
+		*inner_parts = lappend(*inner_parts, inner_index >= 0 ?
+							   inner_rel->part_rels[inner_index] : NULL);
+	}
+
+	pfree(outer_indexes);
+	pfree(inner_indexes);
+}
+
+/*
+ * build_merged_partition_bounds
+ *		Create a PartitionBoundInfo struct from merged partition bounds
+ */
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_kinds, List *merged_indexes,
+							  int null_index, int default_index)
+{
+	PartitionBoundInfo merged_bounds;
+	int			ndatums = list_length(merged_datums);
+	int			pos;
+	ListCell   *lc;
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = ndatums;
+
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) * ndatums);
+	pos = 0;
+	foreach(lc, merged_datums)
+		merged_bounds->datums[pos++] = (Datum *) lfirst(lc);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_kinds) == ndatums);
+		merged_bounds->kind = (PartitionRangeDatumKind **)
+			palloc(sizeof(PartitionRangeDatumKind *) * ndatums);
+		pos = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[pos++] = (PartitionRangeDatumKind *) lfirst(lc);
+
+		/* There are ndatums+1 indexes in the case of range partitioning. */
+		merged_indexes = lappend_int(merged_indexes, -1);
+		ndatums++;
+	}
+	else
+	{
+		Assert(strategy == PARTITION_STRATEGY_LIST);
+		Assert(merged_kinds == NIL);
+		merged_bounds->kind = NULL;
+	}
+
+	Assert(list_length(merged_indexes) == ndatums);
+	merged_bounds->indexes = (int *) palloc(sizeof(int) * ndatums);
+	pos = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[pos++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * get_range_partition
+ *		Get the next non-dummy partition of a range-partitioned relation,
+ *		returning the index of that partition
+ *
+ * *lb and *ub are set to the lower and upper bounds of that partition
+ * respectively, and *lb_pos is advanced to the next lower bound, if any.
+ */
+static int
+get_range_partition(RelOptInfo *rel,
+					PartitionBoundInfo bi,
+					int *lb_pos,
+					PartitionRangeBound *lb,
+					PartitionRangeBound *ub)
+{
+	int			part_index;
+
+	Assert(bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	do {
+		part_index = get_range_partition_internal(bi, lb_pos, lb, ub);
+		if (part_index == -1)
+			return -1;
+	} while (is_dummy_partition(rel, part_index));
+
+	return part_index;
+}
+
+static int
+get_range_partition_internal(PartitionBoundInfo bi,
+							 int *lb_pos,
+							 PartitionRangeBound *lb,
+							 PartitionRangeBound *ub)
+{
+	/* Return the index as -1 if we've exhausted all lower bounds. */
+	if (*lb_pos >= bi->ndatums)
+		return -1;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(*lb_pos + 1 < bi->ndatums);
+
+	/* Set the lower bound. */
+	lb->index = bi->indexes[*lb_pos];
+	lb->datums = bi->datums[*lb_pos];
+	lb->kind = bi->kind[*lb_pos];
+	lb->lower = true;
+	/* Set the upper bound. */
+	ub->index = bi->indexes[*lb_pos + 1];
+	ub->datums = bi->datums[*lb_pos + 1];
+	ub->kind = bi->kind[*lb_pos + 1];
+	ub->lower = false;
+
+	/* The index assigned to an upper bound should be valid. */
+	Assert(ub->index >= 0);
+
+	/*
+	 * Advance the position to the next lower bound.  If there are no bounds
+	 * left beyond the upper bound, we have reached the last lower bound.
+	 */
+	if (*lb_pos + 2 >= bi->ndatums)
+		*lb_pos = bi->ndatums;
+	else
+	{
+		/*
+		 * If the index assigned to the bound next to the upper bound isn't
+		 * valid, that is the next lower bound; else, the upper bound is also
+		 * the lower bound of the next range partition.
+		 */
+		if (bi->indexes[*lb_pos + 2] < 0)
+			*lb_pos = *lb_pos + 2;
+		else
+			*lb_pos = *lb_pos + 1;
+	}
+
+	return ub->index;
+}
+
+/*
+ * compare_range_partitions
+ *		Compare the bounds of two range partitions, and return true if the
+ *		two partitions overlap, false otherwise
+ *
+ * *lb_cmpval is set to -1, 0, or 1 if the outer partition's lower bound is
+ * lower than, equal to, or higher than the inner partition's lower bound
+ * respectively.  Likewise, *ub_cmpval is set to -1, 0, or 1 if the outer
+ * partition's upper bound is lower than, equal to, or higher than the inner
+ * partition's upper bound respectively.
+ */
+static bool
+compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+						 Oid *partcollations,
+						 PartitionRangeBound *outer_lb,
+						 PartitionRangeBound *outer_ub,
+						 PartitionRangeBound *inner_lb,
+						 PartitionRangeBound *inner_ub,
+						 int *lb_cmpval, int *ub_cmpval)
+{
+	/*
+	 * Check if the outer partition's upper bound is lower than the inner
+	 * partition's lower bound; if so the partitions aren't overlapping.
+	 */
+	if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+							 outer_ub, inner_lb) < 0)
+	{
+		*lb_cmpval = -1;
+		*ub_cmpval = -1;
+		return false;
+	}
+
+	/*
+	 * Check if the outer partition's lower bound is higher than the inner
+	 * partition's upper bound; if so the partitions aren't overlapping.
+	 */
+	if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+							 outer_lb, inner_ub) > 0)
+	{
+		*lb_cmpval = 1;
+		*ub_cmpval = 1;
+		return false;
+	}
+
+	/* All other cases indicate overlapping partitions. */
+	*lb_cmpval = compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									  outer_lb, inner_lb);
+	*ub_cmpval = compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									  outer_ub, inner_ub);
+	return true;
+}
+
+/*
+ * get_merged_range_bounds
+ *		Given the bounds of range partitions to be joined, determine the bounds
+ *		of a merged partition produced from the range partitions
+ *
+ * *merged_lb and *merged_ub are set to the lower and upper bounds of the
+ * merged partition.
+ */
+static void
+get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations, JoinType jointype,
+						PartitionRangeBound *outer_lb,
+						PartitionRangeBound *outer_ub,
+						PartitionRangeBound *inner_lb,
+						PartitionRangeBound *inner_ub,
+						int	lb_cmpval, int ub_cmpval,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub)
+{
+	Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								outer_lb, inner_lb) == lb_cmpval);
+	Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								outer_ub, inner_ub) == ub_cmpval);
+
+	switch (jointype)
+	{
+		case JOIN_INNER:
+		case JOIN_SEMI:
+
+			/*
+			 * An INNER/SEMI join will have the rows that fit both sides, so
+			 * the lower bound of the merged partition will be the higher of
+			 * the two lower bounds, and the upper bound of the merged
+			 * partition will be the lower of the two upper bounds.
+			 */
+			*merged_lb = (lb_cmpval > 0) ? *outer_lb : *inner_lb;
+			*merged_ub = (ub_cmpval < 0) ? *outer_ub : *inner_ub;
+			break;
+
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+
+			/*
+			 * A LEFT/ANTI join will have all the rows from the outer side, so
+			 * the bounds of the merged partition will be the same as the outer
+			 * bounds.
+			 */
+			*merged_lb = *outer_lb;
+			*merged_ub = *outer_ub;
+			break;
+
+		case JOIN_FULL:
+
+			/*
+			 * A FULL join will have all the rows from both sides, so the lower
+			 * bound of the merged partition will be the lower of the two lower
+			 * bounds, and the upper bound of the merged partition will be the
+			 * higher of the two upper bounds.
+			 */
+			*merged_lb = (lb_cmpval < 0) ? *outer_lb : *inner_lb;
+			*merged_ub = (ub_cmpval > 0) ? *outer_ub : *inner_ub;
+			break;
+
+		default:
+			elog(ERROR, "unrecognized join type: %d", (int) jointype);
+	}
+}
+
+/*
+ * add_merged_range_bounds
+ *		Add the bounds of a merged partition to the lists of range bounds
+ */
+static void
+add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub,
+						int merged_index,
+						List **merged_datums,
+						List **merged_kinds,
+						List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		/* First merged partition */
+		Assert(!*merged_kinds);
+		Assert(!*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		Assert(*merged_datums);
+		Assert(*merged_kinds);
+		Assert(*merged_indexes);
+
+		/* Get the last upper bound. */
+		prev_ub.index = llast_int(*merged_indexes);
+		prev_ub.datums = (Datum *) llast(*merged_datums);
+		prev_ub.kind = (PartitionRangeDatumKind *) llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		/*
+		 * We pass to partition_rbound_cmp() lower1 as false to prevent it
+		 * from considering the last upper bound to be smaller than the lower
+		 * bound of the merged partition when the values of the two range
+		 * bounds compare equal.
+		 */
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  merged_lb->datums, merged_lb->kind,
+									  false, &prev_ub);
+		Assert(cmpval >= 0);
+	}
+
+	/*
+	 * If the lower bound is higher than the last upper bound, add the lower
+	 * bound with the index as -1 indicating that that is a lower bound; else,
+	 * the last upper bound will be reused as the lower bound of the merged
+	 * partition, so skip this.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, merged_lb->datums);
+		*merged_kinds = lappend(*merged_kinds, merged_lb->kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	/* Add the upper bound and index of the merged partition. */
+	*merged_datums = lappend(*merged_datums, merged_ub->datums);
+	*merged_kinds = lappend(*merged_kinds, merged_ub->kind);
+	*merged_indexes = lappend_int(*merged_indexes, merged_index);
+}
+
 /*
  * partitions_are_ordered
  *		Determine whether the partitions described by 'boundinfo' are ordered,
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index 0ceb809644..622ea2bf63 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -579,8 +579,10 @@ typedef struct PartitionSchemeData *PartitionScheme;
  *		part_scheme - Partitioning scheme of the relation
  *		nparts - Number of partitions
  *		boundinfo - Partition bounds
+ *		merged - true if partition bounds are merged ones
  *		partition_qual - Partition constraint if not the root
  *		part_rels - RelOptInfos for each partition
+ *		all_partrels - Relids set of all partition relids
  *		partexprs, nullable_partexprs - Partition key expressions
  *		partitioned_child_rels - RT indexes of unpruned partitions of
  *								 this relation that are partitioned tables
@@ -718,11 +720,15 @@ typedef struct RelOptInfo
 
 	/* used for partitioned relations */
 	PartitionScheme part_scheme;	/* Partitioning scheme. */
-	int			nparts;			/* number of partitions */
+	int			nparts;			/* number of partitions; 0 = not partitioned;
+								 * -1 = not yet set */
 	struct PartitionBoundInfoData *boundinfo;	/* Partition bounds */
+	bool		merged;			/* true if partition bounds were created by
+								 * partition_bounds_merge() */
 	List	   *partition_qual; /* partition constraint */
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of partitions,
 									 * stored in the same order of bounds */
+	Relids		all_partrels;	/* Relids set of all partition relids */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 	List	   *partitioned_child_rels; /* List of RT indexes. */
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index be940b5192..dfc720720b 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -16,6 +16,7 @@
 #include "nodes/pg_list.h"
 #include "partitioning/partdefs.h"
 #include "utils/relcache.h"
+struct RelOptInfo;				/* avoid including pathnodes.h here */
 
 
 /*
@@ -87,6 +88,14 @@ extern bool partition_bounds_equal(int partnatts, int16 *parttyplen,
 								   PartitionBoundInfo b2);
 extern PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src,
 												PartitionKey key);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+												 FmgrInfo *partsupfunc,
+												 Oid *partcollation,
+												 struct RelOptInfo *outer_rel,
+												 struct RelOptInfo *inner_rel,
+												 JoinType jointype,
+												 List **outer_parts,
+												 List **inner_parts);
 extern bool partitions_are_ordered(PartitionBoundInfo boundinfo, int nparts);
 extern void check_new_partition_bound(char *relname, Relation parent,
 									  PartitionBoundSpec *spec);
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index b3fbe47bde..27588c883e 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -2055,30 +2055,2606 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
                ->  Seq Scan on prt1_n_p2 t1_2
 (10 rows)
 
--- partitionwise join can not be applied if only one of joining tables has
--- default partition
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
-ANALYZE prt2;
+--
+-- Test advanced partition-matching algorithm for partitioned join
+--
+-- Tests for range-partitioned tables
+CREATE TABLE prt1_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_adv_p1 PARTITION OF prt1_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_adv_p2 PARTITION OF prt1_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_adv_p3 PARTITION OF prt1_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_adv_a_idx ON prt1_adv (a);
+INSERT INTO prt1_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_adv;
+CREATE TABLE prt2_adv (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_adv_p1 PARTITION OF prt2_adv FOR VALUES FROM (100) TO (150);
+CREATE TABLE prt2_adv_p2 PARTITION OF prt2_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt2_adv_p3 PARTITION OF prt2_adv FOR VALUES FROM (350) TO (500);
+CREATE INDEX prt2_adv_b_idx ON prt2_adv (b);
+INSERT INTO prt2_adv_p1 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 149) i;
+INSERT INTO prt2_adv_p2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(200, 299) i;
+INSERT INTO prt2_adv_p3 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_adv;
+-- inner join
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
  Sort
    Sort Key: t1.a
-   ->  Hash Join
-         Hash Cond: (t2.b = t1.a)
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 100 | 0 | 0100
+ 125 | 0 | 0125
+ 200 | 0 | 0200
+ 225 | 0 | 0225
+ 250 | 0 | 0250
+ 275 | 0 | 0275
+ 350 | 0 | 0350
+ 375 | 0 | 0375
+(8 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 |     | 
+ 175 | 0175 |     | 
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 300 | 0300 |     | 
+ 325 | 0325 |     | 
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(12 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 150 | 0 | 0150
+ 175 | 0 | 0175
+ 300 | 0 | 0300
+ 325 | 0 | 0325
+(4 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                                QUERY PLAN                                
+--------------------------------------------------------------------------
+ Sort
+   Sort Key: prt1_adv.a, prt2_adv.b
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_adv_1.a = prt2_adv_1.b)
+               Filter: (((175) = prt1_adv_1.a) OR ((425) = prt2_adv_1.b))
+               ->  Seq Scan on prt1_adv_p1 prt1_adv_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 prt2_adv_1
+                           Filter: (a = 0)
+         ->  Hash Full Join
+               Hash Cond: (prt1_adv_2.a = prt2_adv_2.b)
+               Filter: (((175) = prt1_adv_2.a) OR ((425) = prt2_adv_2.b))
+               ->  Seq Scan on prt1_adv_p2 prt1_adv_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 prt2_adv_2
+                           Filter: (a = 0)
+         ->  Hash Full Join
+               Hash Cond: (prt2_adv_3.b = prt1_adv_3.a)
+               Filter: (((175) = prt1_adv_3.a) OR ((425) = prt2_adv_3.b))
+               ->  Seq Scan on prt2_adv_p3 prt2_adv_3
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 prt1_adv_3
+                           Filter: (b = 0)
+(27 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 175 | 0175 |     | 
+     |      | 425 | 0425
+(2 rows)
+
+-- Test cases where one side has an extra partition
+CREATE TABLE prt2_adv_extra PARTITION OF prt2_adv FOR VALUES FROM (500) TO (MAXVALUE);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(500, 599) i;
+ANALYZE prt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 100 | 0 | 0100
+ 125 | 0 | 0125
+ 200 | 0 | 0200
+ 225 | 0 | 0225
+ 250 | 0 | 0250
+ 275 | 0 | 0275
+ 350 | 0 | 0350
+ 375 | 0 | 0375
+(8 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 |     | 
+ 175 | 0175 |     | 
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 300 | 0300 |     | 
+ 325 | 0325 |     | 
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(12 rows)
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t2.a
+   ->  Hash Right Join
+         Hash Cond: (t2.a = t1.b)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2_1
-               ->  Seq Scan on prt2_p2 t2_2
-               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt1_adv_p1 t2_1
+               ->  Seq Scan on prt1_adv_p2 t2_2
+               ->  Seq Scan on prt1_adv_p3 t2_3
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt2_adv_p1 t1_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_adv_p2 t1_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_adv_p3 t1_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_adv_extra t1_4
+                           Filter: (a = 0)
+(18 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 150 | 0 | 0150
+ 175 | 0 | 0175
+ 300 | 0 | 0300
+ 325 | 0 | 0325
+(4 rows)
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt1_adv t2 WHERE t1.b = t2.a) AND t1.a = 0 ORDER BY t1.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_extra t1_4
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 t2_1
+                     ->  Seq Scan on prt1_adv_p2 t2_2
+                     ->  Seq Scan on prt1_adv_p3 t2_3
+(18 rows)
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                           QUERY PLAN                           
+----------------------------------------------------------------
+ Sort
+   Sort Key: prt1_adv.a, prt2_adv.b
+   ->  Hash Full Join
+         Hash Cond: (prt2_adv.b = prt1_adv.a)
+         Filter: (((175) = prt1_adv.a) OR ((425) = prt2_adv.b))
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 prt2_adv_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p2 prt2_adv_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p3 prt2_adv_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_extra prt2_adv_4
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 prt1_adv_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_adv_p2 prt1_adv_2
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_adv_p3 prt1_adv_3
                            Filter: (b = 0)
+(22 rows)
+
+-- 3-way join where not every pair of relations can do partitioned join
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c, t3.a, t3.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) INNER JOIN prt1_adv t3 ON (t1.b = t3.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a, t3.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t2.a
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_adv_p1 t1_1
+                           Filter: (a = 0)
+                     ->  Index Scan using prt1_adv_p1_a_idx on prt1_adv_p1 t3_1
+                           Index Cond: (a = t1_1.b)
+               ->  Index Scan using prt1_adv_p1_a_idx on prt1_adv_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.a = t1_2.b)
+               ->  Seq Scan on prt1_adv_p2 t2_2
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_2.a = t1_2.b)
+                           ->  Seq Scan on prt1_adv_p2 t3_2
+                           ->  Hash
+                                 ->  Seq Scan on prt2_adv_p2 t1_2
+                                       Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_3.a = t1_3.b)
+               ->  Seq Scan on prt1_adv_p3 t2_3
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_3.a = t1_3.b)
+                           ->  Seq Scan on prt1_adv_p3 t3_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_adv_p3 t1_3
+                                       Filter: (a = 0)
+(31 rows)
+
+SELECT t1.b, t1.c, t2.a, t2.c, t3.a, t3.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) INNER JOIN prt1_adv t3 ON (t1.b = t3.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a, t3.a;
+  b  |  c   |  a  |  c   |  a  |  c   
+-----+------+-----+------+-----+------
+ 100 | 0100 | 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275 | 275 | 0275
+ 350 | 0350 | 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375 | 375 | 0375
+(8 rows)
+
+DROP TABLE prt2_adv_extra;
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in such cases
+ALTER TABLE prt2_adv DETACH PARTITION prt2_adv_p3;
+-- Split prt2_adv_p3 into two partitions so that prt1_adv_p3 matches both
+CREATE TABLE prt2_adv_p3_1 PARTITION OF prt2_adv FOR VALUES FROM (350) TO (375);
+CREATE TABLE prt2_adv_p3_2 PARTITION OF prt2_adv FOR VALUES FROM (375) TO (500);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Seq Scan on prt2_adv_p3_1 t2_3
+               ->  Seq Scan on prt2_adv_p3_2 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+                     ->  Seq Scan on prt2_adv_p3_1 t2_3
+                     ->  Seq Scan on prt2_adv_p3_2 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Seq Scan on prt2_adv_p3_1 t2_3
+               ->  Seq Scan on prt2_adv_p3_2 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(17 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+                     ->  Seq Scan on prt2_adv_p3_1 t2_3
+                     ->  Seq Scan on prt2_adv_p3_2 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                           QUERY PLAN                           
+----------------------------------------------------------------
+ Sort
+   Sort Key: prt1_adv.a, prt2_adv.b
+   ->  Hash Full Join
+         Hash Cond: (prt2_adv.b = prt1_adv.a)
+         Filter: (((175) = prt1_adv.a) OR ((425) = prt2_adv.b))
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 prt2_adv_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p2 prt2_adv_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p3_1 prt2_adv_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p3_2 prt2_adv_4
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 prt1_adv_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p2 prt1_adv_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 prt1_adv_3
+                           Filter: (b = 0)
+(22 rows)
+
+DROP TABLE prt2_adv_p3_1;
+DROP TABLE prt2_adv_p3_2;
+ANALYZE prt2_adv;
+-- Test default partitions
+ALTER TABLE prt1_adv DETACH PARTITION prt1_adv_p1;
+-- Change prt1_adv_p1 to the default partition
+ALTER TABLE prt1_adv ATTACH PARTITION prt1_adv_p1 DEFAULT;
+ALTER TABLE prt1_adv DETACH PARTITION prt1_adv_p3;
+ANALYZE prt1_adv;
+-- We can do partitioned join even if only one of relations has the default
+-- partition
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_1
+                           Filter: (b = 0)
+(15 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+(6 rows)
+
+-- Restore prt1_adv_p3
+ALTER TABLE prt1_adv ATTACH PARTITION prt1_adv_p3 FOR VALUES FROM (300) TO (400);
+ANALYZE prt1_adv;
+-- Restore prt2_adv_p3
+ALTER TABLE prt2_adv ATTACH PARTITION prt2_adv_p3 FOR VALUES FROM (350) TO (500);
+ANALYZE prt2_adv;
+-- Partitioned join can't be applied because the default partition of prt1_adv
+-- matches prt2_adv_p1 and prt2_adv_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Seq Scan on prt2_adv_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p2 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p1 t1_3
+                           Filter: (b = 0)
+(16 rows)
+
+ALTER TABLE prt2_adv DETACH PARTITION prt2_adv_p3;
+-- Change prt2_adv_p3 to the default partition
+ALTER TABLE prt2_adv ATTACH PARTITION prt2_adv_p3 DEFAULT;
+ANALYZE prt2_adv;
+-- Partitioned join can't be applied because the default partition of prt1_adv
+-- matches prt2_adv_p1 and prt2_adv_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Seq Scan on prt2_adv_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p2 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p1 t1_3
+                           Filter: (b = 0)
+(16 rows)
+
+DROP TABLE prt1_adv_p3;
+ANALYZE prt1_adv;
+DROP TABLE prt2_adv_p3;
+ANALYZE prt2_adv;
+CREATE TABLE prt3_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt3_adv_p1 PARTITION OF prt3_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt3_adv_p2 PARTITION OF prt3_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt3_adv_a_idx ON prt3_adv (a);
+INSERT INTO prt3_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(200, 399) i;
+ANALYZE prt3_adv;
+-- 3-way join to test the default partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) LEFT JOIN prt3_adv t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b, t3.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t3_1.a = t1_1.a)
+               ->  Seq Scan on prt3_adv_p1 t3_1
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_2.b = t1_1.a)
+                           ->  Seq Scan on prt2_adv_p2 t2_2
+                           ->  Hash
+                                 ->  Seq Scan on prt1_adv_p2 t1_1
+                                       Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t3_2.a = t1_2.a)
+               ->  Seq Scan on prt3_adv_p2 t3_2
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_1.b = t1_2.a)
+                           ->  Seq Scan on prt2_adv_p1 t2_1
+                           ->  Hash
+                                 ->  Seq Scan on prt1_adv_p1 t1_2
+                                       Filter: (b = 0)
+(23 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) LEFT JOIN prt3_adv t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+  a  |  c   |  b  |  c   |  a  |  c   
+-----+------+-----+------+-----+------
+ 100 | 0100 | 100 | 0100 |     | 
+ 125 | 0125 | 125 | 0125 |     | 
+ 150 | 0150 |     |      |     | 
+ 175 | 0175 |     |      |     | 
+ 200 | 0200 | 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275 | 275 | 0275
+(8 rows)
+
+DROP TABLE prt1_adv;
+DROP TABLE prt2_adv;
+DROP TABLE prt3_adv;
+-- Test interaction of partitioned join with partition pruning
+CREATE TABLE prt1_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_adv_p1 PARTITION OF prt1_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_adv_p2 PARTITION OF prt1_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_adv_p3 PARTITION OF prt1_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_adv_a_idx ON prt1_adv (a);
+INSERT INTO prt1_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_adv;
+CREATE TABLE prt2_adv (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_adv_p1 PARTITION OF prt2_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt2_adv_p2 PARTITION OF prt2_adv FOR VALUES FROM (200) TO (400);
+CREATE INDEX prt2_adv_b_idx ON prt2_adv (b);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+                        QUERY PLAN                         
+-----------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: ((a < 300) AND (b = 0))
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: ((a < 300) AND (b = 0))
+(15 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 | 150 | 0150
+ 175 | 0175 | 175 | 0175
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+(8 rows)
+
+DROP TABLE prt1_adv_p3;
+CREATE TABLE prt1_adv_default PARTITION OF prt1_adv DEFAULT;
+ANALYZE prt1_adv;
+CREATE TABLE prt2_adv_default PARTITION OF prt2_adv DEFAULT;
+ANALYZE prt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+                                QUERY PLAN                                
+--------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: ((a >= 100) AND (a < 300) AND (b = 0))
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: ((a >= 100) AND (a < 300) AND (b = 0))
+(15 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 | 150 | 0150
+ 175 | 0175 | 175 | 0175
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+(8 rows)
+
+DROP TABLE prt1_adv;
+DROP TABLE prt2_adv;
+-- Tests for list-partitioned tables
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0001', '0003');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt1_adv_p3 PARTITION OF plt1_adv FOR VALUES IN ('0008', '0009');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+ANALYZE plt1_adv;
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt2_adv_p3 PARTITION OF plt2_adv FOR VALUES IN ('0007', '0009');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+(6 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 1 | 1 | 0001
+ 8 | 8 | 0008
+(2 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+   |      | 2 | 0002
+   |      | 7 | 0007
+(8 rows)
+
+-- Test cases where one side has an extra partition
+CREATE TABLE plt2_adv_extra PARTITION OF plt2_adv FOR VALUES IN ('0000');
+INSERT INTO plt2_adv_extra VALUES (0, 0, '0000');
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+(6 rows)
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2_adv t1 LEFT JOIN plt1_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt1_adv_p1 t2_1
+               ->  Seq Scan on plt1_adv_p2 t2_2
+               ->  Seq Scan on plt1_adv_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_adv_extra t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_adv_p1 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_adv_p2 t1_3
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_adv_p3 t1_4
+                           Filter: (b < 10)
+(18 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 1 | 1 | 0001
+ 8 | 8 | 0008
+(2 rows)
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt2_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt1_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Anti Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_extra t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t1_4
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t2_1
+                     ->  Seq Scan on plt1_adv_p2 t2_2
+                     ->  Seq Scan on plt1_adv_p3 t2_3
+(18 rows)
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt2_adv_extra t2_1
+               ->  Seq Scan on plt2_adv_p1 t2_2
+               ->  Seq Scan on plt2_adv_p2 t2_3
+               ->  Seq Scan on plt2_adv_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+(15 rows)
+
+DROP TABLE plt2_adv_extra;
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in such cases
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2;
+-- Split plt2_adv_p2 into two partitions so that plt1_adv_p2 matches both
+CREATE TABLE plt2_adv_p2_1 PARTITION OF plt2_adv FOR VALUES IN ('0004');
+CREATE TABLE plt2_adv_p2_2 PARTITION OF plt2_adv FOR VALUES IN ('0006');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6);
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_1 t2_2
+               ->  Seq Scan on plt2_adv_p2_2 t2_3
+               ->  Seq Scan on plt2_adv_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Semi Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+                     ->  Seq Scan on plt2_adv_p2_1 t2_2
+                     ->  Seq Scan on plt2_adv_p2_2 t2_3
+                     ->  Seq Scan on plt2_adv_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_1 t2_2
+               ->  Seq Scan on plt2_adv_p2_2 t2_3
+               ->  Seq Scan on plt2_adv_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(17 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Anti Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+                     ->  Seq Scan on plt2_adv_p2_1 t2_2
+                     ->  Seq Scan on plt2_adv_p2_2 t2_3
+                     ->  Seq Scan on plt2_adv_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_1 t2_2
+               ->  Seq Scan on plt2_adv_p2_2 t2_3
+               ->  Seq Scan on plt2_adv_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+(15 rows)
+
+DROP TABLE plt2_adv_p2_1;
+DROP TABLE plt2_adv_p2_2;
+-- Restore plt2_adv_p2
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2 FOR VALUES IN ('0004', '0006');
+-- Test NULL partitions
+ALTER TABLE plt1_adv DETACH PARTITION plt1_adv_p1;
+-- Change plt1_adv_p1 to the NULL partition
+CREATE TABLE plt1_adv_p1_null PARTITION OF plt1_adv FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p3;
+-- Change plt2_adv_p3 to the NULL partition
+CREATE TABLE plt2_adv_p3_null PARTITION OF plt2_adv FOR VALUES IN (NULL, '0007', '0009');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (7, 9);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1_null t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1_null t1_1
+                     Filter: (b < 10)
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+(19 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1_null t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a  |  c   | a |  c   
+----+------+---+------
+ -1 |      |   | 
+  1 | 0001 |   | 
+  3 | 0003 | 3 | 0003
+  4 | 0004 | 4 | 0004
+  6 | 0006 | 6 | 0006
+  8 | 0008 |   | 
+  9 | 0009 | 9 | 0009
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1_null t1_1
+                     Filter: (b < 10)
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+(19 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a  | b  |  c   
+----+----+------
+ -1 | -1 | 
+  1 |  1 | 0001
+  8 |  8 | 0008
+(3 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p1_null t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+ -1 |      |    | 
+  1 | 0001 |    | 
+  3 | 0003 |  3 | 0003
+  4 | 0004 |  4 | 0004
+  6 | 0006 |  6 | 0006
+  8 | 0008 |    | 
+  9 | 0009 |  9 | 0009
+    |      | -1 | 
+    |      |  2 | 0002
+    |      |  7 | 0007
+(10 rows)
+
+DROP TABLE plt1_adv_p1_null;
+-- Restore plt1_adv_p1
+ALTER TABLE plt1_adv ATTACH PARTITION plt1_adv_p1 FOR VALUES IN ('0001', '0003');
+-- Add to plt1_adv the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt1_adv_extra PARTITION OF plt1_adv FOR VALUES IN (NULL);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+DROP TABLE plt2_adv_p3_null;
+-- Restore plt2_adv_p3
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p3 FOR VALUES IN ('0007', '0009');
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Seq Scan on plt2_adv_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_extra t1_4
+                           Filter: (b < 10)
+(18 rows)
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt1_adv_p1 t1_1
+               ->  Seq Scan on plt1_adv_p2 t1_2
+               ->  Seq Scan on plt1_adv_p3 t1_3
+               ->  Seq Scan on plt1_adv_extra t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+                     ->  Seq Scan on plt2_adv_p2 t2_2
+                     ->  Seq Scan on plt2_adv_p3 t2_3
+(15 rows)
+
+-- Add to plt2_adv the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt2_adv_extra PARTITION OF plt2_adv FOR VALUES IN (NULL);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+         ->  Nested Loop Left Join
+               Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+               ->  Seq Scan on plt1_adv_extra t1_4
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_extra t2_4
+(26 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a  |  c   | a |  c   
+----+------+---+------
+ -1 |      |   | 
+  1 | 0001 |   | 
+  3 | 0003 | 3 | 0003
+  4 | 0004 | 4 | 0004
+  6 | 0006 | 6 | 0006
+  8 | 0008 |   | 
+  9 | 0009 | 9 | 0009
+(7 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p3 t2_3
+         ->  Hash Full Join
+               Hash Cond: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+               Filter: ((COALESCE(t1_4.b, 0) < 10) AND (COALESCE(t2_4.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_extra t1_4
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_extra t2_4
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+ -1 |      |    | 
+  1 | 0001 |    | 
+  3 | 0003 |  3 | 0003
+  4 | 0004 |  4 | 0004
+  6 | 0006 |  6 | 0006
+  8 | 0008 |    | 
+  9 | 0009 |  9 | 0009
+    |      | -1 | 
+    |      |  2 | 0002
+    |      |  7 | 0007
+(10 rows)
+
+-- 3-way join to test the NULL partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt1_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t3_1.a = t1_1.a) AND (t3_1.c = t1_1.c))
+               ->  Seq Scan on plt1_adv_p1 t3_1
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+                           ->  Seq Scan on plt2_adv_p1 t2_1
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p1 t1_1
+                                       Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t3_2.a = t1_2.a) AND (t3_2.c = t1_2.c))
+               ->  Seq Scan on plt1_adv_p2 t3_2
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+                           ->  Seq Scan on plt2_adv_p2 t2_2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p2 t1_2
+                                       Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t3_3.a = t1_3.a) AND (t3_3.c = t1_3.c))
+               ->  Seq Scan on plt1_adv_p3 t3_3
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+                           ->  Seq Scan on plt2_adv_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p3 t1_3
+                                       Filter: (b < 10)
+         ->  Nested Loop Left Join
+               Join Filter: ((t1_4.a = t3_4.a) AND (t1_4.c = t3_4.c))
+               ->  Nested Loop Left Join
+                     Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+                     ->  Seq Scan on plt1_adv_extra t1_4
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_adv_extra t2_4
+               ->  Seq Scan on plt1_adv_extra t3_4
+(41 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt1_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a  |  c   | a |  c   | a |  c   
+----+------+---+------+---+------
+ -1 |      |   |      |   | 
+  1 | 0001 |   |      | 1 | 0001
+  3 | 0003 | 3 | 0003 | 3 | 0003
+  4 | 0004 | 4 | 0004 | 4 | 0004
+  6 | 0006 | 6 | 0006 | 6 | 0006
+  8 | 0008 |   |      | 8 | 0008
+  9 | 0009 | 9 | 0009 | 9 | 0009
+(7 rows)
+
+DROP TABLE plt1_adv_extra;
+DROP TABLE plt2_adv_extra;
+-- Test default partitions
+ALTER TABLE plt1_adv DETACH PARTITION plt1_adv_p1;
+-- Change plt1_adv_p1 to the default partition
+ALTER TABLE plt1_adv ATTACH PARTITION plt1_adv_p1 DEFAULT;
+DROP TABLE plt1_adv_p3;
+ANALYZE plt1_adv;
+DROP TABLE plt2_adv_p3;
+ANALYZE plt2_adv;
+-- We can do partitioned join even if only one of relations has the default
+-- partition
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_1
+                           Filter: (b < 10)
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(3 rows)
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2;
+-- Change plt2_adv_p2 to contain '0005' in addition to '0004' and '0006' as
+-- the key values
+CREATE TABLE plt2_adv_p2_ext PARTITION OF plt2_adv FOR VALUES IN ('0004', '0005', '0006');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 5, 6);
+ANALYZE plt2_adv;
+-- Partitioned join can't be applied because the default partition of plt1_adv
+-- matches plt2_adv_p1 and plt2_adv_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_ext t2_2
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p2 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p1 t1_2
+                           Filter: (b < 10)
+(13 rows)
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2_ext;
+-- Change plt2_adv_p2_ext to the default partition
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2_ext DEFAULT;
+ANALYZE plt2_adv;
+-- Partitioned join can't be applied because the default partition of plt1_adv
+-- matches plt2_adv_p1 and plt2_adv_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_ext t2_2
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p2 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p1 t1_2
+                           Filter: (b < 10)
+(13 rows)
+
+DROP TABLE plt2_adv_p2_ext;
+-- Restore plt2_adv_p2
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_adv;
+CREATE TABLE plt3_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_adv_p1 PARTITION OF plt3_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt3_adv_p2 PARTITION OF plt3_adv FOR VALUES IN ('0007', '0009');
+INSERT INTO plt3_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6, 7, 9);
+ANALYZE plt3_adv;
+-- 3-way join to test the default partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t3_1.a = t1_1.a) AND (t3_1.c = t1_1.c))
+               ->  Seq Scan on plt3_adv_p1 t3_1
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+                           ->  Seq Scan on plt2_adv_p2 t2_2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p2 t1_1
+                                       Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t3_2.a = t1_2.a) AND (t3_2.c = t1_2.c))
+               ->  Seq Scan on plt3_adv_p2 t3_2
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+                           ->  Seq Scan on plt2_adv_p1 t2_1
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p1 t1_2
+                                       Filter: (b < 10)
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   | a |  c   
+---+------+---+------+---+------
+ 1 | 0001 |   |      |   | 
+ 3 | 0003 | 3 | 0003 |   | 
+ 4 | 0004 | 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006 | 6 | 0006
+(4 rows)
+
+-- Test cases where one side has the default partition while the other side
+-- has the NULL partition
+DROP TABLE plt2_adv_p1;
+-- Add the NULL partition to plt2_adv
+CREATE TABLE plt2_adv_p1_null PARTITION OF plt2_adv FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p1_null t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_1
+                           Filter: (b < 10)
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 | 1 | 0001
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(4 rows)
+
+DROP TABLE plt2_adv_p1_null;
+-- Add the NULL partition that contains only NULL values as the key values
+CREATE TABLE plt2_adv_p1_null PARTITION OF plt2_adv FOR VALUES IN (NULL);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Seq Scan on plt2_adv_p2 t2
+         ->  Hash
+               ->  Seq Scan on plt1_adv_p2 t1
+                     Filter: (b < 10)
+(8 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(2 rows)
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+DROP TABLE plt3_adv;
+-- Test interaction of partitioned join with partition pruning
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0001');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0002');
+CREATE TABLE plt1_adv_p3 PARTITION OF plt1_adv FOR VALUES IN ('0003');
+CREATE TABLE plt1_adv_p4 PARTITION OF plt1_adv FOR VALUES IN (NULL, '0004', '0005');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN (NULL);
+CREATE TABLE plt2_adv_p3 PARTITION OF plt2_adv FOR VALUES IN ('0003');
+CREATE TABLE plt2_adv_p4 PARTITION OF plt2_adv FOR VALUES IN ('0004', '0005');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+                                       QUERY PLAN                                        
+-----------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p3 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_1
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p4 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p4 t1_2
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 5 | 0005 | 5 | 0005
+(3 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Seq Scan on plt2_adv_p4 t2
+         ->  Hash
+               ->  Seq Scan on plt1_adv_p4 t1
+                     Filter: ((c IS NULL) AND (b < 10))
+(8 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+ a  | c | a | c 
+----+---+---+---
+ -1 |   |   | 
+(1 row)
+
+CREATE TABLE plt1_adv_default PARTITION OF plt1_adv DEFAULT;
+ANALYZE plt1_adv;
+CREATE TABLE plt2_adv_default PARTITION OF plt2_adv DEFAULT;
+ANALYZE plt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+                                       QUERY PLAN                                        
+-----------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p3 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_1
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p4 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p4 t1_2
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 5 | 0005 | 5 | 0005
+(3 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Seq Scan on plt2_adv_p4 t2
+         ->  Hash
+               ->  Seq Scan on plt1_adv_p4 t1
+                     Filter: ((c IS NULL) AND (b < 10))
+(8 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+ a  | c | a | c 
+----+---+---+---
+ -1 |   |   | 
+(1 row)
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+-- Test the process_outer_partition() code path
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1_adv;
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0002');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2_adv;
+CREATE TABLE plt3_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_adv_p1 PARTITION OF plt3_adv FOR VALUES IN ('0001');
+CREATE TABLE plt3_adv_p2 PARTITION OF plt3_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3_adv;
+-- This tests that when merging partitions from plt1_adv and plt2_adv in
+-- merge_list_bounds(), process_outer_partition() returns an already-assigned
+-- merged partition when re-called with plt1_adv_p1 for the second list value
+-- '0001' of that partitin
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.c = t2.c)) FULL JOIN plt3_adv t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2.a, t3.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t3_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_1.c = t2_1.c)
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                     ->  Hash
+                           ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt3_adv_p1 t3_1
+         ->  Hash Full Join
+               Hash Cond: (t1_2.c = t3_2.c)
+               Filter: (((COALESCE(t1_2.a, 0) % 5) <> 3) AND ((COALESCE(t1_2.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_2.c = t2_2.c)
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                     ->  Hash
+                           ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt3_adv_p2 t3_2
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.c = t2.c)) FULL JOIN plt3_adv t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+ a  |  c   | a  |  c   | a  |  c   
+----+------+----+------+----+------
+  0 | 0000 |    |      |    | 
+  5 | 0000 |    |      |    | 
+ 10 | 0000 |    |      |    | 
+ 15 | 0000 |    |      |    | 
+ 20 | 0000 |    |      |    | 
+  1 | 0001 |    |      |  1 | 0001
+  1 | 0001 |    |      |  6 | 0001
+  1 | 0001 |    |      | 11 | 0001
+  1 | 0001 |    |      | 16 | 0001
+  1 | 0001 |    |      | 21 | 0001
+  6 | 0001 |    |      |  1 | 0001
+  6 | 0001 |    |      |  6 | 0001
+  6 | 0001 |    |      | 11 | 0001
+  6 | 0001 |    |      | 16 | 0001
+  6 | 0001 |    |      | 21 | 0001
+ 11 | 0001 |    |      |  1 | 0001
+ 11 | 0001 |    |      |  6 | 0001
+ 11 | 0001 |    |      | 11 | 0001
+ 11 | 0001 |    |      | 16 | 0001
+ 11 | 0001 |    |      | 21 | 0001
+ 16 | 0001 |    |      |  1 | 0001
+ 16 | 0001 |    |      |  6 | 0001
+ 16 | 0001 |    |      | 11 | 0001
+ 16 | 0001 |    |      | 16 | 0001
+ 16 | 0001 |    |      | 21 | 0001
+ 21 | 0001 |    |      |  1 | 0001
+ 21 | 0001 |    |      |  6 | 0001
+ 21 | 0001 |    |      | 11 | 0001
+ 21 | 0001 |    |      | 16 | 0001
+ 21 | 0001 |    |      | 21 | 0001
+  2 | 0002 |  2 | 0002 |    | 
+  2 | 0002 |  7 | 0002 |    | 
+  2 | 0002 | 12 | 0002 |    | 
+  2 | 0002 | 17 | 0002 |    | 
+  2 | 0002 | 22 | 0002 |    | 
+  7 | 0002 |  2 | 0002 |    | 
+  7 | 0002 |  7 | 0002 |    | 
+  7 | 0002 | 12 | 0002 |    | 
+  7 | 0002 | 17 | 0002 |    | 
+  7 | 0002 | 22 | 0002 |    | 
+ 12 | 0002 |  2 | 0002 |    | 
+ 12 | 0002 |  7 | 0002 |    | 
+ 12 | 0002 | 12 | 0002 |    | 
+ 12 | 0002 | 17 | 0002 |    | 
+ 12 | 0002 | 22 | 0002 |    | 
+ 17 | 0002 |  2 | 0002 |    | 
+ 17 | 0002 |  7 | 0002 |    | 
+ 17 | 0002 | 12 | 0002 |    | 
+ 17 | 0002 | 17 | 0002 |    | 
+ 17 | 0002 | 22 | 0002 |    | 
+ 22 | 0002 |  2 | 0002 |    | 
+ 22 | 0002 |  7 | 0002 |    | 
+ 22 | 0002 | 12 | 0002 |    | 
+ 22 | 0002 | 17 | 0002 |    | 
+ 22 | 0002 | 22 | 0002 |    | 
+(55 rows)
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+DROP TABLE plt3_adv;
+-- Tests for multi-level partitioned tables
+CREATE TABLE alpha (a double precision, b int, c text) PARTITION BY RANGE (a);
+CREATE TABLE alpha_neg PARTITION OF alpha FOR VALUES FROM ('-Infinity') TO (0) PARTITION BY RANGE (b);
+CREATE TABLE alpha_pos PARTITION OF alpha FOR VALUES FROM (0) TO (10.0) PARTITION BY LIST (c);
+CREATE TABLE alpha_neg_p1 PARTITION OF alpha_neg FOR VALUES FROM (100) TO (200);
+CREATE TABLE alpha_neg_p2 PARTITION OF alpha_neg FOR VALUES FROM (200) TO (300);
+CREATE TABLE alpha_neg_p3 PARTITION OF alpha_neg FOR VALUES FROM (300) TO (400);
+CREATE TABLE alpha_pos_p1 PARTITION OF alpha_pos FOR VALUES IN ('0001', '0003');
+CREATE TABLE alpha_pos_p2 PARTITION OF alpha_pos FOR VALUES IN ('0004', '0006');
+CREATE TABLE alpha_pos_p3 PARTITION OF alpha_pos FOR VALUES IN ('0008', '0009');
+INSERT INTO alpha_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+INSERT INTO alpha_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+ANALYZE alpha;
+CREATE TABLE beta (a double precision, b int, c text) PARTITION BY RANGE (a);
+CREATE TABLE beta_neg PARTITION OF beta FOR VALUES FROM (-10.0) TO (0) PARTITION BY RANGE (b);
+CREATE TABLE beta_pos PARTITION OF beta FOR VALUES FROM (0) TO ('Infinity') PARTITION BY LIST (c);
+CREATE TABLE beta_neg_p1 PARTITION OF beta_neg FOR VALUES FROM (100) TO (150);
+CREATE TABLE beta_neg_p2 PARTITION OF beta_neg FOR VALUES FROM (200) TO (300);
+CREATE TABLE beta_neg_p3 PARTITION OF beta_neg FOR VALUES FROM (350) TO (500);
+CREATE TABLE beta_pos_p1 PARTITION OF beta_pos FOR VALUES IN ('0002', '0003');
+CREATE TABLE beta_pos_p2 PARTITION OF beta_pos FOR VALUES IN ('0004', '0006');
+CREATE TABLE beta_pos_p3 PARTITION OF beta_pos FOR VALUES IN ('0007', '0009');
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+ANALYZE beta;
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.b
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.b = t2_1.b))
+               ->  Seq Scan on alpha_neg_p1 t1_1
+                     Filter: ((b >= 125) AND (b < 225))
+               ->  Hash
+                     ->  Seq Scan on beta_neg_p1 t2_1
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.b = t1_2.b))
+               ->  Seq Scan on beta_neg_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on alpha_neg_p2 t1_2
+                           Filter: ((b >= 125) AND (b < 225))
+         ->  Hash Join
+               Hash Cond: ((t2_4.a = t1_4.a) AND (t2_4.b = t1_4.b))
+               ->  Append
+                     ->  Seq Scan on beta_pos_p1 t2_4
+                     ->  Seq Scan on beta_pos_p2 t2_5
+                     ->  Seq Scan on beta_pos_p3 t2_6
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on alpha_pos_p1 t1_4
+                                 Filter: ((b >= 125) AND (b < 225))
+                           ->  Seq Scan on alpha_pos_p2 t1_5
+                                 Filter: ((b >= 125) AND (b < 225))
+                           ->  Seq Scan on alpha_pos_p3 t1_6
+                                 Filter: ((b >= 125) AND (b < 225))
+(29 rows)
+
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b;
+ a  |  b  |  c   | a  |  b  |  c   
+----+-----+------+----+-----+------
+ -1 | 126 | 0006 | -1 | 126 | 0006
+ -1 | 129 | 0009 | -1 | 129 | 0009
+ -1 | 133 | 0003 | -1 | 133 | 0003
+ -1 | 134 | 0004 | -1 | 134 | 0004
+ -1 | 136 | 0006 | -1 | 136 | 0006
+ -1 | 139 | 0009 | -1 | 139 | 0009
+ -1 | 143 | 0003 | -1 | 143 | 0003
+ -1 | 144 | 0004 | -1 | 144 | 0004
+ -1 | 146 | 0006 | -1 | 146 | 0006
+ -1 | 149 | 0009 | -1 | 149 | 0009
+ -1 | 203 | 0003 | -1 | 203 | 0003
+ -1 | 204 | 0004 | -1 | 204 | 0004
+ -1 | 206 | 0006 | -1 | 206 | 0006
+ -1 | 209 | 0009 | -1 | 209 | 0009
+ -1 | 213 | 0003 | -1 | 213 | 0003
+ -1 | 214 | 0004 | -1 | 214 | 0004
+ -1 | 216 | 0006 | -1 | 216 | 0006
+ -1 | 219 | 0009 | -1 | 219 | 0009
+ -1 | 223 | 0003 | -1 | 223 | 0003
+ -1 | 224 | 0004 | -1 | 224 | 0004
+  1 | 126 | 0006 |  1 | 126 | 0006
+  1 | 129 | 0009 |  1 | 129 | 0009
+  1 | 133 | 0003 |  1 | 133 | 0003
+  1 | 134 | 0004 |  1 | 134 | 0004
+  1 | 136 | 0006 |  1 | 136 | 0006
+  1 | 139 | 0009 |  1 | 139 | 0009
+  1 | 143 | 0003 |  1 | 143 | 0003
+  1 | 144 | 0004 |  1 | 144 | 0004
+  1 | 146 | 0006 |  1 | 146 | 0006
+  1 | 149 | 0009 |  1 | 149 | 0009
+  1 | 203 | 0003 |  1 | 203 | 0003
+  1 | 204 | 0004 |  1 | 204 | 0004
+  1 | 206 | 0006 |  1 | 206 | 0006
+  1 | 209 | 0009 |  1 | 209 | 0009
+  1 | 213 | 0003 |  1 | 213 | 0003
+  1 | 214 | 0004 |  1 | 214 | 0004
+  1 | 216 | 0006 |  1 | 216 | 0006
+  1 | 219 | 0009 |  1 | 219 | 0009
+  1 | 223 | 0003 |  1 | 223 | 0003
+  1 | 224 | 0004 |  1 | 224 | 0004
+(40 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b, t2.b;
+                                                              QUERY PLAN                                                              
+--------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.b, t2.b
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Append
+                     ->  Seq Scan on alpha_neg_p1 t1_2
+                           Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+                     ->  Seq Scan on alpha_neg_p2 t1_3
+                           Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on beta_neg_p1 t2_2
+                                 Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+                           ->  Seq Scan on beta_neg_p2 t2_3
+                                 Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Nested Loop
+               Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+               ->  Seq Scan on alpha_pos_p2 t1_4
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Seq Scan on beta_pos_p2 t2_4
+                     Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Nested Loop
+               Join Filter: ((t1_5.a = t2_5.a) AND (t1_5.c = t2_5.c))
+               ->  Seq Scan on alpha_pos_p3 t1_5
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Seq Scan on beta_pos_p3 t2_5
+                     Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+(28 rows)
+
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b, t2.b;
+ a  |  b  |  c   | a  |  b  |  c   
+----+-----+------+----+-----+------
+ -1 | 104 | 0004 | -1 | 104 | 0004
+ -1 | 104 | 0004 | -1 | 204 | 0004
+ -1 | 109 | 0009 | -1 | 109 | 0009
+ -1 | 109 | 0009 | -1 | 209 | 0009
+ -1 | 204 | 0004 | -1 | 104 | 0004
+ -1 | 204 | 0004 | -1 | 204 | 0004
+ -1 | 209 | 0009 | -1 | 109 | 0009
+ -1 | 209 | 0009 | -1 | 209 | 0009
+  1 | 104 | 0004 |  1 | 104 | 0004
+  1 | 104 | 0004 |  1 | 204 | 0004
+  1 | 109 | 0009 |  1 | 109 | 0009
+  1 | 109 | 0009 |  1 | 209 | 0009
+  1 | 204 | 0004 |  1 | 104 | 0004
+  1 | 204 | 0004 |  1 | 204 | 0004
+  1 | 209 | 0009 |  1 | 109 | 0009
+  1 | 209 | 0009 |  1 | 209 | 0009
 (16 rows)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;
+                                                           QUERY PLAN                                                           
+--------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.b
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on alpha_neg_p1 t1_1
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Hash
+                     ->  Seq Scan on beta_neg_p1 t2_1
+                           Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Hash Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on alpha_neg_p2 t1_2
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Hash
+                     ->  Seq Scan on beta_neg_p2 t2_2
+                           Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Nested Loop
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.b = t2_3.b) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on alpha_pos_p2 t1_3
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Seq Scan on beta_pos_p2 t2_3
+                     Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Nested Loop
+               Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.b = t2_4.b) AND (t1_4.c = t2_4.c))
+               ->  Seq Scan on alpha_pos_p3 t1_4
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Seq Scan on beta_pos_p3 t2_4
+                     Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+(29 rows)
+
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;
+ a  |  b  |  c   | a  |  b  |  c   
+----+-----+------+----+-----+------
+ -1 | 104 | 0004 | -1 | 104 | 0004
+ -1 | 109 | 0009 | -1 | 109 | 0009
+ -1 | 204 | 0004 | -1 | 204 | 0004
+ -1 | 209 | 0009 | -1 | 209 | 0009
+  1 | 104 | 0004 |  1 | 104 | 0004
+  1 | 109 | 0009 |  1 | 109 | 0009
+  1 | 204 | 0004 |  1 | 204 | 0004
+  1 | 209 | 0009 |  1 | 209 | 0009
+(8 rows)
+
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 575ba7b8d4..df30f851e8 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -445,11 +445,638 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
 
--- partitionwise join can not be applied if only one of joining tables has
--- default partition
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
-ANALYZE prt2;
 
+--
+-- Test advanced partition-matching algorithm for partitioned join
+--
+
+-- Tests for range-partitioned tables
+CREATE TABLE prt1_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_adv_p1 PARTITION OF prt1_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_adv_p2 PARTITION OF prt1_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_adv_p3 PARTITION OF prt1_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_adv_a_idx ON prt1_adv (a);
+INSERT INTO prt1_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_adv;
+
+CREATE TABLE prt2_adv (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_adv_p1 PARTITION OF prt2_adv FOR VALUES FROM (100) TO (150);
+CREATE TABLE prt2_adv_p2 PARTITION OF prt2_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt2_adv_p3 PARTITION OF prt2_adv FOR VALUES FROM (350) TO (500);
+CREATE INDEX prt2_adv_b_idx ON prt2_adv (b);
+INSERT INTO prt2_adv_p1 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 149) i;
+INSERT INTO prt2_adv_p2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(200, 299) i;
+INSERT INTO prt2_adv_p3 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_adv;
+
+-- inner join
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+-- Test cases where one side has an extra partition
+CREATE TABLE prt2_adv_extra PARTITION OF prt2_adv FOR VALUES FROM (500) TO (MAXVALUE);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(500, 599) i;
+ANALYZE prt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt1_adv t2 WHERE t1.b = t2.a) AND t1.a = 0 ORDER BY t1.b;
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+-- 3-way join where not every pair of relations can do partitioned join
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c, t3.a, t3.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) INNER JOIN prt1_adv t3 ON (t1.b = t3.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a, t3.a;
+SELECT t1.b, t1.c, t2.a, t2.c, t3.a, t3.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) INNER JOIN prt1_adv t3 ON (t1.b = t3.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a, t3.a;
+
+DROP TABLE prt2_adv_extra;
+
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in such cases
+ALTER TABLE prt2_adv DETACH PARTITION prt2_adv_p3;
+-- Split prt2_adv_p3 into two partitions so that prt1_adv_p3 matches both
+CREATE TABLE prt2_adv_p3_1 PARTITION OF prt2_adv FOR VALUES FROM (350) TO (375);
+CREATE TABLE prt2_adv_p3_2 PARTITION OF prt2_adv FOR VALUES FROM (375) TO (500);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+DROP TABLE prt2_adv_p3_1;
+DROP TABLE prt2_adv_p3_2;
+ANALYZE prt2_adv;
+
+-- Test default partitions
+ALTER TABLE prt1_adv DETACH PARTITION prt1_adv_p1;
+-- Change prt1_adv_p1 to the default partition
+ALTER TABLE prt1_adv ATTACH PARTITION prt1_adv_p1 DEFAULT;
+ALTER TABLE prt1_adv DETACH PARTITION prt1_adv_p3;
+ANALYZE prt1_adv;
+
+-- We can do partitioned join even if only one of relations has the default
+-- partition
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- Restore prt1_adv_p3
+ALTER TABLE prt1_adv ATTACH PARTITION prt1_adv_p3 FOR VALUES FROM (300) TO (400);
+ANALYZE prt1_adv;
+
+-- Restore prt2_adv_p3
+ALTER TABLE prt2_adv ATTACH PARTITION prt2_adv_p3 FOR VALUES FROM (350) TO (500);
+ANALYZE prt2_adv;
+
+-- Partitioned join can't be applied because the default partition of prt1_adv
+-- matches prt2_adv_p1 and prt2_adv_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+ALTER TABLE prt2_adv DETACH PARTITION prt2_adv_p3;
+-- Change prt2_adv_p3 to the default partition
+ALTER TABLE prt2_adv ATTACH PARTITION prt2_adv_p3 DEFAULT;
+ANALYZE prt2_adv;
+
+-- Partitioned join can't be applied because the default partition of prt1_adv
+-- matches prt2_adv_p1 and prt2_adv_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt1_adv_p3;
+ANALYZE prt1_adv;
+
+DROP TABLE prt2_adv_p3;
+ANALYZE prt2_adv;
+
+CREATE TABLE prt3_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt3_adv_p1 PARTITION OF prt3_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt3_adv_p2 PARTITION OF prt3_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt3_adv_a_idx ON prt3_adv (a);
+INSERT INTO prt3_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(200, 399) i;
+ANALYZE prt3_adv;
+
+-- 3-way join to test the default partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) LEFT JOIN prt3_adv t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) LEFT JOIN prt3_adv t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+
+DROP TABLE prt1_adv;
+DROP TABLE prt2_adv;
+DROP TABLE prt3_adv;
+
+-- Test interaction of partitioned join with partition pruning
+CREATE TABLE prt1_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_adv_p1 PARTITION OF prt1_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_adv_p2 PARTITION OF prt1_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_adv_p3 PARTITION OF prt1_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_adv_a_idx ON prt1_adv (a);
+INSERT INTO prt1_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_adv;
+
+CREATE TABLE prt2_adv (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_adv_p1 PARTITION OF prt2_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt2_adv_p2 PARTITION OF prt2_adv FOR VALUES FROM (200) TO (400);
+CREATE INDEX prt2_adv_b_idx ON prt2_adv (b);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt1_adv_p3;
+CREATE TABLE prt1_adv_default PARTITION OF prt1_adv DEFAULT;
+ANALYZE prt1_adv;
+
+CREATE TABLE prt2_adv_default PARTITION OF prt2_adv DEFAULT;
+ANALYZE prt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt1_adv;
+DROP TABLE prt2_adv;
+
+
+-- Tests for list-partitioned tables
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0001', '0003');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt1_adv_p3 PARTITION OF plt1_adv FOR VALUES IN ('0008', '0009');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+ANALYZE plt1_adv;
+
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt2_adv_p3 PARTITION OF plt2_adv FOR VALUES IN ('0007', '0009');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+-- Test cases where one side has an extra partition
+CREATE TABLE plt2_adv_extra PARTITION OF plt2_adv FOR VALUES IN ('0000');
+INSERT INTO plt2_adv_extra VALUES (0, 0, '0000');
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2_adv t1 LEFT JOIN plt1_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt2_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt1_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt2_adv_extra;
+
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in such cases
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2;
+-- Split plt2_adv_p2 into two partitions so that plt1_adv_p2 matches both
+CREATE TABLE plt2_adv_p2_1 PARTITION OF plt2_adv FOR VALUES IN ('0004');
+CREATE TABLE plt2_adv_p2_2 PARTITION OF plt2_adv FOR VALUES IN ('0006');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6);
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt2_adv_p2_1;
+DROP TABLE plt2_adv_p2_2;
+-- Restore plt2_adv_p2
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2 FOR VALUES IN ('0004', '0006');
+
+-- Test NULL partitions
+ALTER TABLE plt1_adv DETACH PARTITION plt1_adv_p1;
+-- Change plt1_adv_p1 to the NULL partition
+CREATE TABLE plt1_adv_p1_null PARTITION OF plt1_adv FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p3;
+-- Change plt2_adv_p3 to the NULL partition
+CREATE TABLE plt2_adv_p3_null PARTITION OF plt2_adv FOR VALUES IN (NULL, '0007', '0009');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (7, 9);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt1_adv_p1_null;
+-- Restore plt1_adv_p1
+ALTER TABLE plt1_adv ATTACH PARTITION plt1_adv_p1 FOR VALUES IN ('0001', '0003');
+-- Add to plt1_adv the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt1_adv_extra PARTITION OF plt1_adv FOR VALUES IN (NULL);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+
+DROP TABLE plt2_adv_p3_null;
+-- Restore plt2_adv_p3
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p3 FOR VALUES IN ('0007', '0009');
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+-- Add to plt2_adv the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt2_adv_extra PARTITION OF plt2_adv FOR VALUES IN (NULL);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+-- 3-way join to test the NULL partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt1_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt1_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt1_adv_extra;
+DROP TABLE plt2_adv_extra;
+
+-- Test default partitions
+ALTER TABLE plt1_adv DETACH PARTITION plt1_adv_p1;
+-- Change plt1_adv_p1 to the default partition
+ALTER TABLE plt1_adv ATTACH PARTITION plt1_adv_p1 DEFAULT;
+DROP TABLE plt1_adv_p3;
+ANALYZE plt1_adv;
+
+DROP TABLE plt2_adv_p3;
+ANALYZE plt2_adv;
+
+-- We can do partitioned join even if only one of relations has the default
+-- partition
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2;
+-- Change plt2_adv_p2 to contain '0005' in addition to '0004' and '0006' as
+-- the key values
+CREATE TABLE plt2_adv_p2_ext PARTITION OF plt2_adv FOR VALUES IN ('0004', '0005', '0006');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 5, 6);
+ANALYZE plt2_adv;
+
+-- Partitioned join can't be applied because the default partition of plt1_adv
+-- matches plt2_adv_p1 and plt2_adv_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2_ext;
+-- Change plt2_adv_p2_ext to the default partition
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2_ext DEFAULT;
+ANALYZE plt2_adv;
+
+-- Partitioned join can't be applied because the default partition of plt1_adv
+-- matches plt2_adv_p1 and plt2_adv_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt2_adv_p2_ext;
+-- Restore plt2_adv_p2
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_adv;
+
+CREATE TABLE plt3_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_adv_p1 PARTITION OF plt3_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt3_adv_p2 PARTITION OF plt3_adv FOR VALUES IN ('0007', '0009');
+INSERT INTO plt3_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6, 7, 9);
+ANALYZE plt3_adv;
+
+-- 3-way join to test the default partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Test cases where one side has the default partition while the other side
+-- has the NULL partition
+DROP TABLE plt2_adv_p1;
+-- Add the NULL partition to plt2_adv
+CREATE TABLE plt2_adv_p1_null PARTITION OF plt2_adv FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt2_adv_p1_null;
+-- Add the NULL partition that contains only NULL values as the key values
+CREATE TABLE plt2_adv_p1_null PARTITION OF plt2_adv FOR VALUES IN (NULL);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+DROP TABLE plt3_adv;
+
+-- Test interaction of partitioned join with partition pruning
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0001');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0002');
+CREATE TABLE plt1_adv_p3 PARTITION OF plt1_adv FOR VALUES IN ('0003');
+CREATE TABLE plt1_adv_p4 PARTITION OF plt1_adv FOR VALUES IN (NULL, '0004', '0005');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN (NULL);
+CREATE TABLE plt2_adv_p3 PARTITION OF plt2_adv FOR VALUES IN ('0003');
+CREATE TABLE plt2_adv_p4 PARTITION OF plt2_adv FOR VALUES IN ('0004', '0005');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+
+CREATE TABLE plt1_adv_default PARTITION OF plt1_adv DEFAULT;
+ANALYZE plt1_adv;
+
+CREATE TABLE plt2_adv_default PARTITION OF plt2_adv DEFAULT;
+ANALYZE plt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+
+-- Test the process_outer_partition() code path
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1_adv;
+
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0002');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2_adv;
+
+CREATE TABLE plt3_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_adv_p1 PARTITION OF plt3_adv FOR VALUES IN ('0001');
+CREATE TABLE plt3_adv_p2 PARTITION OF plt3_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3_adv;
+
+-- This tests that when merging partitions from plt1_adv and plt2_adv in
+-- merge_list_bounds(), process_outer_partition() returns an already-assigned
+-- merged partition when re-called with plt1_adv_p1 for the second list value
+-- '0001' of that partitin
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.c = t2.c)) FULL JOIN plt3_adv t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.c = t2.c)) FULL JOIN plt3_adv t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+DROP TABLE plt3_adv;
+
+
+-- Tests for multi-level partitioned tables
+CREATE TABLE alpha (a double precision, b int, c text) PARTITION BY RANGE (a);
+CREATE TABLE alpha_neg PARTITION OF alpha FOR VALUES FROM ('-Infinity') TO (0) PARTITION BY RANGE (b);
+CREATE TABLE alpha_pos PARTITION OF alpha FOR VALUES FROM (0) TO (10.0) PARTITION BY LIST (c);
+CREATE TABLE alpha_neg_p1 PARTITION OF alpha_neg FOR VALUES FROM (100) TO (200);
+CREATE TABLE alpha_neg_p2 PARTITION OF alpha_neg FOR VALUES FROM (200) TO (300);
+CREATE TABLE alpha_neg_p3 PARTITION OF alpha_neg FOR VALUES FROM (300) TO (400);
+CREATE TABLE alpha_pos_p1 PARTITION OF alpha_pos FOR VALUES IN ('0001', '0003');
+CREATE TABLE alpha_pos_p2 PARTITION OF alpha_pos FOR VALUES IN ('0004', '0006');
+CREATE TABLE alpha_pos_p3 PARTITION OF alpha_pos FOR VALUES IN ('0008', '0009');
+INSERT INTO alpha_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+INSERT INTO alpha_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+ANALYZE alpha;
+
+CREATE TABLE beta (a double precision, b int, c text) PARTITION BY RANGE (a);
+CREATE TABLE beta_neg PARTITION OF beta FOR VALUES FROM (-10.0) TO (0) PARTITION BY RANGE (b);
+CREATE TABLE beta_pos PARTITION OF beta FOR VALUES FROM (0) TO ('Infinity') PARTITION BY LIST (c);
+CREATE TABLE beta_neg_p1 PARTITION OF beta_neg FOR VALUES FROM (100) TO (150);
+CREATE TABLE beta_neg_p2 PARTITION OF beta_neg FOR VALUES FROM (200) TO (300);
+CREATE TABLE beta_neg_p3 PARTITION OF beta_neg FOR VALUES FROM (350) TO (500);
+CREATE TABLE beta_pos_p1 PARTITION OF beta_pos FOR VALUES IN ('0002', '0003');
+CREATE TABLE beta_pos_p2 PARTITION OF beta_pos FOR VALUES IN ('0004', '0006');
+CREATE TABLE beta_pos_p3 PARTITION OF beta_pos FOR VALUES IN ('0007', '0009');
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+ANALYZE beta;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b;
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b, t2.b;
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b, t2.b;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;
-- 
2.21.1

0002-remove-merged-flag.patchtext/plain; charset=us-asciiDownload
From 391c8658aebf01097dd219617fcf1e425c0883d1 Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tv@fuzzy.cz>
Date: Thu, 26 Mar 2020 00:48:46 +0100
Subject: [PATCH 2/3] remove merged flag

---
 src/backend/optimizer/path/joinrels.c | 11 +++++------
 src/backend/partitioning/partbounds.c | 12 +++++++-----
 2 files changed, 12 insertions(+), 11 deletions(-)

diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 530ebed245..0b082fc915 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1360,7 +1360,6 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	bool		merged = false;
 	List	   *parts1 = NIL;
 	List	   *parts2 = NIL;
 	ListCell   *lcr1 = NULL;
@@ -1397,6 +1396,8 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	Assert(joinrel->part_scheme == rel1->part_scheme &&
 		   joinrel->part_scheme == rel2->part_scheme);
 
+	Assert(!(joinrel->merged && (joinrel->nparts <= 0)));
+
 	/*
 	 * If we don't have the partition bounds for the join rel yet, try to
 	 * compute those along with pairs of partitions to be joined.
@@ -1446,12 +1447,11 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 				return;
 			}
 			nparts = list_length(parts1);
-			merged = true;
+			joinrel->merged = true;
 		}
 
 		Assert(nparts > 0);
 		joinrel->boundinfo = boundinfo;
-		joinrel->merged = merged;
 		joinrel->nparts = nparts;
 		joinrel->part_rels =
 			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
@@ -1475,11 +1475,10 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 									&parts1, &parts2);
 			Assert(list_length(parts1) == joinrel->nparts);
 			Assert(list_length(parts2) == joinrel->nparts);
-			merged = true;
 		}
 	}
 
-	if (merged)
+	if (joinrel->merged)
 	{
 		lcr1 = list_head(parts1);
 		lcr2 = list_head(parts2);
@@ -1503,7 +1502,7 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
-		if (merged)
+		if (joinrel->merged)
 		{
 			child_rel1 = lfirst_node(RelOptInfo, lcr1);
 			child_rel2 = lfirst_node(RelOptInfo, lcr2);
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 24dbc2c8f3..ae426c1a30 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -1018,24 +1018,25 @@ partition_bounds_merge(int partnatts,
 {
 	PartitionBoundInfo outer_binfo = outer_rel->boundinfo;
 	PartitionBoundInfo inner_binfo = inner_rel->boundinfo;
-	char		strategy;
 
 	/*
 	 * Currently, this function is called only from try_partitionwise_join(),
 	 * so the join type should be INNER, LEFT, FULL, SEMI, or ANTI.
+	 *
+	 * XXX Maybe an assert would be more appropriate? Or maybe just
+	 * bail out by returning NULL? Not sure.
 	 */
 	if (jointype != JOIN_INNER && jointype != JOIN_LEFT && 
 		jointype != JOIN_FULL && jointype != JOIN_SEMI &&
 		jointype != JOIN_ANTI)
-		elog(ERROR, "unrecognized join type: %d", (int) jointype);
+		elog(ERROR, "unexpected join type: %d", (int) jointype);
 
 	/* Bail out if the partitioning strategies are different. */
 	if (outer_binfo->strategy != inner_binfo->strategy)
 		return NULL;
 
-	strategy = outer_binfo->strategy;
 	*outer_parts = *inner_parts = NIL;
-	switch (strategy)
+	switch (outer_binfo->strategy)
 	{
 		case PARTITION_STRATEGY_HASH:
 
@@ -1075,7 +1076,8 @@ partition_bounds_merge(int partnatts,
 									  inner_parts);
 
 		default:
-			elog(ERROR, "unexpected partition strategy: %d", (int) strategy);
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) outer_binfo->strategy);
 			return NULL;				/* keep compiler quiet */
 	}
 }
-- 
2.21.1

0003-split-try_partitionwise_join.patchtext/plain; charset=us-asciiDownload
From dc5f82f8c74a04c55b03bc8c6d09ffd7f93f70ca Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tv@fuzzy.cz>
Date: Thu, 26 Mar 2020 01:15:04 +0100
Subject: [PATCH 3/3] split try_partitionwise_join

---
 src/backend/optimizer/path/joinrels.c | 146 ++++++++++++++------------
 1 file changed, 78 insertions(+), 68 deletions(-)

diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 0b082fc915..314b0267c3 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1333,71 +1333,12 @@ restriction_is_constant_false(List *restrictlist,
 	return false;
 }
 
-/*
- * Assess whether join between given two partitioned relations can be broken
- * down into joins between matching partitions; a technique called
- * "partitionwise join"
- *
- * Partitionwise join is possible when a. Joining relations have same
- * partitioning scheme b. There exists an equi-join between the partition keys
- * of the two relations.
- *
- * Partitionwise join is planned as follows (details: optimizer/README.)
- *
- * 1. Create the RelOptInfos for joins between matching partitions i.e
- * child-joins and add paths to them.
- *
- * 2. Construct Append or MergeAppend paths across the set of child joins.
- * This second phase is implemented by generate_partitionwise_join_paths().
- *
- * The RelOptInfo, SpecialJoinInfo and restrictlist for each child join are
- * obtained by translating the respective parent join structures.
- */
 static void
-try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
-					   RelOptInfo *joinrel, SpecialJoinInfo *parent_sjinfo,
-					   List *parent_restrictlist)
+compute_partition_bounds(PlannerInfo *root, RelOptInfo *rel1,
+						 RelOptInfo *rel2, RelOptInfo *joinrel,
+						 SpecialJoinInfo *parent_sjinfo,
+						 List **parts1, List **parts2)
 {
-	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
-	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	List	   *parts1 = NIL;
-	List	   *parts2 = NIL;
-	ListCell   *lcr1 = NULL;
-	ListCell   *lcr2 = NULL;
-	int			cnt_parts;
-
-	/* Guard against stack overflow due to overly deep partition hierarchy. */
-	check_stack_depth();
-
-	/* Nothing to do, if the join relation is not partitioned. */
-	if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
-		return;
-
-	/* The join relation should have consider_partitionwise_join set. */
-	Assert(joinrel->consider_partitionwise_join);
-
-	/*
-	 * We can not perform partitionwise join if either of the joining relations
-	 * is not partitioned.
-	 */
-	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
-		return;
-
-	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
-
-	/* The joining relations should have consider_partitionwise_join set. */
-	Assert(rel1->consider_partitionwise_join &&
-		   rel2->consider_partitionwise_join);
-
-	/*
-	 * The partition scheme of the join relation should match that of the
-	 * joining relations.
-	 */
-	Assert(joinrel->part_scheme == rel1->part_scheme &&
-		   joinrel->part_scheme == rel2->part_scheme);
-
-	Assert(!(joinrel->merged && (joinrel->nparts <= 0)));
-
 	/*
 	 * If we don't have the partition bounds for the join rel yet, try to
 	 * compute those along with pairs of partitions to be joined.
@@ -1440,13 +1381,13 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 											   part_scheme->partcollation,
 											   rel1, rel2,
 											   parent_sjinfo->jointype,
-											   &parts1, &parts2);
+											   parts1, parts2);
 			if (boundinfo == NULL)
 			{
 				joinrel->nparts = 0;
 				return;
 			}
-			nparts = list_length(parts1);
+			nparts = list_length(*parts1);
 			joinrel->merged = true;
 		}
 
@@ -1472,11 +1413,80 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		if (joinrel->merged)
 		{
 			get_matching_part_pairs(root, joinrel, rel1, rel2,
-									&parts1, &parts2);
-			Assert(list_length(parts1) == joinrel->nparts);
-			Assert(list_length(parts2) == joinrel->nparts);
+									parts1, parts2);
+			Assert(list_length(*parts1) == joinrel->nparts);
+			Assert(list_length(*parts2) == joinrel->nparts);
 		}
 	}
+}
+
+/*
+ * Assess whether join between given two partitioned relations can be broken
+ * down into joins between matching partitions; a technique called
+ * "partitionwise join"
+ *
+ * Partitionwise join is possible when a. Joining relations have same
+ * partitioning scheme b. There exists an equi-join between the partition keys
+ * of the two relations.
+ *
+ * Partitionwise join is planned as follows (details: optimizer/README.)
+ *
+ * 1. Create the RelOptInfos for joins between matching partitions i.e
+ * child-joins and add paths to them.
+ *
+ * 2. Construct Append or MergeAppend paths across the set of child joins.
+ * This second phase is implemented by generate_partitionwise_join_paths().
+ *
+ * The RelOptInfo, SpecialJoinInfo and restrictlist for each child join are
+ * obtained by translating the respective parent join structures.
+ */
+static void
+try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
+					   RelOptInfo *joinrel, SpecialJoinInfo *parent_sjinfo,
+					   List *parent_restrictlist)
+{
+	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
+	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
+	List	   *parts1 = NIL;
+	List	   *parts2 = NIL;
+	ListCell   *lcr1 = NULL;
+	ListCell   *lcr2 = NULL;
+	int			cnt_parts;
+
+	/* Guard against stack overflow due to overly deep partition hierarchy. */
+	check_stack_depth();
+
+	/* Nothing to do, if the join relation is not partitioned. */
+	if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
+		return;
+
+	/* The join relation should have consider_partitionwise_join set. */
+	Assert(joinrel->consider_partitionwise_join);
+
+	/*
+	 * We can not perform partitionwise join if either of the joining relations
+	 * is not partitioned.
+	 */
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
+	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
+
+	/* The joining relations should have consider_partitionwise_join set. */
+	Assert(rel1->consider_partitionwise_join &&
+		   rel2->consider_partitionwise_join);
+
+	/*
+	 * The partition scheme of the join relation should match that of the
+	 * joining relations.
+	 */
+	Assert(joinrel->part_scheme == rel1->part_scheme &&
+		   joinrel->part_scheme == rel2->part_scheme);
+
+	Assert(!(joinrel->merged && (joinrel->nparts <= 0)));
+
+	compute_partition_bounds(root, rel1, rel2, joinrel, parent_sjinfo,
+							 &parts1, &parts2);
 
 	if (joinrel->merged)
 	{
-- 
2.21.1

#127Ashutosh Bapat
ashutosh.bapat@2ndquadrant.com
In reply to: Tomas Vondra (#126)
5 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Thu, 26 Mar 2020 at 05:47, Tomas Vondra <tomas.vondra@2ndquadrant.com>
wrote:

Hi,

three more comments after eye-balling the code for a bit longer.

1) The patch probably needs to tweak config.sgml which says this about
the enable_partitionwise_join GUC:

.. Partitionwise join currently applies only when the join conditions
include all the partition keys, which must be of the same data type
and have exactly matching sets of child partitions. ..

Done. Actually this wasn't updated when partition pruning was introduced,
which could cause a partitionwise join to be not used even when those
conditions were met. Similarly when a query involved whole row reference.
It's hard to explain all the conditions under which partitionwise join
technique will be used. But I have tried to keep it easy to understand.

Which is probably incorrect, because the point of this patch is not to
require exact match of the partitions, right?

2) Do we really need the 'merged' flag in try_partitionwise_join? Can't
we simply use the joinrel->merged flag directly? ISTM the we always
start with joinrel->merged=false, and then only ever set it to true in
some cases. I've tried doing that, per the attached 0002 patch. The
regression tests seem to work fine.

Thanks. I just added a small prologue to compute_partition_bounds().

I noticed this because I've tried moving part of the function into a
separate function, and removing the variable makes that simpler.

The patch also does a couple additional minor tweaks.

3) I think the for nparts comment is somewhat misleading:

int nparts; /* number of partitions; 0 = not partitioned;
* -1 = not yet set */

which says that nparts=0 means not partitioned. But then there are
conditions like this:

/* Nothing to do, if the join relation is not partitioned. */
if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
return;

which (ignoring the obsolete comment) seems to say we can have nparts==0
even for partitioned tables, no?

See my previous mail.

Anyway, attached is the original patch (0001) and two patches with
proposed changes. 0002 removes the "merged" flag as explained in (2),
0003 splits the try_partitionwise_join() function into two parts.

I'm saying these changes have to happen and it's a bit crude (and it
might be a bit of a bikeshedding).

I have added 0005 with the changes I described in this as well as the
previous mail. 0004 is just some white space fixes.

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

--
Best Wishes,
Ashutosh

Attachments:

0002-remove-merged-flag.patchtext/x-patch; charset=US-ASCII; name=0002-remove-merged-flag.patchDownload
From 59a3bd7b33fcf9c89ea46c4efe3e4912477236fb Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tv@fuzzy.cz>
Date: Thu, 26 Mar 2020 00:48:46 +0100
Subject: [PATCH 2/5] remove merged flag

---
 src/backend/optimizer/path/joinrels.c | 11 +++++------
 src/backend/partitioning/partbounds.c | 12 +++++++-----
 2 files changed, 12 insertions(+), 11 deletions(-)

diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 530ebed245..0b082fc915 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1360,7 +1360,6 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	bool		merged = false;
 	List	   *parts1 = NIL;
 	List	   *parts2 = NIL;
 	ListCell   *lcr1 = NULL;
@@ -1397,6 +1396,8 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	Assert(joinrel->part_scheme == rel1->part_scheme &&
 		   joinrel->part_scheme == rel2->part_scheme);
 
+	Assert(!(joinrel->merged && (joinrel->nparts <= 0)));
+
 	/*
 	 * If we don't have the partition bounds for the join rel yet, try to
 	 * compute those along with pairs of partitions to be joined.
@@ -1446,12 +1447,11 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 				return;
 			}
 			nparts = list_length(parts1);
-			merged = true;
+			joinrel->merged = true;
 		}
 
 		Assert(nparts > 0);
 		joinrel->boundinfo = boundinfo;
-		joinrel->merged = merged;
 		joinrel->nparts = nparts;
 		joinrel->part_rels =
 			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
@@ -1475,11 +1475,10 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 									&parts1, &parts2);
 			Assert(list_length(parts1) == joinrel->nparts);
 			Assert(list_length(parts2) == joinrel->nparts);
-			merged = true;
 		}
 	}
 
-	if (merged)
+	if (joinrel->merged)
 	{
 		lcr1 = list_head(parts1);
 		lcr2 = list_head(parts2);
@@ -1503,7 +1502,7 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
-		if (merged)
+		if (joinrel->merged)
 		{
 			child_rel1 = lfirst_node(RelOptInfo, lcr1);
 			child_rel2 = lfirst_node(RelOptInfo, lcr2);
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 24dbc2c8f3..ae426c1a30 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -1018,24 +1018,25 @@ partition_bounds_merge(int partnatts,
 {
 	PartitionBoundInfo outer_binfo = outer_rel->boundinfo;
 	PartitionBoundInfo inner_binfo = inner_rel->boundinfo;
-	char		strategy;
 
 	/*
 	 * Currently, this function is called only from try_partitionwise_join(),
 	 * so the join type should be INNER, LEFT, FULL, SEMI, or ANTI.
+	 *
+	 * XXX Maybe an assert would be more appropriate? Or maybe just
+	 * bail out by returning NULL? Not sure.
 	 */
 	if (jointype != JOIN_INNER && jointype != JOIN_LEFT && 
 		jointype != JOIN_FULL && jointype != JOIN_SEMI &&
 		jointype != JOIN_ANTI)
-		elog(ERROR, "unrecognized join type: %d", (int) jointype);
+		elog(ERROR, "unexpected join type: %d", (int) jointype);
 
 	/* Bail out if the partitioning strategies are different. */
 	if (outer_binfo->strategy != inner_binfo->strategy)
 		return NULL;
 
-	strategy = outer_binfo->strategy;
 	*outer_parts = *inner_parts = NIL;
-	switch (strategy)
+	switch (outer_binfo->strategy)
 	{
 		case PARTITION_STRATEGY_HASH:
 
@@ -1075,7 +1076,8 @@ partition_bounds_merge(int partnatts,
 									  inner_parts);
 
 		default:
-			elog(ERROR, "unexpected partition strategy: %d", (int) strategy);
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) outer_binfo->strategy);
 			return NULL;				/* keep compiler quiet */
 	}
 }
-- 
2.17.1

0003-split-try_partitionwise_join.patchtext/x-patch; charset=US-ASCII; name=0003-split-try_partitionwise_join.patchDownload
From 04ff682ae15136a987d97c3ef6ec571327979019 Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tv@fuzzy.cz>
Date: Thu, 26 Mar 2020 01:15:04 +0100
Subject: [PATCH 3/5] split try_partitionwise_join

---
 src/backend/optimizer/path/joinrels.c | 146 ++++++++++++++------------
 1 file changed, 78 insertions(+), 68 deletions(-)

diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 0b082fc915..314b0267c3 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1333,71 +1333,12 @@ restriction_is_constant_false(List *restrictlist,
 	return false;
 }
 
-/*
- * Assess whether join between given two partitioned relations can be broken
- * down into joins between matching partitions; a technique called
- * "partitionwise join"
- *
- * Partitionwise join is possible when a. Joining relations have same
- * partitioning scheme b. There exists an equi-join between the partition keys
- * of the two relations.
- *
- * Partitionwise join is planned as follows (details: optimizer/README.)
- *
- * 1. Create the RelOptInfos for joins between matching partitions i.e
- * child-joins and add paths to them.
- *
- * 2. Construct Append or MergeAppend paths across the set of child joins.
- * This second phase is implemented by generate_partitionwise_join_paths().
- *
- * The RelOptInfo, SpecialJoinInfo and restrictlist for each child join are
- * obtained by translating the respective parent join structures.
- */
 static void
-try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
-					   RelOptInfo *joinrel, SpecialJoinInfo *parent_sjinfo,
-					   List *parent_restrictlist)
+compute_partition_bounds(PlannerInfo *root, RelOptInfo *rel1,
+						 RelOptInfo *rel2, RelOptInfo *joinrel,
+						 SpecialJoinInfo *parent_sjinfo,
+						 List **parts1, List **parts2)
 {
-	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
-	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	List	   *parts1 = NIL;
-	List	   *parts2 = NIL;
-	ListCell   *lcr1 = NULL;
-	ListCell   *lcr2 = NULL;
-	int			cnt_parts;
-
-	/* Guard against stack overflow due to overly deep partition hierarchy. */
-	check_stack_depth();
-
-	/* Nothing to do, if the join relation is not partitioned. */
-	if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
-		return;
-
-	/* The join relation should have consider_partitionwise_join set. */
-	Assert(joinrel->consider_partitionwise_join);
-
-	/*
-	 * We can not perform partitionwise join if either of the joining relations
-	 * is not partitioned.
-	 */
-	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
-		return;
-
-	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
-
-	/* The joining relations should have consider_partitionwise_join set. */
-	Assert(rel1->consider_partitionwise_join &&
-		   rel2->consider_partitionwise_join);
-
-	/*
-	 * The partition scheme of the join relation should match that of the
-	 * joining relations.
-	 */
-	Assert(joinrel->part_scheme == rel1->part_scheme &&
-		   joinrel->part_scheme == rel2->part_scheme);
-
-	Assert(!(joinrel->merged && (joinrel->nparts <= 0)));
-
 	/*
 	 * If we don't have the partition bounds for the join rel yet, try to
 	 * compute those along with pairs of partitions to be joined.
@@ -1440,13 +1381,13 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 											   part_scheme->partcollation,
 											   rel1, rel2,
 											   parent_sjinfo->jointype,
-											   &parts1, &parts2);
+											   parts1, parts2);
 			if (boundinfo == NULL)
 			{
 				joinrel->nparts = 0;
 				return;
 			}
-			nparts = list_length(parts1);
+			nparts = list_length(*parts1);
 			joinrel->merged = true;
 		}
 
@@ -1472,11 +1413,80 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		if (joinrel->merged)
 		{
 			get_matching_part_pairs(root, joinrel, rel1, rel2,
-									&parts1, &parts2);
-			Assert(list_length(parts1) == joinrel->nparts);
-			Assert(list_length(parts2) == joinrel->nparts);
+									parts1, parts2);
+			Assert(list_length(*parts1) == joinrel->nparts);
+			Assert(list_length(*parts2) == joinrel->nparts);
 		}
 	}
+}
+
+/*
+ * Assess whether join between given two partitioned relations can be broken
+ * down into joins between matching partitions; a technique called
+ * "partitionwise join"
+ *
+ * Partitionwise join is possible when a. Joining relations have same
+ * partitioning scheme b. There exists an equi-join between the partition keys
+ * of the two relations.
+ *
+ * Partitionwise join is planned as follows (details: optimizer/README.)
+ *
+ * 1. Create the RelOptInfos for joins between matching partitions i.e
+ * child-joins and add paths to them.
+ *
+ * 2. Construct Append or MergeAppend paths across the set of child joins.
+ * This second phase is implemented by generate_partitionwise_join_paths().
+ *
+ * The RelOptInfo, SpecialJoinInfo and restrictlist for each child join are
+ * obtained by translating the respective parent join structures.
+ */
+static void
+try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
+					   RelOptInfo *joinrel, SpecialJoinInfo *parent_sjinfo,
+					   List *parent_restrictlist)
+{
+	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
+	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
+	List	   *parts1 = NIL;
+	List	   *parts2 = NIL;
+	ListCell   *lcr1 = NULL;
+	ListCell   *lcr2 = NULL;
+	int			cnt_parts;
+
+	/* Guard against stack overflow due to overly deep partition hierarchy. */
+	check_stack_depth();
+
+	/* Nothing to do, if the join relation is not partitioned. */
+	if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
+		return;
+
+	/* The join relation should have consider_partitionwise_join set. */
+	Assert(joinrel->consider_partitionwise_join);
+
+	/*
+	 * We can not perform partitionwise join if either of the joining relations
+	 * is not partitioned.
+	 */
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
+	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
+
+	/* The joining relations should have consider_partitionwise_join set. */
+	Assert(rel1->consider_partitionwise_join &&
+		   rel2->consider_partitionwise_join);
+
+	/*
+	 * The partition scheme of the join relation should match that of the
+	 * joining relations.
+	 */
+	Assert(joinrel->part_scheme == rel1->part_scheme &&
+		   joinrel->part_scheme == rel2->part_scheme);
+
+	Assert(!(joinrel->merged && (joinrel->nparts <= 0)));
+
+	compute_partition_bounds(root, rel1, rel2, joinrel, parent_sjinfo,
+							 &parts1, &parts2);
 
 	if (joinrel->merged)
 	{
-- 
2.17.1

0001-v33.patchtext/x-patch; charset=US-ASCII; name=0001-v33.patchDownload
From b496b5f811ec2b5f534e64d40e081e128bc89368 Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tv@fuzzy.cz>
Date: Wed, 25 Mar 2020 22:43:28 +0100
Subject: [PATCH 1/5] v33

---
 src/backend/nodes/outfuncs.c                 |    2 +
 src/backend/optimizer/path/joinrels.c        |  242 +-
 src/backend/optimizer/util/inherit.c         |    2 +
 src/backend/optimizer/util/relnode.c         |   43 +-
 src/backend/partitioning/partbounds.c        | 1838 ++++++++++++
 src/include/nodes/pathnodes.h                |    8 +-
 src/include/partitioning/partbounds.h        |    9 +
 src/test/regress/expected/partition_join.out | 2608 +++++++++++++++++-
 src/test/regress/sql/partition_join.sql      |  639 ++++-
 9 files changed, 5315 insertions(+), 76 deletions(-)

diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index e084c3f069..79c768409c 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -2288,6 +2288,8 @@ _outRelOptInfo(StringInfo str, const RelOptInfo *node)
 	WRITE_BOOL_FIELD(has_eclass_joins);
 	WRITE_BOOL_FIELD(consider_partitionwise_join);
 	WRITE_BITMAPSET_FIELD(top_parent_relids);
+	WRITE_BOOL_FIELD(merged);
+	WRITE_BITMAPSET_FIELD(all_partrels);
 	WRITE_NODE_FIELD(partitioned_child_rels);
 }
 
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index a21c295b99..530ebed245 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -48,6 +48,9 @@ static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 												Relids left_relids, Relids right_relids);
 static int	match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel,
 										 bool strict_op);
+static void get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2);
 
 
 /*
@@ -1357,25 +1360,30 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	bool		merged = false;
+	List	   *parts1 = NIL;
+	List	   *parts2 = NIL;
+	ListCell   *lcr1 = NULL;
+	ListCell   *lcr2 = NULL;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partitionwise join if either of the joining relations
+	 * is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1390,34 +1398,104 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		   joinrel->part_scheme == rel2->part_scheme);
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * If we don't have the partition bounds for the join rel yet, try to
+	 * compute those along with pairs of partitions to be joined.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	if (joinrel->nparts == -1)
+	{
+		PartitionScheme part_scheme = joinrel->part_scheme;
+		PartitionBoundInfo boundinfo = NULL;
+		int			nparts = 0;
+
+		Assert(joinrel->boundinfo == NULL);
+		Assert(joinrel->part_rels == NULL);
+
+		/*
+		 * See if the partition bounds for inputs are exactly the same, in
+		 * which case we don't need to work hard: the join rel have the same
+		 * partition bounds as inputs, and the partitions with the same
+		 * cardinal positions form the pairs.
+		 *
+		 * Note: even in cases where one or both inputs have merged bounds,
+		 * it would be possible for both the bounds to be exactly the same, but
+		 * it seems unlikely to be worth the cycles to check.
+		 */
+		if (!rel1->merged &&
+			!rel2->merged &&
+			rel1->nparts == rel2->nparts &&
+			partition_bounds_equal(part_scheme->partnatts,
+								   part_scheme->parttyplen,
+								   part_scheme->parttypbyval,
+								   rel1->boundinfo, rel2->boundinfo))
+		{
+			boundinfo = rel1->boundinfo;
+			nparts = rel1->nparts;
+		}
+		else
+		{
+			/* Try merging the partition bounds for inputs. */
+			boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											   part_scheme->partsupfunc,
+											   part_scheme->partcollation,
+											   rel1, rel2,
+											   parent_sjinfo->jointype,
+											   &parts1, &parts2);
+			if (boundinfo == NULL)
+			{
+				joinrel->nparts = 0;
+				return;
+			}
+			nparts = list_length(parts1);
+			merged = true;
+		}
+
+		Assert(nparts > 0);
+		joinrel->boundinfo = boundinfo;
+		joinrel->merged = merged;
+		joinrel->nparts = nparts;
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+	}
+	else
+	{
+		Assert(joinrel->nparts > 0);
+		Assert(joinrel->boundinfo);
+		Assert(joinrel->part_rels);
+
+		/*
+		 * If the join rel's merged flag is true, it means inputs are not
+		 * guaranteed to have the same partition bounds, therefore we can't
+		 * assume that the partitions at the same cardinal positions form the
+		 * pairs; let get_matching_part_pairs() generate the pairs.  Otherwise,
+		 * nothing to do since we can assume that.
+		 */
+		if (joinrel->merged)
+		{
+			get_matching_part_pairs(root, joinrel, rel1, rel2,
+									&parts1, &parts2);
+			Assert(list_length(parts1) == joinrel->nparts);
+			Assert(list_length(parts2) == joinrel->nparts);
+			merged = true;
+		}
+	}
 
-	nparts = joinrel->nparts;
+	if (merged)
+	{
+		lcr1 = list_head(parts1);
+		lcr2 = list_head(parts2);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
-		bool		rel1_empty = (child_rel1 == NULL ||
-								  IS_DUMMY_REL(child_rel1));
-		bool		rel2_empty = (child_rel2 == NULL ||
-								  IS_DUMMY_REL(child_rel2));
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		bool		rel1_empty;
+		bool		rel2_empty;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1425,6 +1503,22 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		if (merged)
+		{
+			child_rel1 = lfirst_node(RelOptInfo, lcr1);
+			child_rel2 = lfirst_node(RelOptInfo, lcr2);
+			lcr1 = lnext(parts1, lcr1);
+			lcr2 = lnext(parts2, lcr2);
+		}
+		else
+		{
+			child_rel1 = rel1->part_rels[cnt_parts];
+			child_rel2 = rel2->part_rels[cnt_parts];
+		}
+
+		rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1));
+		rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2));
+
 		/*
 		 * Check for cases where we can prove that this segment of the join
 		 * returns no rows, due to one or both inputs being empty (including
@@ -1522,6 +1616,8 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 												 child_sjinfo,
 												 child_sjinfo->jointype);
 			joinrel->part_rels[cnt_parts] = child_joinrel;
+			joinrel->all_partrels = bms_add_members(joinrel->all_partrels,
+													child_joinrel->relids);
 		}
 
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
@@ -1738,3 +1834,99 @@ match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op)
 
 	return -1;
 }
+
+/*
+ * get_matching_part_pairs
+ *		Generate pairs of partitions to be joined from the two inputs
+ */
+static void
+get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2)
+{
+	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
+	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
+	int 		cnt_parts;
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
+	{
+		RelOptInfo *child_joinrel = joinrel->part_rels[cnt_parts];
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		Relids		child_relids1;
+		Relids		child_relids2;
+
+		/*
+		 * If this segment of the join is empty, it means that this segment
+		 * was ignored when previously creating child-join paths for it in
+		 * try_partitionwise_join() as it would not contribute to the join
+		 * result, due to one or both inputs being empty; add NULL to each of
+		 * the given lists so that this segment will be ignored again in that
+		 * function.
+		 */
+		if (!child_joinrel)
+		{
+			*parts1 = lappend(*parts1, NULL);
+			*parts2 = lappend(*parts2, NULL);
+			continue;
+		}
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel1 side.
+		 */
+		child_relids1 = bms_intersect(child_joinrel->relids,
+									  rel1->all_partrels);
+		Assert(bms_num_members(child_relids1) == bms_num_members(rel1->relids));
+
+		/*
+		 * Get a child rel for rel1 with the relids.  Note that we should have
+		 * the child rel even if rel1 is a join rel, because in that case the
+		 * partitions specified in the relids would have matching/overlapping
+		 * boundaries, so the specified partitions should be considered as ones
+		 * to be joined when planning partitionwise joins of rel1, meaning that
+		 * the child rel would have been built by the time we get here.
+		 */
+		if (rel1_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids1);
+
+			child_rel1 = find_base_rel(root, varno);
+		}
+		else
+			child_rel1 = find_join_rel(root, child_relids1);
+		Assert(child_rel1);
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel2 side.
+		 */
+		child_relids2 = bms_intersect(child_joinrel->relids,
+									  rel2->all_partrels);
+		Assert(bms_num_members(child_relids2) == bms_num_members(rel2->relids));
+
+		/*
+		 * Get a child rel for rel2 with the relids.  See above comments.
+		 */
+		if (rel2_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids2);
+
+			child_rel2 = find_base_rel(root, varno);
+		}
+		else
+			child_rel2 = find_join_rel(root, child_relids2);
+		Assert(child_rel2);
+
+		/*
+		 * The join of rel1 and rel2 is legal, so is the join of the child
+		 * rels obtained above; add them to the given lists as a join pair
+		 * producing this join segment.
+		 */
+		*parts1 = lappend(*parts1, child_rel1);
+		*parts2 = lappend(*parts2, child_rel2);
+	}
+}
diff --git a/src/backend/optimizer/util/inherit.c b/src/backend/optimizer/util/inherit.c
index 7db67fdf34..3132fd35a5 100644
--- a/src/backend/optimizer/util/inherit.c
+++ b/src/backend/optimizer/util/inherit.c
@@ -376,6 +376,8 @@ expand_partitioned_rtentry(PlannerInfo *root, RelOptInfo *relinfo,
 		/* Create the otherrel RelOptInfo too. */
 		childrelinfo = build_simple_rel(root, childRTindex, relinfo);
 		relinfo->part_rels[i] = childrelinfo;
+		relinfo->all_partrels = bms_add_members(relinfo->all_partrels,
+												childrelinfo->relids);
 
 		/* If this child is itself partitioned, recurse */
 		if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 374f93890b..0e4944ac8e 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -240,10 +240,12 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->has_eclass_joins = false;
 	rel->consider_partitionwise_join = false;	/* might get changed later */
 	rel->part_scheme = NULL;
-	rel->nparts = 0;
+	rel->nparts = -1;
 	rel->boundinfo = NULL;
+	rel->merged = false;
 	rel->partition_qual = NIL;
 	rel->part_rels = NULL;
+	rel->all_partrels = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->partitioned_child_rels = NIL;
@@ -653,10 +655,12 @@ build_join_rel(PlannerInfo *root,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -829,10 +833,12 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -1639,7 +1645,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1652,24 +1658,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1680,18 +1668,17 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 		   !joinrel->boundinfo);
 
 	/*
-	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * If the join relation is partitioned, it use the same partitioning scheme
+	 * as the joining relations.
+	 *
+	 * Note: we calculate the partition bounds, number of partitions, and
+	 * child-join relations of the join relation in try_partitionwise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 4c47f54a57..24dbc2c8f3 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -26,6 +26,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/pathnodes.h"
 #include "parser/parse_coerce.h"
 #include "partitioning/partbounds.h"
 #include "partitioning/partdesc.h"
@@ -68,6 +69,25 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+/* Per-partitioned-relation data for merge_list_bounds()/merge_range_bounds() */
+typedef struct PartitionMap
+{
+	int			nparts;			/* number of partitions */
+	int		   *merged_indexes;	/* indexes of merged partitions */
+	bool	   *merged;			/* flags to indicate whether partitions are
+								 * merged with non-dummy partitions */
+	bool		did_remapping;	/* did we re-map partitions? */
+	int		   *old_indexes;	/* old indexes of merged partitions if
+								 * did_remapping */
+} PartitionMap;
+
+/* Macro for comparing two range bounds */
+#define compare_range_bounds(partnatts, partsupfunc, partcollations, \
+							 bound1, bound2) \
+	(partition_rbound_cmp(partnatts, partsupfunc, partcollations, \
+						  (bound1)->datums, (bound1)->kind, (bound1)->lower, \
+						  bound2))
+
 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);
@@ -79,6 +99,116 @@ static PartitionBoundInfo create_list_bounds(PartitionBoundSpec **boundspecs,
 											 int nparts, PartitionKey key, int **mapping);
 static PartitionBoundInfo create_range_bounds(PartitionBoundSpec **boundspecs,
 											  int nparts, PartitionKey key, int **mapping);
+static PartitionBoundInfo merge_list_bounds(FmgrInfo *partsupfunc,
+											Oid *collations,
+											RelOptInfo *outer_rel,
+											RelOptInfo *inner_rel,
+											JoinType jointype,
+											List **outer_parts,
+											List **inner_parts);
+static PartitionBoundInfo merge_range_bounds(int partnatts,
+											 FmgrInfo *partsupfuncs,
+											 Oid *partcollations,
+											 RelOptInfo *outer_rel,
+											 RelOptInfo *inner_rel,
+											 JoinType jointype,
+											 List **outer_parts,
+											 List **inner_parts);
+static void init_partition_map(RelOptInfo *rel, PartitionMap *map);
+static void free_partition_map(PartitionMap *map);
+static bool is_dummy_partition(RelOptInfo *rel, int part_index);
+static int merge_matching_partitions(PartitionMap *outer_map,
+									 PartitionMap *inner_map,
+									 int outer_part,
+									 int inner_part,
+									 int *next_index);
+static int process_outer_partition(PartitionMap *outer_map,
+								   PartitionMap *inner_map,
+								   bool outer_has_default,
+								   bool inner_has_default,
+								   int outer_index,
+								   int inner_default,
+								   JoinType jointype,
+								   int *next_index,
+								   int *default_index);
+static int process_inner_partition(PartitionMap *outer_map,
+								   PartitionMap *inner_map,
+								   bool outer_has_default,
+								   bool inner_has_default,
+								   int inner_index,
+								   int outer_default,
+								   JoinType jointype,
+								   int *next_index,
+								   int *default_index);
+static void merge_null_partitions(PartitionMap *outer_map,
+								  PartitionMap *inner_map,
+								  bool outer_has_null,
+								  bool inner_has_null,
+								  int outer_null,
+								  int inner_null,
+								  JoinType jointype,
+								  int *next_index,
+								  int *null_index);
+static void merge_default_partitions(PartitionMap *outer_map,
+									 PartitionMap *inner_map,
+									 bool outer_has_default,
+									 bool inner_has_default,
+									 int outer_default,
+									 int inner_default,
+									 JoinType jointype,
+									 int *next_index,
+									 int *default_index);
+static int merge_partition_with_dummy(PartitionMap *map, int index,
+									  int *next_index);
+static void fix_merged_indexes(PartitionMap *outer_map,
+							   PartitionMap *inner_map,
+							   int nmerged, List *merged_indexes);
+static void generate_matching_part_pairs(RelOptInfo *outer_rel,
+										 RelOptInfo *inner_rel,
+										 PartitionMap *outer_map,
+										 PartitionMap *inner_map,
+										 int nmerged,
+										 List **outer_parts,
+										 List **inner_parts);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+														List *merged_datums,
+														List *merged_kinds,
+														List *merged_indexes,
+														int null_index,
+														int default_index);
+static int get_range_partition(RelOptInfo *rel,
+							   PartitionBoundInfo bi,
+							   int *lb_pos,
+							   PartitionRangeBound *lb,
+							   PartitionRangeBound *ub);
+static int get_range_partition_internal(PartitionBoundInfo bi,
+										int *lb_pos,
+										PartitionRangeBound *lb,
+										PartitionRangeBound *ub);
+static bool compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+									 Oid *partcollations,
+									 PartitionRangeBound *outer_lb,
+									 PartitionRangeBound *outer_ub,
+									 PartitionRangeBound *inner_lb,
+									 PartitionRangeBound *inner_ub,
+									 int *lb_cmpval, int *ub_cmpval);
+static void get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+									Oid *partcollations, JoinType jointype,
+									PartitionRangeBound *outer_lb,
+									PartitionRangeBound *outer_ub,
+									PartitionRangeBound *inner_lb,
+									PartitionRangeBound *inner_ub,
+									int	lb_cmpval, int ub_cmpval,
+									PartitionRangeBound *merged_lb,
+									PartitionRangeBound *merged_ub);
+static void add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+									Oid *partcollations,
+									PartitionRangeBound *merged_lb,
+									PartitionRangeBound *merged_ub,
+									int merged_index,
+									List **merged_datums,
+									List **merged_kinds,
+									List **merged_indexes);
 static PartitionRangeBound *make_one_partition_rbound(PartitionKey key, int index,
 													  List *datums, bool lower);
 static int32 partition_hbound_cmp(int modulus1, int remainder1, int modulus2,
@@ -866,6 +996,1714 @@ partition_bounds_copy(PartitionBoundInfo src,
 	return dest;
 }
 
+/*
+ * partition_bounds_merge
+ *
+ * This function builds and returns the partition bounds for a join relation
+ * between input relations, creating two lists of partitions, which are
+ * returned to *outer_parts and *inner_parts respectively.  The lists contain
+ * the same number of partitions, and the partitions at the same positions in
+ * the lists indicate join pairs used for partitioned join.
+ *
+ * This function returns NULL, setting *outer_parts and *inner_parts to NIL,
+ * if a partition on one side matches multiple partitions on the other side,
+ * in which case we currently don't support partitioned join.
+ */
+PartitionBoundInfo
+partition_bounds_merge(int partnatts,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype,
+					   List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo outer_binfo = outer_rel->boundinfo;
+	PartitionBoundInfo inner_binfo = inner_rel->boundinfo;
+	char		strategy;
+
+	/*
+	 * Currently, this function is called only from try_partitionwise_join(),
+	 * so the join type should be INNER, LEFT, FULL, SEMI, or ANTI.
+	 */
+	if (jointype != JOIN_INNER && jointype != JOIN_LEFT && 
+		jointype != JOIN_FULL && jointype != JOIN_SEMI &&
+		jointype != JOIN_ANTI)
+		elog(ERROR, "unrecognized join type: %d", (int) jointype);
+
+	/* Bail out if the partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	strategy = outer_binfo->strategy;
+	*outer_parts = *inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+
+			/*
+			 * For hash partitioned tables, we currently support partitioned
+			 * join only when the partition bounds for them exactly match.
+			 *
+			 * XXX: it might be possible to relax the restriction to support
+			 * cases where hash partitioned tables have missing partitions
+			 * and/or different moduli, but it's not clear if it would be
+			 * useful to support the former case since it's unusual to have
+			 * missing partitions.  On the other hand, it would be useful to
+			 * support the latter case, but in that case, there is a high
+			 * probability that a partition on one side will match multiple
+			 * partitions on the other side, which is the scenario the current
+			 * implementation of partitioned join can't handle.
+			 */
+			return NULL;
+
+		case PARTITION_STRATEGY_LIST:
+			return merge_list_bounds(partsupfunc,
+									 partcollation,
+									 outer_rel,
+									 inner_rel,
+									 jointype,
+									 outer_parts,
+									 inner_parts);
+
+		case PARTITION_STRATEGY_RANGE:
+			return merge_range_bounds(partnatts,
+									  partsupfunc,
+									  partcollation,
+									  outer_rel,
+									  inner_rel,
+									  jointype,
+									  outer_parts,
+									  inner_parts);
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", (int) strategy);
+			return NULL;				/* keep compiler quiet */
+	}
+}
+
+/*
+ * merge_list_bounds
+ *		Create the partition bounds for a join relation between list
+ *		partitioned tables, if possible
+ */
+static PartitionBoundInfo
+merge_list_bounds(FmgrInfo *partsupfunc, Oid *partcollation,
+				  RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+				  JoinType jointype,
+				  List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			outer_pos;
+	int			inner_pos;
+	int			next_index = 0;
+	int			null_index = -1;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+	/* List partitioning doesn't require kinds. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * If the default partitions (if any) have been proven empty, deem them
+	 * non-existent.
+	 */
+	if (outer_has_default && is_dummy_partition(outer_rel, outer_default))
+		outer_has_default = false;
+	if (inner_has_default && is_dummy_partition(inner_rel, inner_default))
+		inner_has_default = false;
+
+	/*
+	 * Merge partitions from both sides.  In each iteration we compare a pair
+	 * of list values, one from each side, and decide whether the corresponding
+	 * partitions match or not.  If the two values match exactly, move to the
+	 * next pair of list values, otherwise move to the next list value on the
+	 * side with a smaller list value.
+	 */
+	outer_pos = inner_pos = 0;
+	while (outer_pos < outer_bi->ndatums || inner_pos < inner_bi->ndatums)
+	{
+		int			outer_index = -1;
+		int			inner_index = -1;
+		Datum	   *outer_datums;
+		Datum	   *inner_datums;
+		int			cmpval;
+		Datum	   *merged_datum = NULL;
+		int			merged_index = -1;
+
+		if (outer_pos < outer_bi->ndatums)
+		{
+			/*
+			 * If the partition on the outer side has been proven empty, ignore
+			 * it and move to the next datum on the outer side.
+			 */
+			outer_index = outer_bi->indexes[outer_pos];
+			if (is_dummy_partition(outer_rel, outer_index))
+			{
+				outer_pos++;
+				continue;
+			}
+		}
+		if (inner_pos < inner_bi->ndatums)
+		{
+			/*
+			 * If the partition on the inner side has been proven empty, ignore
+			 * it and move to the next datum on the inner side.
+			 */
+			inner_index = inner_bi->indexes[inner_pos];
+			if (is_dummy_partition(inner_rel, inner_index))
+			{
+				inner_pos++;
+				continue;
+			}
+		}
+
+		/* Get the list values. */
+		outer_datums = outer_pos < outer_bi->ndatums ?
+			outer_bi->datums[outer_pos] : NULL;
+		inner_datums = inner_pos < inner_bi->ndatums ?
+			inner_bi->datums[inner_pos] : NULL;
+
+		/*
+		 * We run this loop till both sides finish.  This allows us to avoid
+		 * duplicating code to handle the remaining values on the side which
+		 * finishes later.  For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra value higher than any other value on the unfinished side.
+		 * That way we advance the values on the unfinished side till all of
+		 * its values are exhausted.
+		 */
+		if (outer_pos >= outer_bi->ndatums)
+			cmpval = 1;
+		else if (inner_pos >= inner_bi->ndatums)
+			cmpval = -1;
+		else
+		{
+			Assert(outer_datums != NULL && inner_datums != NULL);
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 outer_datums[0],
+													 inner_datums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			/* Two list values match exactly. */
+			Assert(outer_pos < outer_bi->ndatums);
+			Assert(inner_pos < inner_bi->ndatums);
+			Assert(outer_index >= 0);
+			Assert(inner_index >= 0);
+
+			/*
+			 * Try merging both paritions.  If successful, add the list value
+			 * and index of the merged partition below.
+			 */
+			merged_index = merge_matching_partitions(&outer_map, &inner_map,
+													 outer_index, inner_index,
+													 &next_index);
+			if (merged_index == -1)
+				goto cleanup;
+
+			merged_datum = outer_datums;
+
+			/* Move to the next pair of list values. */
+			outer_pos++;
+			inner_pos++;
+		}
+		else if (cmpval < 0)
+		{
+			/* A list value missing from the inner side. */
+			Assert(outer_pos < outer_bi->ndatums);
+
+			/*
+			 * If the inner side has the default partition, or this is an outer
+			 * join, try to assign a merged partition to the outer partition
+			 * (see process_outer_partition()).  Otherwise, the outer partition
+			 * will not contribute to the result.
+			 */
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				/* Get the outer partition. */
+				outer_index = outer_bi->indexes[outer_pos];
+				Assert(outer_index >= 0);
+				merged_index = process_outer_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   outer_index,
+													   inner_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+				merged_datum = outer_datums;
+			}
+
+			/* Move to the next list value on the outer side. */
+			outer_pos++;
+		}
+		else
+		{
+			/* A list value missing from the outer side. */
+			Assert(cmpval > 0);
+			Assert(inner_pos < inner_bi->ndatums);
+
+			/*
+			 * If the outer side has the default partition, or this is a FULL
+			 * join, try to assign a merged partition to the inner partition
+			 * (see process_inner_partition()).  Otherwise, the inner partition
+			 * will not contribute to the result.
+			 */
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				/* Get the inner partition. */
+				inner_index = inner_bi->indexes[inner_pos];
+				Assert(inner_index >= 0);
+				merged_index = process_inner_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   inner_index,
+													   outer_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+				merged_datum = inner_datums;
+			}
+
+			/* Move to the next list value on the inner side. */
+			inner_pos++;
+		}
+
+		/*
+		 * If we assigned a merged partition, add the list value and index of
+		 * the merged partition if appropriate.
+		 */
+		if (merged_index >= 0 && merged_index != default_index)
+		{
+			merged_datums = lappend(merged_datums, merged_datum);
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+		}
+	}
+
+	/*
+	 * If the NULL partitions (if any) have been proven empty, deem them
+	 * non-existent.
+	 */
+	if (outer_has_null &&
+		is_dummy_partition(outer_rel, outer_bi->null_index))
+		outer_has_null = false;
+	if (inner_has_null &&
+		is_dummy_partition(inner_rel, inner_bi->null_index))
+		inner_has_null = false;
+
+	/* Merge the NULL partitions if any. */
+	if (outer_has_null || inner_has_null)
+		merge_null_partitions(&outer_map, &inner_map,
+							  outer_has_null, inner_has_null,
+							  outer_bi->null_index, inner_bi->null_index,
+							  jointype, &next_index, &null_index);
+	else
+		Assert(null_index == -1);
+
+	/* Merge the default partitions if any. */
+	if (outer_has_default || inner_has_default)
+		merge_default_partitions(&outer_map, &inner_map,
+								 outer_has_default, inner_has_default,
+								 outer_default, inner_default,
+								 jointype, &next_index, &default_index);
+	else
+		Assert(default_index == -1);
+
+	/* If we have merged partitions, create the partition bounds. */
+	if (next_index > 0)
+	{
+		/* Fix the merged_indexes list if necessary. */
+		if (outer_map.did_remapping || inner_map.did_remapping)
+		{
+			Assert(jointype == JOIN_FULL);
+			fix_merged_indexes(&outer_map, &inner_map,
+							   next_index, merged_indexes);
+		}
+
+		/* Use maps to match partitions from inputs. */
+		generate_matching_part_pairs(outer_rel, inner_rel,
+									 &outer_map, &inner_map,
+									 next_index,
+									 outer_parts, inner_parts);
+		Assert(*outer_parts != NIL);
+		Assert(*inner_parts != NIL);
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) <= next_index);
+
+		/* Make a PartitionBoundInfo struct to return. */
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  NIL,
+													  merged_indexes,
+													  null_index,
+													  default_index);
+		Assert(merged_bounds);
+	}
+
+cleanup:
+	/* Free local memory before returning. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+
+	return merged_bounds;
+}
+
+/*
+ * merge_range_bounds
+ *		Create the partition bounds for a join relation between range
+ *		partitioned tables, if possible
+ */
+static PartitionBoundInfo
+merge_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+				   Oid *partcollations,
+				   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+				   JoinType jointype,
+				   List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			outer_index;
+	int			inner_index;
+	int			outer_lb_pos;
+	int			inner_lb_pos;
+	PartitionRangeBound outer_lb;
+	PartitionRangeBound outer_ub;
+	PartitionRangeBound inner_lb;
+	PartitionRangeBound inner_ub;
+	int			next_index = 0;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_kinds = NIL;
+	List	   *merged_indexes = NIL;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * If the default partitions (if any) have been proven empty, deem them
+	 * non-existent.
+	 */
+	if (outer_has_default && is_dummy_partition(outer_rel, outer_default))
+		outer_has_default = false;
+	if (inner_has_default && is_dummy_partition(inner_rel, inner_default))
+		inner_has_default = false;
+
+	/*
+	 * Merge partitions from both sides.  In each iteration we compare a pair
+	 * of ranges, one from each side, and decide whether the corresponding
+	 * partitions match or not.  If the two ranges overlap, move to the next
+	 * pair of ranges, otherwise move to the next range on the side with a
+	 * lower range.  outer_lb_pos/inner_lb_pos keep track of the positions of
+	 * lower bounds in the datums arrays in the outer/inner PartitionBoundInfos
+	 * respectively.
+	 */
+	outer_lb_pos = inner_lb_pos = 0;
+	outer_index = get_range_partition(outer_rel, outer_bi, &outer_lb_pos,
+									  &outer_lb, &outer_ub);
+	inner_index = get_range_partition(inner_rel, inner_bi, &inner_lb_pos,
+									  &inner_lb, &inner_ub);
+	while (outer_index >= 0 || inner_index >= 0)
+	{
+		bool		overlap;
+		int			ub_cmpval;
+		int			lb_cmpval;
+		PartitionRangeBound merged_lb = {-1, NULL, NULL, true};
+		PartitionRangeBound merged_ub = {-1, NULL, NULL, false};
+		int			merged_index = -1;
+
+		/*
+		 * We run this loop till both sides finish.  This allows us to avoid
+		 * duplicating code to handle the remaining ranges on the side which
+		 * finishes later.  For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra range higher than any other range on the unfinished side.
+		 * That way we advance the ranges on the unfinished side till all of
+		 * its ranges are exhausted.
+		 */
+		if (outer_index == -1)
+		{
+			overlap = false;
+			lb_cmpval = 1;
+			ub_cmpval = 1;
+		}
+		else if (inner_index == -1)
+		{
+			overlap = false;
+			lb_cmpval = -1;
+			ub_cmpval = -1;
+		}
+		else
+			overlap = compare_range_partitions(partnatts, partsupfuncs,
+											   partcollations,
+											   &outer_lb, &outer_ub,
+											   &inner_lb, &inner_ub,
+											   &lb_cmpval, &ub_cmpval);
+
+		if (overlap)
+		{
+			/* Two ranges overlap; form a join pair. */
+
+			PartitionRangeBound save_outer_ub;
+			PartitionRangeBound save_inner_ub;
+
+			/* Both partitions should not have been merged yet. */
+			Assert(outer_index >= 0);
+			Assert(outer_map.merged_indexes[outer_index] == -1 &&
+				   outer_map.merged[outer_index] == false);
+			Assert(inner_index >= 0);
+			Assert(inner_map.merged_indexes[inner_index] == -1 &&
+				   inner_map.merged[inner_index] == false);
+
+			/*
+			 * Get the index of the merged partition.  Both partitions aren't
+			 * merged yet, so the partitions should be merged successfully.
+			 */
+			merged_index = merge_matching_partitions(&outer_map, &inner_map,
+													 outer_index, inner_index,
+													 &next_index);
+			Assert(merged_index >= 0);
+
+			/* Get the range of the merged partition. */
+			get_merged_range_bounds(partnatts, partsupfuncs,
+									partcollations, jointype,
+									&outer_lb, &outer_ub,
+									&inner_lb, &inner_ub,
+									lb_cmpval, ub_cmpval,
+									&merged_lb, &merged_ub);
+
+			/* Save the upper bounds of both partitions for use below. */
+			save_outer_ub = outer_ub;
+			save_inner_ub = inner_ub;
+
+			/* Move to the next pair of ranges. */
+			outer_index = get_range_partition(outer_rel, outer_bi, &outer_lb_pos,
+											  &outer_lb, &outer_ub);
+			inner_index = get_range_partition(inner_rel, inner_bi, &inner_lb_pos,
+											  &inner_lb, &inner_ub);
+
+			/*
+			 * If the range of a partition on one side overlaps the range of
+			 * the next partition on the other side, that will cause the
+			 * partition on one side to match at least two partitions on the
+			 * other side, which is the case that we currently don't support
+			 * partitioned join for; give up.
+			 */
+			if (ub_cmpval > 0 && inner_index >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &save_outer_ub, &inner_lb) > 0)
+				goto cleanup;
+			if (ub_cmpval < 0 && outer_index >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &outer_lb, &save_inner_ub) < 0)
+				goto cleanup;
+
+			/*
+			 * A row from a non-overlapping portion (if any) of a partition
+			 * on one side might find its join partner in the default
+			 * partition (if any) on the other side, causing the same
+			 * situation as above; give up in that case.
+			 */
+			if ((outer_has_default && (lb_cmpval > 0 || ub_cmpval < 0)) ||
+				(inner_has_default && (lb_cmpval < 0 || ub_cmpval > 0)))
+				goto cleanup;
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* A non-overlapping outer range. */
+
+			/* The outer partition should not have been merged yet. */
+			Assert(outer_index >= 0);
+			Assert(outer_map.merged_indexes[outer_index] == -1 &&
+				   outer_map.merged[outer_index] == false);
+
+			/*
+			 * If the inner side has the default partition, or this is an outer
+			 * join, try to assign a merged partition to the outer partition
+			 * (see process_outer_partition()).  Otherwise, the outer partition
+			 * will not contribute to the result.
+			 */
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				merged_index = process_outer_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   outer_index,
+													   inner_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+				merged_lb = outer_lb;
+				merged_ub = outer_ub;
+			}
+
+			/* Move to the next range on the outer side. */
+			outer_index = get_range_partition(outer_rel, outer_bi, &outer_lb_pos,
+											  &outer_lb, &outer_ub);
+		}
+		else
+		{
+			/* A non-overlapping inner range. */
+			Assert(ub_cmpval > 0);
+
+			/* The inner partition should not have been merged yet. */
+			Assert(inner_index >= 0);
+			Assert(inner_map.merged_indexes[inner_index] == -1 &&
+				   inner_map.merged[inner_index] == false);
+
+			/*
+			 * If the outer side has the default partition, or this is a FULL
+			 * join, try to assign a merged partition to the inner partition
+			 * (see process_inner_partition()).  Otherwise, the inner partition
+			 * will not contribute to the result.
+			 */
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				merged_index = process_inner_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   inner_index,
+													   outer_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+				merged_lb = inner_lb;
+				merged_ub = inner_ub;
+			}
+
+			/* Move to the next range on the inner side. */
+			inner_index = get_range_partition(inner_rel, inner_bi, &inner_lb_pos,
+											  &inner_lb, &inner_ub);
+		}
+
+		/*
+		 * If we assigned a merged partition, add the range bounds and index of
+		 * the merged partition if appropriate.
+		 */
+		if (merged_index >= 0 && merged_index != default_index)
+			add_merged_range_bounds(partnatts, partsupfuncs, partcollations,
+									&merged_lb, &merged_ub, merged_index,
+									&merged_datums, &merged_kinds,
+									&merged_indexes);
+	}
+
+	/* Merge the default partitions if any. */
+	if (outer_has_default || inner_has_default)
+		merge_default_partitions(&outer_map, &inner_map,
+								 outer_has_default, inner_has_default,
+								 outer_default, inner_default,
+								 jointype, &next_index, &default_index);
+	else
+		Assert(default_index == -1);
+
+	/* If we have merged partitions, create the partition bounds. */
+	if (next_index > 0)
+	{
+		/*
+		 * Unlike the case of list partitioning, we wouldn't have re-merged
+		 * partitions, so did_remapping should be left alone.
+		 */
+		Assert(!outer_map.did_remapping);
+		Assert(!inner_map.did_remapping);
+
+		/* Use maps to match partitions from inputs. */
+		generate_matching_part_pairs(outer_rel, inner_rel,
+									 &outer_map, &inner_map,
+									 next_index,
+									 outer_parts, inner_parts);
+		Assert(*outer_parts != NIL);
+		Assert(*inner_parts != NIL);
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+
+		/* Make a PartitionBoundInfo struct to return. */
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_kinds,
+													  merged_indexes,
+													  -1,
+													  default_index);
+		Assert(merged_bounds);
+	}
+
+cleanup:
+	/* Free local memory before returning. */
+	list_free(merged_datums);
+	list_free(merged_kinds);
+	list_free(merged_indexes);
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+
+	return merged_bounds;
+}
+
+/*
+ * init_partition_map
+ *		Initialize a PartitionMap struct for given relation
+ */
+static void
+init_partition_map(RelOptInfo *rel, PartitionMap *map)
+{
+	int			nparts = rel->nparts;
+	int			i;
+
+	map->nparts = nparts;
+	map->merged_indexes = (int *) palloc(sizeof(int) * nparts);
+	map->merged = (bool *) palloc(sizeof(bool) * nparts);
+	map->did_remapping = false;
+	map->old_indexes = (int *) palloc(sizeof(int) * nparts);
+	for (i = 0; i < nparts; i++)
+	{
+		map->merged_indexes[i] = map->old_indexes[i] = -1;
+		map->merged[i] = false;
+	}
+}
+
+/*
+ * free_partition_map
+ */
+static void
+free_partition_map(PartitionMap *map)
+{
+	pfree(map->merged_indexes);
+	pfree(map->merged);
+	pfree(map->old_indexes);
+}
+
+/*
+ * is_dummy_partition --- has partition been proven empty?
+ */
+static bool
+is_dummy_partition(RelOptInfo *rel, int part_index)
+{
+	RelOptInfo *part_rel;
+
+	Assert(part_index >= 0);
+	part_rel = rel->part_rels[part_index];
+	if (part_rel == NULL || IS_DUMMY_REL(part_rel))
+		return true;
+	return false;
+}
+
+/*
+ * merge_matching_partitions
+ *		Try to merge given outer/inner partitions, and return the index of a
+ *		merged partition produced from them if successful, -1 otherwise
+ *
+ * If the merged partition is newly created, *next_index is incremented.
+ */
+static int
+merge_matching_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						  int outer_index, int inner_index, int *next_index)
+{
+	int 		outer_merged_index;
+	int 		inner_merged_index;
+	bool 		outer_merged;
+	bool 		inner_merged;
+
+	Assert(outer_index >= 0 && outer_index < outer_map->nparts);
+	outer_merged_index = outer_map->merged_indexes[outer_index];
+	outer_merged = outer_map->merged[outer_index];
+	Assert(inner_index >= 0 && inner_index < inner_map->nparts);
+	inner_merged_index = inner_map->merged_indexes[inner_index];
+	inner_merged = inner_map->merged[inner_index];
+
+	/*
+	 * Handle cases where we have already assigned a merged partition to each
+	 * of the given partitions.
+	 */
+	if (outer_merged_index >= 0 && inner_merged_index >= 0)
+	{
+		/*
+		 * If the mereged partitions are the same, no need to do anything;
+		 * return the index of the merged partitions.  Otherwise, if each of
+		 * the given partitions has been merged with a dummy partition on the
+		 * other side, re-map them to either of the two merged partitions.
+		 * Otherwise, they can't be merged, so return -1.
+		 */
+		if (outer_merged_index == inner_merged_index)
+		{
+			Assert(outer_merged);
+			Assert(inner_merged);
+			return outer_merged_index;
+		}
+		if (!outer_merged && !inner_merged)
+		{
+			/*
+			 * This can only happen for a list-partitioning case.  We re-map
+			 * them to the merged partition with the smaller of the two merged
+			 * indexes to preserve the property that the canonical order of
+			 * list partitions is determined by the indexes assigned to the
+			 * smallest list value of each partition.
+			 */
+			if (outer_merged_index < inner_merged_index)
+			{
+				outer_map->merged[outer_index] = true;
+				inner_map->merged_indexes[inner_index] = outer_merged_index;
+				inner_map->merged[inner_index] = true;
+				inner_map->did_remapping = true;
+				inner_map->old_indexes[inner_index] = inner_merged_index;
+				return outer_merged_index;
+			}
+			else
+			{
+				inner_map->merged[inner_index] = true;
+				outer_map->merged_indexes[outer_index] = inner_merged_index;
+				outer_map->merged[outer_index] = true;
+				outer_map->did_remapping = true;
+				outer_map->old_indexes[outer_index] = outer_merged_index;
+				return inner_merged_index;
+			}
+		}
+		return -1;
+	}
+
+	/* At least one of the given partitions should not have yet been merged. */
+	Assert(outer_merged_index == -1 || inner_merged_index == -1);
+
+	/*
+	 * If neither of them has been merged, merge them.  Otherwise, if one has
+	 * been merged with a dummy relation on the other side (and the other
+	 * hasn't yet been merged with anything), re-merge them.  Otherwise, they
+	 * can't be merged, so return -1.
+	 */
+	if (outer_merged_index == -1 && inner_merged_index == -1)
+	{
+		int 		merged_index = *next_index;
+
+		Assert(!outer_merged);
+		Assert(!inner_merged);
+		outer_map->merged_indexes[outer_index] = merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged_indexes[inner_index] = merged_index;
+		inner_map->merged[inner_index] = true;
+		*next_index = *next_index + 1;
+		return merged_index;
+	}
+	if (outer_merged_index >= 0 && !outer_map->merged[outer_index])
+	{
+		Assert(inner_merged_index == -1);
+		Assert(!inner_merged);
+		inner_map->merged_indexes[inner_index] = outer_merged_index;
+		inner_map->merged[inner_index] = true;
+		outer_map->merged[outer_index] = true;
+		return outer_merged_index;
+	}
+	if (inner_merged_index >= 0 && !inner_map->merged[inner_index])
+	{
+		Assert(outer_merged_index == -1);
+		Assert(!outer_merged);
+		outer_map->merged_indexes[outer_index] = inner_merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged[inner_index] = true;
+		return inner_merged_index;
+	}
+	return -1;
+}
+
+/*
+ * process_outer_partition
+ *		Try to assign given outer partition a merged partition, and return the
+ *		index of the merged partition if successful, -1 otherwise
+ *
+ * If the partition is newly created, *next_index is incremented.  Also, if it
+ * is the default partition of the join relation, *default_partition is set to
+ * the index if not already done.
+ */
+static int
+process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index)
+{
+	int 		merged_index = -1;
+
+	Assert(outer_index >= 0);
+
+	/*
+	 * If the inner side has the default partition, a row from the outer
+	 * partition might find its join partner in the default partition; try
+	 * merging the outer partition with the default partition.  Otherwise, this
+	 * should be an outer join, in which case the outer partition has to be
+	 * scanned all the way anyway; merge the outer partition with a dummy
+	 * partition on the other side.
+	 */
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0);
+
+		/*
+		 * If the outer side has the default partition as well, the default
+		 * partition on the inner side will have two matching partitions on the
+		 * other side: the outer partition and the default partition on the
+		 * outer side.  Partitionwise join doesn't handle this scenario yet.
+		 */
+		if (outer_has_default)
+			return -1;
+
+		merged_index = merge_matching_partitions(outer_map, inner_map,
+												 outer_index, inner_default,
+												 next_index);
+		if (merged_index == -1)
+			return -1;
+
+		/*
+		 * If this is a FULL join, the default partition on the inner side
+		 * has to be scanned all the way anyway, so the resulting partition
+		 * will contain all key values from the default partition, which any
+		 * other partition of the join relation will not contain.  Thus the
+		 * resutling partition will act as the default partition of the join
+		 * relation; record the index in *default_index if not already done.
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (*default_index == -1)
+				*default_index = merged_index;
+			else
+				Assert(*default_index == merged_index);
+		}
+	}
+	else
+	{
+		Assert(IS_OUTER_JOIN(jointype));
+		Assert(jointype != JOIN_RIGHT);
+
+		/* If we have already assigned a partition, no need to do anything. */
+		merged_index = outer_map->merged_indexes[outer_index];
+		if (merged_index == -1)
+			merged_index = merge_partition_with_dummy(outer_map, outer_index,
+													  next_index);
+	}
+	return merged_index;
+}
+
+/*
+ * process_inner_partition
+ *		Try to assign given inner partition a merged partition, and return the
+ *		index of the merged partition if successful, -1 otherwise
+ *
+ * If the partition is newly created, *next_index is incremented.  Also, if it
+ * is the default partition of the join relation, *default_partition is set to
+ * the index if not already done.
+ */
+static int
+process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index)
+{
+	int 		merged_index = -1;
+
+	Assert(inner_index >= 0);
+
+	/*
+	 * If the outer side has the default partition, a row from the inner
+	 * partition might find its join partner in the default partition; try
+	 * merging the inner partition with the default partition.  Otherwise, this
+	 * should be a FULL join, in which case the inner partition has to be
+	 * scanned all the way anyway; merge the inner partition with a dummy
+	 * partition on the other side.
+	 */
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0);
+
+		/*
+		 * If the inner side has the default partition as well, the default
+		 * partition on the outer side will have two matching partitions on the
+		 * other side: the inner partition and the default partition on the
+		 * inner side.  Partitionwise join doesn't handle this scenario yet.
+		 */
+		if (inner_has_default)
+			return -1;
+
+		merged_index = merge_matching_partitions(outer_map, inner_map,
+												 outer_default, inner_index,
+												 next_index);
+		if (merged_index == -1)
+			return -1;
+
+		/*
+		 * If this is an outer join, the default partition on the outer side
+		 * has to be scanned all the way anyway, so the resulting partition
+		 * will contain all key values from the default partition, which any
+		 * other partition of the join relation will not contain.  Thus the
+		 * resutling partition will act as the default partition of the join
+		 * relation; record the index in *default_index if not already done.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			if (*default_index == -1)
+				*default_index = merged_index;
+			else
+				Assert(*default_index == merged_index);
+		}
+	}
+	else
+	{
+		Assert(jointype == JOIN_FULL);
+
+		/* If we have already assigned a partition, no need to do anything. */
+		merged_index = inner_map->merged_indexes[inner_index];
+		if (merged_index == -1)
+			merged_index = merge_partition_with_dummy(inner_map, inner_index,
+													  next_index);
+	}
+	return merged_index;
+}
+
+/*
+ * merge_null_partitions
+ *		Merge the NULL partitions from a join's outer and inner sides.
+ *
+ * If the merged partition produced from them is the NULL partition of the join
+ * relation, *null_index is set to the index of the merged partition.
+ *
+ * Note: We assume here that the join clause for a partitioned join is strict
+ * because have_partkey_equi_join() requires that the corresponding operator
+ * be mergejoinable, and we currently assume that mergejoinable operators are
+ * strict (see MJEvalOuterValues()/MJEvalInnerValues()).
+ */
+static void
+merge_null_partitions(PartitionMap *outer_map,
+					  PartitionMap *inner_map,
+					  bool outer_has_null,
+					  bool inner_has_null,
+					  int outer_null,
+					  int inner_null,
+					  JoinType jointype,
+					  int *next_index,
+					  int *null_index)
+{
+	bool 		consider_outer_null = false;
+	bool 		consider_inner_null = false;
+
+	Assert(outer_has_null || inner_has_null);
+	Assert(*null_index == -1);
+
+	/*
+	 * Check whether the NULL partitions have already been merged and if so,
+	 * set the consider_outer_null/consider_inner_null flags.
+	 */
+	if (outer_has_null)
+	{
+		Assert(outer_null >= 0 && outer_null < outer_map->nparts);
+		if (outer_map->merged_indexes[outer_null] == -1)
+		 	consider_outer_null = true;
+	}
+	if (inner_has_null)
+	{
+		Assert(inner_null >= 0 && inner_null < inner_map->nparts);
+		if (inner_map->merged_indexes[inner_null] == -1)
+		 	consider_inner_null = true;
+	}
+
+	/* If both flags are set false, we don't need to do anything. */
+	if (!consider_outer_null && !consider_inner_null)
+		return;
+
+	if (consider_outer_null && !consider_inner_null)
+	{
+		Assert(outer_has_null);
+
+		/*
+		 * If this is an outer join, the NULL partition on the outer side has
+		 * to be scanned all the way anyway; merge the NULL partition with a
+		 * dummy partition on the other side.  In that case consider_outer_null
+		 * means that the NULL partition only contains NULL values as the key
+		 * values, so the merged partition will do so; treat it as the NULL
+		 * partition of the join relation.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			*null_index = merge_partition_with_dummy(outer_map, outer_null,
+													 next_index);
+		}
+	}
+	else if (!consider_outer_null && consider_inner_null)
+	{
+		Assert(inner_has_null);
+
+		/*
+		 * If this is a FULL join, the NULL partition on the inner side has
+		 * to be scanned all the way anyway; merge the NULL partition with a
+		 * dummy partition on the other side.  In that case consider_inner_null
+		 * means that the NULL partition only contains NULL values as the key
+		 * values, so the merged partition will do so; treat it as the NULL
+		 * partition of the join relation.
+		 */
+		if (jointype == JOIN_FULL)
+			*null_index = merge_partition_with_dummy(inner_map, inner_null,
+													 next_index);
+	}
+	else
+	{
+		Assert(consider_outer_null && consider_inner_null);
+		Assert(outer_has_null);
+		Assert(inner_has_null);
+
+		/*
+		 * If this is an outer join, the NULL partition on the outer side (and
+		 * that on the inner side if this is a FULL join) have to be scanned
+		 * all the way anyway, so merge them.  Note that each of the NULL
+		 * partitions isn't merged yet, so they should be merged successfully.
+		 * Like the above, each of the NULL partitions only contains NULL
+		 * values as the key values, so the merged partition will do so; treat
+		 * it as the NULL partition of the join relation.
+		 *
+		 * Note: if this an INNER/SEMI join, the join clause will never be
+		 * satisfied by two NULL values (see comments above), so both the NULL
+		 * partitions can be eliminated.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			*null_index = merge_matching_partitions(outer_map, inner_map,
+													outer_null, inner_null,
+													next_index);
+			Assert(*null_index >= 0);
+		}
+	}
+}
+
+/*
+ * merge_default_partitions
+ *		Merge the default partitions from a join's outer and inner sides.
+ *
+ * If the merged partition produced from them is the default partition of the
+ * join relation, *default_index is set to the index of the merged partition.
+ */
+static void
+merge_default_partitions(PartitionMap *outer_map,
+						 PartitionMap *inner_map,
+						 bool outer_has_default,
+						 bool inner_has_default,
+						 int outer_default,
+						 int inner_default,
+						 JoinType jointype,
+						 int *next_index,
+						 int *default_index)
+{
+	int 		outer_merged_index = -1;
+	int 		inner_merged_index = -1;
+
+	Assert(outer_has_default || inner_has_default);
+
+	/* Get the merged partition indexes for the default partitions. */
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0 && outer_default < outer_map->nparts);
+		outer_merged_index = outer_map->merged_indexes[outer_default];
+	}
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0 && inner_default < inner_map->nparts);
+		inner_merged_index = inner_map->merged_indexes[inner_default];
+	}
+
+	if (outer_has_default && !inner_has_default)
+	{
+		/*
+		 * If this is an outer join, the default partition on the outer side
+		 * has to be scanned all the way anyway; if we have not yet assigned a
+		 * partition, merge the default partition with a dummy partition on the
+		 * other side.  The merged partition will act as the default partition
+		 * of the join relation (see comments in process_inner_partition()).
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			if (outer_merged_index == -1)
+			{
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(outer_map,
+															outer_default,
+															next_index);
+			}
+			else
+				Assert(*default_index == outer_merged_index);
+		}
+		else
+			Assert(*default_index == -1);
+	}
+	else if (!outer_has_default && inner_has_default)
+	{
+		/*
+		 * If this is a FULL join, the default partition on the inner side
+		 * has to be scanned all the way anyway; if we have not yet assigned a
+		 * partition, merge the default partition with a dummy partition on the
+		 * other side.  The merged partition will act as the default partition
+		 * of the join relation (see comments in process_outer_partition()).
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (inner_merged_index == -1)
+			{
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(inner_map,
+															inner_default,
+															next_index);
+			}
+			else
+				Assert(*default_index == inner_merged_index);
+		}
+		else
+			Assert(*default_index == -1);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		/*
+		 * The default partitions have to be joined with each other, so merge
+		 * them.  Note that each of the default partitions isn't merged yet
+		 * (see, process_outer_partition()/process_innerer_partition()), so
+		 * they should be merged successfully.  The merged partition will act
+		 * as the default partition of the join relation.
+		 */
+		Assert(outer_merged_index == -1);
+		Assert(inner_merged_index == -1);
+		Assert(*default_index == -1);
+		*default_index = merge_matching_partitions(outer_map,
+												   inner_map,
+												   outer_default,
+												   inner_default,
+												   next_index);
+		Assert(*default_index >= 0);
+	}
+}
+
+/*
+ * merge_partition_with_dummy
+ *		Assign given partition a new partition of a join relation
+ *
+ * Note: The caller assumes that the given partition doesn't have a non-dummy
+ * matching partition on the other side, but if the given partition finds the
+ * matchig partition later, we will adjust the assignment.
+ */
+static int
+merge_partition_with_dummy(PartitionMap *map, int index, int *next_index)
+{
+	int 		merged_index = *next_index;
+
+	Assert(index >= 0 && index < map->nparts);
+	Assert(map->merged_indexes[index] == -1);
+	Assert(!map->merged[index]);
+	map->merged_indexes[index] = merged_index;
+	/* Leave the merged flag alone! */
+	*next_index = *next_index + 1;
+	return merged_index;
+}
+
+/*
+ * fix_merged_indexes
+ *		Adjust merged indexes of re-merged partitions
+ */
+static void
+fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes)
+{
+	int		   *new_indexes;
+	int			merged_index;
+	int			i;
+	ListCell   *lc;
+
+	Assert(nmerged > 0);
+
+	new_indexes = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		new_indexes[i] = -1;
+
+	/* Build the mapping of old merged indexes to new merged indexes. */
+	if (outer_map->did_remapping)
+	{
+		for (i = 0; i < outer_map->nparts; i++)
+		{
+			merged_index = outer_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = outer_map->merged_indexes[i];
+		}
+	}
+	if (inner_map->did_remapping)
+	{
+		for (i = 0; i < inner_map->nparts; i++)
+		{
+			merged_index = inner_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = inner_map->merged_indexes[i];
+		}
+	}
+
+	/* Fix the merged_indexes list using the mapping. */
+	foreach(lc, merged_indexes)
+	{
+		merged_index = lfirst_int(lc);
+		Assert(merged_index >= 0);
+		if (new_indexes[merged_index] >= 0)
+			lfirst_int(lc) = new_indexes[merged_index];
+	}
+
+	pfree(new_indexes);
+}
+
+/*
+ * generate_matching_part_pairs
+ *		Generate a pair of lists of partitions that produce merged partitions
+ *
+ * The lists of partitions are built in the order of merged partition indexes,
+ * and returned in *outer_parts and *inner_parts.
+ */
+static void
+generate_matching_part_pairs(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 PartitionMap *outer_map, PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_parts, List **inner_parts)
+{
+	int			outer_nparts = outer_map->nparts;
+	int			inner_nparts = inner_map->nparts;
+	int		   *outer_indexes;
+	int		   *inner_indexes;
+	int			max_nparts;
+	int 		i;
+
+	Assert(nmerged > 0);
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	outer_indexes = (int *) palloc(sizeof(int) * nmerged);
+	inner_indexes = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		outer_indexes[i] = inner_indexes[i] = -1;
+
+	/* Set pairs of matching partitions. */
+	Assert(outer_nparts == outer_rel->nparts);
+	Assert(inner_nparts == inner_rel->nparts);
+	max_nparts = Max(outer_nparts, inner_nparts);
+	for (i = 0; i < max_nparts; i++)
+	{
+		if (i < outer_nparts)
+		{
+			int 		merged_index = outer_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				outer_indexes[merged_index] = i;
+			}
+		}
+		if (i < inner_nparts)
+		{
+			int 		merged_index = inner_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				inner_indexes[merged_index] = i;
+			}
+		}
+	}
+
+	/* Build the list pairs. */
+	for (i = 0; i < nmerged; i++)
+	{
+		int			outer_index = outer_indexes[i];
+		int			inner_index = inner_indexes[i];
+
+		/*
+		 * If both partitions are dummy, it means the merged partition that had
+		 * been assigned to the outer/inner partition was removed when
+		 * re-merging the outer/inner partition in merge_matching_partitions();
+		 * ignore the merged partition.
+		 */
+		if (outer_index == -1 && inner_index == -1)
+			continue;
+
+		*outer_parts = lappend(*outer_parts, outer_index >= 0 ?
+							   outer_rel->part_rels[outer_index] : NULL);
+		*inner_parts = lappend(*inner_parts, inner_index >= 0 ?
+							   inner_rel->part_rels[inner_index] : NULL);
+	}
+
+	pfree(outer_indexes);
+	pfree(inner_indexes);
+}
+
+/*
+ * build_merged_partition_bounds
+ *		Create a PartitionBoundInfo struct from merged partition bounds
+ */
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_kinds, List *merged_indexes,
+							  int null_index, int default_index)
+{
+	PartitionBoundInfo merged_bounds;
+	int			ndatums = list_length(merged_datums);
+	int			pos;
+	ListCell   *lc;
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = ndatums;
+
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) * ndatums);
+	pos = 0;
+	foreach(lc, merged_datums)
+		merged_bounds->datums[pos++] = (Datum *) lfirst(lc);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_kinds) == ndatums);
+		merged_bounds->kind = (PartitionRangeDatumKind **)
+			palloc(sizeof(PartitionRangeDatumKind *) * ndatums);
+		pos = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[pos++] = (PartitionRangeDatumKind *) lfirst(lc);
+
+		/* There are ndatums+1 indexes in the case of range partitioning. */
+		merged_indexes = lappend_int(merged_indexes, -1);
+		ndatums++;
+	}
+	else
+	{
+		Assert(strategy == PARTITION_STRATEGY_LIST);
+		Assert(merged_kinds == NIL);
+		merged_bounds->kind = NULL;
+	}
+
+	Assert(list_length(merged_indexes) == ndatums);
+	merged_bounds->indexes = (int *) palloc(sizeof(int) * ndatums);
+	pos = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[pos++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * get_range_partition
+ *		Get the next non-dummy partition of a range-partitioned relation,
+ *		returning the index of that partition
+ *
+ * *lb and *ub are set to the lower and upper bounds of that partition
+ * respectively, and *lb_pos is advanced to the next lower bound, if any.
+ */
+static int
+get_range_partition(RelOptInfo *rel,
+					PartitionBoundInfo bi,
+					int *lb_pos,
+					PartitionRangeBound *lb,
+					PartitionRangeBound *ub)
+{
+	int			part_index;
+
+	Assert(bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	do {
+		part_index = get_range_partition_internal(bi, lb_pos, lb, ub);
+		if (part_index == -1)
+			return -1;
+	} while (is_dummy_partition(rel, part_index));
+
+	return part_index;
+}
+
+static int
+get_range_partition_internal(PartitionBoundInfo bi,
+							 int *lb_pos,
+							 PartitionRangeBound *lb,
+							 PartitionRangeBound *ub)
+{
+	/* Return the index as -1 if we've exhausted all lower bounds. */
+	if (*lb_pos >= bi->ndatums)
+		return -1;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(*lb_pos + 1 < bi->ndatums);
+
+	/* Set the lower bound. */
+	lb->index = bi->indexes[*lb_pos];
+	lb->datums = bi->datums[*lb_pos];
+	lb->kind = bi->kind[*lb_pos];
+	lb->lower = true;
+	/* Set the upper bound. */
+	ub->index = bi->indexes[*lb_pos + 1];
+	ub->datums = bi->datums[*lb_pos + 1];
+	ub->kind = bi->kind[*lb_pos + 1];
+	ub->lower = false;
+
+	/* The index assigned to an upper bound should be valid. */
+	Assert(ub->index >= 0);
+
+	/*
+	 * Advance the position to the next lower bound.  If there are no bounds
+	 * left beyond the upper bound, we have reached the last lower bound.
+	 */
+	if (*lb_pos + 2 >= bi->ndatums)
+		*lb_pos = bi->ndatums;
+	else
+	{
+		/*
+		 * If the index assigned to the bound next to the upper bound isn't
+		 * valid, that is the next lower bound; else, the upper bound is also
+		 * the lower bound of the next range partition.
+		 */
+		if (bi->indexes[*lb_pos + 2] < 0)
+			*lb_pos = *lb_pos + 2;
+		else
+			*lb_pos = *lb_pos + 1;
+	}
+
+	return ub->index;
+}
+
+/*
+ * compare_range_partitions
+ *		Compare the bounds of two range partitions, and return true if the
+ *		two partitions overlap, false otherwise
+ *
+ * *lb_cmpval is set to -1, 0, or 1 if the outer partition's lower bound is
+ * lower than, equal to, or higher than the inner partition's lower bound
+ * respectively.  Likewise, *ub_cmpval is set to -1, 0, or 1 if the outer
+ * partition's upper bound is lower than, equal to, or higher than the inner
+ * partition's upper bound respectively.
+ */
+static bool
+compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+						 Oid *partcollations,
+						 PartitionRangeBound *outer_lb,
+						 PartitionRangeBound *outer_ub,
+						 PartitionRangeBound *inner_lb,
+						 PartitionRangeBound *inner_ub,
+						 int *lb_cmpval, int *ub_cmpval)
+{
+	/*
+	 * Check if the outer partition's upper bound is lower than the inner
+	 * partition's lower bound; if so the partitions aren't overlapping.
+	 */
+	if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+							 outer_ub, inner_lb) < 0)
+	{
+		*lb_cmpval = -1;
+		*ub_cmpval = -1;
+		return false;
+	}
+
+	/*
+	 * Check if the outer partition's lower bound is higher than the inner
+	 * partition's upper bound; if so the partitions aren't overlapping.
+	 */
+	if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+							 outer_lb, inner_ub) > 0)
+	{
+		*lb_cmpval = 1;
+		*ub_cmpval = 1;
+		return false;
+	}
+
+	/* All other cases indicate overlapping partitions. */
+	*lb_cmpval = compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									  outer_lb, inner_lb);
+	*ub_cmpval = compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									  outer_ub, inner_ub);
+	return true;
+}
+
+/*
+ * get_merged_range_bounds
+ *		Given the bounds of range partitions to be joined, determine the bounds
+ *		of a merged partition produced from the range partitions
+ *
+ * *merged_lb and *merged_ub are set to the lower and upper bounds of the
+ * merged partition.
+ */
+static void
+get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations, JoinType jointype,
+						PartitionRangeBound *outer_lb,
+						PartitionRangeBound *outer_ub,
+						PartitionRangeBound *inner_lb,
+						PartitionRangeBound *inner_ub,
+						int	lb_cmpval, int ub_cmpval,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub)
+{
+	Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								outer_lb, inner_lb) == lb_cmpval);
+	Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								outer_ub, inner_ub) == ub_cmpval);
+
+	switch (jointype)
+	{
+		case JOIN_INNER:
+		case JOIN_SEMI:
+
+			/*
+			 * An INNER/SEMI join will have the rows that fit both sides, so
+			 * the lower bound of the merged partition will be the higher of
+			 * the two lower bounds, and the upper bound of the merged
+			 * partition will be the lower of the two upper bounds.
+			 */
+			*merged_lb = (lb_cmpval > 0) ? *outer_lb : *inner_lb;
+			*merged_ub = (ub_cmpval < 0) ? *outer_ub : *inner_ub;
+			break;
+
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+
+			/*
+			 * A LEFT/ANTI join will have all the rows from the outer side, so
+			 * the bounds of the merged partition will be the same as the outer
+			 * bounds.
+			 */
+			*merged_lb = *outer_lb;
+			*merged_ub = *outer_ub;
+			break;
+
+		case JOIN_FULL:
+
+			/*
+			 * A FULL join will have all the rows from both sides, so the lower
+			 * bound of the merged partition will be the lower of the two lower
+			 * bounds, and the upper bound of the merged partition will be the
+			 * higher of the two upper bounds.
+			 */
+			*merged_lb = (lb_cmpval < 0) ? *outer_lb : *inner_lb;
+			*merged_ub = (ub_cmpval > 0) ? *outer_ub : *inner_ub;
+			break;
+
+		default:
+			elog(ERROR, "unrecognized join type: %d", (int) jointype);
+	}
+}
+
+/*
+ * add_merged_range_bounds
+ *		Add the bounds of a merged partition to the lists of range bounds
+ */
+static void
+add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub,
+						int merged_index,
+						List **merged_datums,
+						List **merged_kinds,
+						List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		/* First merged partition */
+		Assert(!*merged_kinds);
+		Assert(!*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		Assert(*merged_datums);
+		Assert(*merged_kinds);
+		Assert(*merged_indexes);
+
+		/* Get the last upper bound. */
+		prev_ub.index = llast_int(*merged_indexes);
+		prev_ub.datums = (Datum *) llast(*merged_datums);
+		prev_ub.kind = (PartitionRangeDatumKind *) llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		/*
+		 * We pass to partition_rbound_cmp() lower1 as false to prevent it
+		 * from considering the last upper bound to be smaller than the lower
+		 * bound of the merged partition when the values of the two range
+		 * bounds compare equal.
+		 */
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  merged_lb->datums, merged_lb->kind,
+									  false, &prev_ub);
+		Assert(cmpval >= 0);
+	}
+
+	/*
+	 * If the lower bound is higher than the last upper bound, add the lower
+	 * bound with the index as -1 indicating that that is a lower bound; else,
+	 * the last upper bound will be reused as the lower bound of the merged
+	 * partition, so skip this.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, merged_lb->datums);
+		*merged_kinds = lappend(*merged_kinds, merged_lb->kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	/* Add the upper bound and index of the merged partition. */
+	*merged_datums = lappend(*merged_datums, merged_ub->datums);
+	*merged_kinds = lappend(*merged_kinds, merged_ub->kind);
+	*merged_indexes = lappend_int(*merged_indexes, merged_index);
+}
+
 /*
  * partitions_are_ordered
  *		Determine whether the partitions described by 'boundinfo' are ordered,
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index 0ceb809644..622ea2bf63 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -579,8 +579,10 @@ typedef struct PartitionSchemeData *PartitionScheme;
  *		part_scheme - Partitioning scheme of the relation
  *		nparts - Number of partitions
  *		boundinfo - Partition bounds
+ *		merged - true if partition bounds are merged ones
  *		partition_qual - Partition constraint if not the root
  *		part_rels - RelOptInfos for each partition
+ *		all_partrels - Relids set of all partition relids
  *		partexprs, nullable_partexprs - Partition key expressions
  *		partitioned_child_rels - RT indexes of unpruned partitions of
  *								 this relation that are partitioned tables
@@ -718,11 +720,15 @@ typedef struct RelOptInfo
 
 	/* used for partitioned relations */
 	PartitionScheme part_scheme;	/* Partitioning scheme. */
-	int			nparts;			/* number of partitions */
+	int			nparts;			/* number of partitions; 0 = not partitioned;
+								 * -1 = not yet set */
 	struct PartitionBoundInfoData *boundinfo;	/* Partition bounds */
+	bool		merged;			/* true if partition bounds were created by
+								 * partition_bounds_merge() */
 	List	   *partition_qual; /* partition constraint */
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of partitions,
 									 * stored in the same order of bounds */
+	Relids		all_partrels;	/* Relids set of all partition relids */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 	List	   *partitioned_child_rels; /* List of RT indexes. */
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index be940b5192..dfc720720b 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -16,6 +16,7 @@
 #include "nodes/pg_list.h"
 #include "partitioning/partdefs.h"
 #include "utils/relcache.h"
+struct RelOptInfo;				/* avoid including pathnodes.h here */
 
 
 /*
@@ -87,6 +88,14 @@ extern bool partition_bounds_equal(int partnatts, int16 *parttyplen,
 								   PartitionBoundInfo b2);
 extern PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src,
 												PartitionKey key);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+												 FmgrInfo *partsupfunc,
+												 Oid *partcollation,
+												 struct RelOptInfo *outer_rel,
+												 struct RelOptInfo *inner_rel,
+												 JoinType jointype,
+												 List **outer_parts,
+												 List **inner_parts);
 extern bool partitions_are_ordered(PartitionBoundInfo boundinfo, int nparts);
 extern void check_new_partition_bound(char *relname, Relation parent,
 									  PartitionBoundSpec *spec);
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index b3fbe47bde..27588c883e 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -2055,30 +2055,2606 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
                ->  Seq Scan on prt1_n_p2 t1_2
 (10 rows)
 
--- partitionwise join can not be applied if only one of joining tables has
--- default partition
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
-ANALYZE prt2;
+--
+-- Test advanced partition-matching algorithm for partitioned join
+--
+-- Tests for range-partitioned tables
+CREATE TABLE prt1_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_adv_p1 PARTITION OF prt1_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_adv_p2 PARTITION OF prt1_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_adv_p3 PARTITION OF prt1_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_adv_a_idx ON prt1_adv (a);
+INSERT INTO prt1_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_adv;
+CREATE TABLE prt2_adv (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_adv_p1 PARTITION OF prt2_adv FOR VALUES FROM (100) TO (150);
+CREATE TABLE prt2_adv_p2 PARTITION OF prt2_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt2_adv_p3 PARTITION OF prt2_adv FOR VALUES FROM (350) TO (500);
+CREATE INDEX prt2_adv_b_idx ON prt2_adv (b);
+INSERT INTO prt2_adv_p1 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 149) i;
+INSERT INTO prt2_adv_p2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(200, 299) i;
+INSERT INTO prt2_adv_p3 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_adv;
+-- inner join
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
  Sort
    Sort Key: t1.a
-   ->  Hash Join
-         Hash Cond: (t2.b = t1.a)
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 100 | 0 | 0100
+ 125 | 0 | 0125
+ 200 | 0 | 0200
+ 225 | 0 | 0225
+ 250 | 0 | 0250
+ 275 | 0 | 0275
+ 350 | 0 | 0350
+ 375 | 0 | 0375
+(8 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 |     | 
+ 175 | 0175 |     | 
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 300 | 0300 |     | 
+ 325 | 0325 |     | 
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(12 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 150 | 0 | 0150
+ 175 | 0 | 0175
+ 300 | 0 | 0300
+ 325 | 0 | 0325
+(4 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                                QUERY PLAN                                
+--------------------------------------------------------------------------
+ Sort
+   Sort Key: prt1_adv.a, prt2_adv.b
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_adv_1.a = prt2_adv_1.b)
+               Filter: (((175) = prt1_adv_1.a) OR ((425) = prt2_adv_1.b))
+               ->  Seq Scan on prt1_adv_p1 prt1_adv_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 prt2_adv_1
+                           Filter: (a = 0)
+         ->  Hash Full Join
+               Hash Cond: (prt1_adv_2.a = prt2_adv_2.b)
+               Filter: (((175) = prt1_adv_2.a) OR ((425) = prt2_adv_2.b))
+               ->  Seq Scan on prt1_adv_p2 prt1_adv_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 prt2_adv_2
+                           Filter: (a = 0)
+         ->  Hash Full Join
+               Hash Cond: (prt2_adv_3.b = prt1_adv_3.a)
+               Filter: (((175) = prt1_adv_3.a) OR ((425) = prt2_adv_3.b))
+               ->  Seq Scan on prt2_adv_p3 prt2_adv_3
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 prt1_adv_3
+                           Filter: (b = 0)
+(27 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 175 | 0175 |     | 
+     |      | 425 | 0425
+(2 rows)
+
+-- Test cases where one side has an extra partition
+CREATE TABLE prt2_adv_extra PARTITION OF prt2_adv FOR VALUES FROM (500) TO (MAXVALUE);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(500, 599) i;
+ANALYZE prt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 100 | 0 | 0100
+ 125 | 0 | 0125
+ 200 | 0 | 0200
+ 225 | 0 | 0225
+ 250 | 0 | 0250
+ 275 | 0 | 0275
+ 350 | 0 | 0350
+ 375 | 0 | 0375
+(8 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 |     | 
+ 175 | 0175 |     | 
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 300 | 0300 |     | 
+ 325 | 0325 |     | 
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(12 rows)
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t2.a
+   ->  Hash Right Join
+         Hash Cond: (t2.a = t1.b)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2_1
-               ->  Seq Scan on prt2_p2 t2_2
-               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt1_adv_p1 t2_1
+               ->  Seq Scan on prt1_adv_p2 t2_2
+               ->  Seq Scan on prt1_adv_p3 t2_3
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt2_adv_p1 t1_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_adv_p2 t1_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_adv_p3 t1_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_adv_extra t1_4
+                           Filter: (a = 0)
+(18 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 150 | 0 | 0150
+ 175 | 0 | 0175
+ 300 | 0 | 0300
+ 325 | 0 | 0325
+(4 rows)
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt1_adv t2 WHERE t1.b = t2.a) AND t1.a = 0 ORDER BY t1.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_extra t1_4
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 t2_1
+                     ->  Seq Scan on prt1_adv_p2 t2_2
+                     ->  Seq Scan on prt1_adv_p3 t2_3
+(18 rows)
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                           QUERY PLAN                           
+----------------------------------------------------------------
+ Sort
+   Sort Key: prt1_adv.a, prt2_adv.b
+   ->  Hash Full Join
+         Hash Cond: (prt2_adv.b = prt1_adv.a)
+         Filter: (((175) = prt1_adv.a) OR ((425) = prt2_adv.b))
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 prt2_adv_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p2 prt2_adv_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p3 prt2_adv_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_extra prt2_adv_4
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 prt1_adv_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_adv_p2 prt1_adv_2
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_adv_p3 prt1_adv_3
                            Filter: (b = 0)
+(22 rows)
+
+-- 3-way join where not every pair of relations can do partitioned join
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c, t3.a, t3.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) INNER JOIN prt1_adv t3 ON (t1.b = t3.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a, t3.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t2.a
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_adv_p1 t1_1
+                           Filter: (a = 0)
+                     ->  Index Scan using prt1_adv_p1_a_idx on prt1_adv_p1 t3_1
+                           Index Cond: (a = t1_1.b)
+               ->  Index Scan using prt1_adv_p1_a_idx on prt1_adv_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.a = t1_2.b)
+               ->  Seq Scan on prt1_adv_p2 t2_2
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_2.a = t1_2.b)
+                           ->  Seq Scan on prt1_adv_p2 t3_2
+                           ->  Hash
+                                 ->  Seq Scan on prt2_adv_p2 t1_2
+                                       Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_3.a = t1_3.b)
+               ->  Seq Scan on prt1_adv_p3 t2_3
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_3.a = t1_3.b)
+                           ->  Seq Scan on prt1_adv_p3 t3_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_adv_p3 t1_3
+                                       Filter: (a = 0)
+(31 rows)
+
+SELECT t1.b, t1.c, t2.a, t2.c, t3.a, t3.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) INNER JOIN prt1_adv t3 ON (t1.b = t3.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a, t3.a;
+  b  |  c   |  a  |  c   |  a  |  c   
+-----+------+-----+------+-----+------
+ 100 | 0100 | 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275 | 275 | 0275
+ 350 | 0350 | 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375 | 375 | 0375
+(8 rows)
+
+DROP TABLE prt2_adv_extra;
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in such cases
+ALTER TABLE prt2_adv DETACH PARTITION prt2_adv_p3;
+-- Split prt2_adv_p3 into two partitions so that prt1_adv_p3 matches both
+CREATE TABLE prt2_adv_p3_1 PARTITION OF prt2_adv FOR VALUES FROM (350) TO (375);
+CREATE TABLE prt2_adv_p3_2 PARTITION OF prt2_adv FOR VALUES FROM (375) TO (500);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Seq Scan on prt2_adv_p3_1 t2_3
+               ->  Seq Scan on prt2_adv_p3_2 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+                     ->  Seq Scan on prt2_adv_p3_1 t2_3
+                     ->  Seq Scan on prt2_adv_p3_2 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Seq Scan on prt2_adv_p3_1 t2_3
+               ->  Seq Scan on prt2_adv_p3_2 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(17 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+                     ->  Seq Scan on prt2_adv_p3_1 t2_3
+                     ->  Seq Scan on prt2_adv_p3_2 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                           QUERY PLAN                           
+----------------------------------------------------------------
+ Sort
+   Sort Key: prt1_adv.a, prt2_adv.b
+   ->  Hash Full Join
+         Hash Cond: (prt2_adv.b = prt1_adv.a)
+         Filter: (((175) = prt1_adv.a) OR ((425) = prt2_adv.b))
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 prt2_adv_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p2 prt2_adv_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p3_1 prt2_adv_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p3_2 prt2_adv_4
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 prt1_adv_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p2 prt1_adv_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 prt1_adv_3
+                           Filter: (b = 0)
+(22 rows)
+
+DROP TABLE prt2_adv_p3_1;
+DROP TABLE prt2_adv_p3_2;
+ANALYZE prt2_adv;
+-- Test default partitions
+ALTER TABLE prt1_adv DETACH PARTITION prt1_adv_p1;
+-- Change prt1_adv_p1 to the default partition
+ALTER TABLE prt1_adv ATTACH PARTITION prt1_adv_p1 DEFAULT;
+ALTER TABLE prt1_adv DETACH PARTITION prt1_adv_p3;
+ANALYZE prt1_adv;
+-- We can do partitioned join even if only one of relations has the default
+-- partition
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_1
+                           Filter: (b = 0)
+(15 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+(6 rows)
+
+-- Restore prt1_adv_p3
+ALTER TABLE prt1_adv ATTACH PARTITION prt1_adv_p3 FOR VALUES FROM (300) TO (400);
+ANALYZE prt1_adv;
+-- Restore prt2_adv_p3
+ALTER TABLE prt2_adv ATTACH PARTITION prt2_adv_p3 FOR VALUES FROM (350) TO (500);
+ANALYZE prt2_adv;
+-- Partitioned join can't be applied because the default partition of prt1_adv
+-- matches prt2_adv_p1 and prt2_adv_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Seq Scan on prt2_adv_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p2 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p1 t1_3
+                           Filter: (b = 0)
+(16 rows)
+
+ALTER TABLE prt2_adv DETACH PARTITION prt2_adv_p3;
+-- Change prt2_adv_p3 to the default partition
+ALTER TABLE prt2_adv ATTACH PARTITION prt2_adv_p3 DEFAULT;
+ANALYZE prt2_adv;
+-- Partitioned join can't be applied because the default partition of prt1_adv
+-- matches prt2_adv_p1 and prt2_adv_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Seq Scan on prt2_adv_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p2 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p1 t1_3
+                           Filter: (b = 0)
+(16 rows)
+
+DROP TABLE prt1_adv_p3;
+ANALYZE prt1_adv;
+DROP TABLE prt2_adv_p3;
+ANALYZE prt2_adv;
+CREATE TABLE prt3_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt3_adv_p1 PARTITION OF prt3_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt3_adv_p2 PARTITION OF prt3_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt3_adv_a_idx ON prt3_adv (a);
+INSERT INTO prt3_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(200, 399) i;
+ANALYZE prt3_adv;
+-- 3-way join to test the default partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) LEFT JOIN prt3_adv t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b, t3.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t3_1.a = t1_1.a)
+               ->  Seq Scan on prt3_adv_p1 t3_1
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_2.b = t1_1.a)
+                           ->  Seq Scan on prt2_adv_p2 t2_2
+                           ->  Hash
+                                 ->  Seq Scan on prt1_adv_p2 t1_1
+                                       Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t3_2.a = t1_2.a)
+               ->  Seq Scan on prt3_adv_p2 t3_2
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_1.b = t1_2.a)
+                           ->  Seq Scan on prt2_adv_p1 t2_1
+                           ->  Hash
+                                 ->  Seq Scan on prt1_adv_p1 t1_2
+                                       Filter: (b = 0)
+(23 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) LEFT JOIN prt3_adv t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+  a  |  c   |  b  |  c   |  a  |  c   
+-----+------+-----+------+-----+------
+ 100 | 0100 | 100 | 0100 |     | 
+ 125 | 0125 | 125 | 0125 |     | 
+ 150 | 0150 |     |      |     | 
+ 175 | 0175 |     |      |     | 
+ 200 | 0200 | 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275 | 275 | 0275
+(8 rows)
+
+DROP TABLE prt1_adv;
+DROP TABLE prt2_adv;
+DROP TABLE prt3_adv;
+-- Test interaction of partitioned join with partition pruning
+CREATE TABLE prt1_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_adv_p1 PARTITION OF prt1_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_adv_p2 PARTITION OF prt1_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_adv_p3 PARTITION OF prt1_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_adv_a_idx ON prt1_adv (a);
+INSERT INTO prt1_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_adv;
+CREATE TABLE prt2_adv (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_adv_p1 PARTITION OF prt2_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt2_adv_p2 PARTITION OF prt2_adv FOR VALUES FROM (200) TO (400);
+CREATE INDEX prt2_adv_b_idx ON prt2_adv (b);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+                        QUERY PLAN                         
+-----------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: ((a < 300) AND (b = 0))
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: ((a < 300) AND (b = 0))
+(15 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 | 150 | 0150
+ 175 | 0175 | 175 | 0175
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+(8 rows)
+
+DROP TABLE prt1_adv_p3;
+CREATE TABLE prt1_adv_default PARTITION OF prt1_adv DEFAULT;
+ANALYZE prt1_adv;
+CREATE TABLE prt2_adv_default PARTITION OF prt2_adv DEFAULT;
+ANALYZE prt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+                                QUERY PLAN                                
+--------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: ((a >= 100) AND (a < 300) AND (b = 0))
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: ((a >= 100) AND (a < 300) AND (b = 0))
+(15 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 | 150 | 0150
+ 175 | 0175 | 175 | 0175
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+(8 rows)
+
+DROP TABLE prt1_adv;
+DROP TABLE prt2_adv;
+-- Tests for list-partitioned tables
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0001', '0003');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt1_adv_p3 PARTITION OF plt1_adv FOR VALUES IN ('0008', '0009');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+ANALYZE plt1_adv;
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt2_adv_p3 PARTITION OF plt2_adv FOR VALUES IN ('0007', '0009');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+(6 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 1 | 1 | 0001
+ 8 | 8 | 0008
+(2 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+   |      | 2 | 0002
+   |      | 7 | 0007
+(8 rows)
+
+-- Test cases where one side has an extra partition
+CREATE TABLE plt2_adv_extra PARTITION OF plt2_adv FOR VALUES IN ('0000');
+INSERT INTO plt2_adv_extra VALUES (0, 0, '0000');
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+(6 rows)
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2_adv t1 LEFT JOIN plt1_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt1_adv_p1 t2_1
+               ->  Seq Scan on plt1_adv_p2 t2_2
+               ->  Seq Scan on plt1_adv_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_adv_extra t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_adv_p1 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_adv_p2 t1_3
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_adv_p3 t1_4
+                           Filter: (b < 10)
+(18 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 1 | 1 | 0001
+ 8 | 8 | 0008
+(2 rows)
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt2_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt1_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Anti Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_extra t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t1_4
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t2_1
+                     ->  Seq Scan on plt1_adv_p2 t2_2
+                     ->  Seq Scan on plt1_adv_p3 t2_3
+(18 rows)
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt2_adv_extra t2_1
+               ->  Seq Scan on plt2_adv_p1 t2_2
+               ->  Seq Scan on plt2_adv_p2 t2_3
+               ->  Seq Scan on plt2_adv_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+(15 rows)
+
+DROP TABLE plt2_adv_extra;
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in such cases
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2;
+-- Split plt2_adv_p2 into two partitions so that plt1_adv_p2 matches both
+CREATE TABLE plt2_adv_p2_1 PARTITION OF plt2_adv FOR VALUES IN ('0004');
+CREATE TABLE plt2_adv_p2_2 PARTITION OF plt2_adv FOR VALUES IN ('0006');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6);
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_1 t2_2
+               ->  Seq Scan on plt2_adv_p2_2 t2_3
+               ->  Seq Scan on plt2_adv_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Semi Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+                     ->  Seq Scan on plt2_adv_p2_1 t2_2
+                     ->  Seq Scan on plt2_adv_p2_2 t2_3
+                     ->  Seq Scan on plt2_adv_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_1 t2_2
+               ->  Seq Scan on plt2_adv_p2_2 t2_3
+               ->  Seq Scan on plt2_adv_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(17 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Anti Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+                     ->  Seq Scan on plt2_adv_p2_1 t2_2
+                     ->  Seq Scan on plt2_adv_p2_2 t2_3
+                     ->  Seq Scan on plt2_adv_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_1 t2_2
+               ->  Seq Scan on plt2_adv_p2_2 t2_3
+               ->  Seq Scan on plt2_adv_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+(15 rows)
+
+DROP TABLE plt2_adv_p2_1;
+DROP TABLE plt2_adv_p2_2;
+-- Restore plt2_adv_p2
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2 FOR VALUES IN ('0004', '0006');
+-- Test NULL partitions
+ALTER TABLE plt1_adv DETACH PARTITION plt1_adv_p1;
+-- Change plt1_adv_p1 to the NULL partition
+CREATE TABLE plt1_adv_p1_null PARTITION OF plt1_adv FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p3;
+-- Change plt2_adv_p3 to the NULL partition
+CREATE TABLE plt2_adv_p3_null PARTITION OF plt2_adv FOR VALUES IN (NULL, '0007', '0009');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (7, 9);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1_null t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1_null t1_1
+                     Filter: (b < 10)
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+(19 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1_null t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a  |  c   | a |  c   
+----+------+---+------
+ -1 |      |   | 
+  1 | 0001 |   | 
+  3 | 0003 | 3 | 0003
+  4 | 0004 | 4 | 0004
+  6 | 0006 | 6 | 0006
+  8 | 0008 |   | 
+  9 | 0009 | 9 | 0009
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1_null t1_1
+                     Filter: (b < 10)
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+(19 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a  | b  |  c   
+----+----+------
+ -1 | -1 | 
+  1 |  1 | 0001
+  8 |  8 | 0008
+(3 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p1_null t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+ -1 |      |    | 
+  1 | 0001 |    | 
+  3 | 0003 |  3 | 0003
+  4 | 0004 |  4 | 0004
+  6 | 0006 |  6 | 0006
+  8 | 0008 |    | 
+  9 | 0009 |  9 | 0009
+    |      | -1 | 
+    |      |  2 | 0002
+    |      |  7 | 0007
+(10 rows)
+
+DROP TABLE plt1_adv_p1_null;
+-- Restore plt1_adv_p1
+ALTER TABLE plt1_adv ATTACH PARTITION plt1_adv_p1 FOR VALUES IN ('0001', '0003');
+-- Add to plt1_adv the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt1_adv_extra PARTITION OF plt1_adv FOR VALUES IN (NULL);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+DROP TABLE plt2_adv_p3_null;
+-- Restore plt2_adv_p3
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p3 FOR VALUES IN ('0007', '0009');
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Seq Scan on plt2_adv_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_extra t1_4
+                           Filter: (b < 10)
+(18 rows)
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt1_adv_p1 t1_1
+               ->  Seq Scan on plt1_adv_p2 t1_2
+               ->  Seq Scan on plt1_adv_p3 t1_3
+               ->  Seq Scan on plt1_adv_extra t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+                     ->  Seq Scan on plt2_adv_p2 t2_2
+                     ->  Seq Scan on plt2_adv_p3 t2_3
+(15 rows)
+
+-- Add to plt2_adv the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt2_adv_extra PARTITION OF plt2_adv FOR VALUES IN (NULL);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+         ->  Nested Loop Left Join
+               Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+               ->  Seq Scan on plt1_adv_extra t1_4
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_extra t2_4
+(26 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a  |  c   | a |  c   
+----+------+---+------
+ -1 |      |   | 
+  1 | 0001 |   | 
+  3 | 0003 | 3 | 0003
+  4 | 0004 | 4 | 0004
+  6 | 0006 | 6 | 0006
+  8 | 0008 |   | 
+  9 | 0009 | 9 | 0009
+(7 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p3 t2_3
+         ->  Hash Full Join
+               Hash Cond: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+               Filter: ((COALESCE(t1_4.b, 0) < 10) AND (COALESCE(t2_4.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_extra t1_4
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_extra t2_4
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+ -1 |      |    | 
+  1 | 0001 |    | 
+  3 | 0003 |  3 | 0003
+  4 | 0004 |  4 | 0004
+  6 | 0006 |  6 | 0006
+  8 | 0008 |    | 
+  9 | 0009 |  9 | 0009
+    |      | -1 | 
+    |      |  2 | 0002
+    |      |  7 | 0007
+(10 rows)
+
+-- 3-way join to test the NULL partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt1_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t3_1.a = t1_1.a) AND (t3_1.c = t1_1.c))
+               ->  Seq Scan on plt1_adv_p1 t3_1
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+                           ->  Seq Scan on plt2_adv_p1 t2_1
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p1 t1_1
+                                       Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t3_2.a = t1_2.a) AND (t3_2.c = t1_2.c))
+               ->  Seq Scan on plt1_adv_p2 t3_2
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+                           ->  Seq Scan on plt2_adv_p2 t2_2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p2 t1_2
+                                       Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t3_3.a = t1_3.a) AND (t3_3.c = t1_3.c))
+               ->  Seq Scan on plt1_adv_p3 t3_3
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+                           ->  Seq Scan on plt2_adv_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p3 t1_3
+                                       Filter: (b < 10)
+         ->  Nested Loop Left Join
+               Join Filter: ((t1_4.a = t3_4.a) AND (t1_4.c = t3_4.c))
+               ->  Nested Loop Left Join
+                     Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+                     ->  Seq Scan on plt1_adv_extra t1_4
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_adv_extra t2_4
+               ->  Seq Scan on plt1_adv_extra t3_4
+(41 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt1_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a  |  c   | a |  c   | a |  c   
+----+------+---+------+---+------
+ -1 |      |   |      |   | 
+  1 | 0001 |   |      | 1 | 0001
+  3 | 0003 | 3 | 0003 | 3 | 0003
+  4 | 0004 | 4 | 0004 | 4 | 0004
+  6 | 0006 | 6 | 0006 | 6 | 0006
+  8 | 0008 |   |      | 8 | 0008
+  9 | 0009 | 9 | 0009 | 9 | 0009
+(7 rows)
+
+DROP TABLE plt1_adv_extra;
+DROP TABLE plt2_adv_extra;
+-- Test default partitions
+ALTER TABLE plt1_adv DETACH PARTITION plt1_adv_p1;
+-- Change plt1_adv_p1 to the default partition
+ALTER TABLE plt1_adv ATTACH PARTITION plt1_adv_p1 DEFAULT;
+DROP TABLE plt1_adv_p3;
+ANALYZE plt1_adv;
+DROP TABLE plt2_adv_p3;
+ANALYZE plt2_adv;
+-- We can do partitioned join even if only one of relations has the default
+-- partition
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_1
+                           Filter: (b < 10)
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(3 rows)
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2;
+-- Change plt2_adv_p2 to contain '0005' in addition to '0004' and '0006' as
+-- the key values
+CREATE TABLE plt2_adv_p2_ext PARTITION OF plt2_adv FOR VALUES IN ('0004', '0005', '0006');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 5, 6);
+ANALYZE plt2_adv;
+-- Partitioned join can't be applied because the default partition of plt1_adv
+-- matches plt2_adv_p1 and plt2_adv_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_ext t2_2
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p2 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p1 t1_2
+                           Filter: (b < 10)
+(13 rows)
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2_ext;
+-- Change plt2_adv_p2_ext to the default partition
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2_ext DEFAULT;
+ANALYZE plt2_adv;
+-- Partitioned join can't be applied because the default partition of plt1_adv
+-- matches plt2_adv_p1 and plt2_adv_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_ext t2_2
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p2 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p1 t1_2
+                           Filter: (b < 10)
+(13 rows)
+
+DROP TABLE plt2_adv_p2_ext;
+-- Restore plt2_adv_p2
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_adv;
+CREATE TABLE plt3_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_adv_p1 PARTITION OF plt3_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt3_adv_p2 PARTITION OF plt3_adv FOR VALUES IN ('0007', '0009');
+INSERT INTO plt3_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6, 7, 9);
+ANALYZE plt3_adv;
+-- 3-way join to test the default partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t3_1.a = t1_1.a) AND (t3_1.c = t1_1.c))
+               ->  Seq Scan on plt3_adv_p1 t3_1
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+                           ->  Seq Scan on plt2_adv_p2 t2_2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p2 t1_1
+                                       Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t3_2.a = t1_2.a) AND (t3_2.c = t1_2.c))
+               ->  Seq Scan on plt3_adv_p2 t3_2
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+                           ->  Seq Scan on plt2_adv_p1 t2_1
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p1 t1_2
+                                       Filter: (b < 10)
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   | a |  c   
+---+------+---+------+---+------
+ 1 | 0001 |   |      |   | 
+ 3 | 0003 | 3 | 0003 |   | 
+ 4 | 0004 | 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006 | 6 | 0006
+(4 rows)
+
+-- Test cases where one side has the default partition while the other side
+-- has the NULL partition
+DROP TABLE plt2_adv_p1;
+-- Add the NULL partition to plt2_adv
+CREATE TABLE plt2_adv_p1_null PARTITION OF plt2_adv FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p1_null t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_1
+                           Filter: (b < 10)
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 | 1 | 0001
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(4 rows)
+
+DROP TABLE plt2_adv_p1_null;
+-- Add the NULL partition that contains only NULL values as the key values
+CREATE TABLE plt2_adv_p1_null PARTITION OF plt2_adv FOR VALUES IN (NULL);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Seq Scan on plt2_adv_p2 t2
+         ->  Hash
+               ->  Seq Scan on plt1_adv_p2 t1
+                     Filter: (b < 10)
+(8 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(2 rows)
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+DROP TABLE plt3_adv;
+-- Test interaction of partitioned join with partition pruning
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0001');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0002');
+CREATE TABLE plt1_adv_p3 PARTITION OF plt1_adv FOR VALUES IN ('0003');
+CREATE TABLE plt1_adv_p4 PARTITION OF plt1_adv FOR VALUES IN (NULL, '0004', '0005');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN (NULL);
+CREATE TABLE plt2_adv_p3 PARTITION OF plt2_adv FOR VALUES IN ('0003');
+CREATE TABLE plt2_adv_p4 PARTITION OF plt2_adv FOR VALUES IN ('0004', '0005');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+                                       QUERY PLAN                                        
+-----------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p3 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_1
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p4 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p4 t1_2
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 5 | 0005 | 5 | 0005
+(3 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Seq Scan on plt2_adv_p4 t2
+         ->  Hash
+               ->  Seq Scan on plt1_adv_p4 t1
+                     Filter: ((c IS NULL) AND (b < 10))
+(8 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+ a  | c | a | c 
+----+---+---+---
+ -1 |   |   | 
+(1 row)
+
+CREATE TABLE plt1_adv_default PARTITION OF plt1_adv DEFAULT;
+ANALYZE plt1_adv;
+CREATE TABLE plt2_adv_default PARTITION OF plt2_adv DEFAULT;
+ANALYZE plt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+                                       QUERY PLAN                                        
+-----------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p3 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_1
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p4 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p4 t1_2
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 5 | 0005 | 5 | 0005
+(3 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Seq Scan on plt2_adv_p4 t2
+         ->  Hash
+               ->  Seq Scan on plt1_adv_p4 t1
+                     Filter: ((c IS NULL) AND (b < 10))
+(8 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+ a  | c | a | c 
+----+---+---+---
+ -1 |   |   | 
+(1 row)
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+-- Test the process_outer_partition() code path
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1_adv;
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0002');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2_adv;
+CREATE TABLE plt3_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_adv_p1 PARTITION OF plt3_adv FOR VALUES IN ('0001');
+CREATE TABLE plt3_adv_p2 PARTITION OF plt3_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3_adv;
+-- This tests that when merging partitions from plt1_adv and plt2_adv in
+-- merge_list_bounds(), process_outer_partition() returns an already-assigned
+-- merged partition when re-called with plt1_adv_p1 for the second list value
+-- '0001' of that partitin
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.c = t2.c)) FULL JOIN plt3_adv t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2.a, t3.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t3_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_1.c = t2_1.c)
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                     ->  Hash
+                           ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt3_adv_p1 t3_1
+         ->  Hash Full Join
+               Hash Cond: (t1_2.c = t3_2.c)
+               Filter: (((COALESCE(t1_2.a, 0) % 5) <> 3) AND ((COALESCE(t1_2.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_2.c = t2_2.c)
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                     ->  Hash
+                           ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt3_adv_p2 t3_2
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.c = t2.c)) FULL JOIN plt3_adv t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+ a  |  c   | a  |  c   | a  |  c   
+----+------+----+------+----+------
+  0 | 0000 |    |      |    | 
+  5 | 0000 |    |      |    | 
+ 10 | 0000 |    |      |    | 
+ 15 | 0000 |    |      |    | 
+ 20 | 0000 |    |      |    | 
+  1 | 0001 |    |      |  1 | 0001
+  1 | 0001 |    |      |  6 | 0001
+  1 | 0001 |    |      | 11 | 0001
+  1 | 0001 |    |      | 16 | 0001
+  1 | 0001 |    |      | 21 | 0001
+  6 | 0001 |    |      |  1 | 0001
+  6 | 0001 |    |      |  6 | 0001
+  6 | 0001 |    |      | 11 | 0001
+  6 | 0001 |    |      | 16 | 0001
+  6 | 0001 |    |      | 21 | 0001
+ 11 | 0001 |    |      |  1 | 0001
+ 11 | 0001 |    |      |  6 | 0001
+ 11 | 0001 |    |      | 11 | 0001
+ 11 | 0001 |    |      | 16 | 0001
+ 11 | 0001 |    |      | 21 | 0001
+ 16 | 0001 |    |      |  1 | 0001
+ 16 | 0001 |    |      |  6 | 0001
+ 16 | 0001 |    |      | 11 | 0001
+ 16 | 0001 |    |      | 16 | 0001
+ 16 | 0001 |    |      | 21 | 0001
+ 21 | 0001 |    |      |  1 | 0001
+ 21 | 0001 |    |      |  6 | 0001
+ 21 | 0001 |    |      | 11 | 0001
+ 21 | 0001 |    |      | 16 | 0001
+ 21 | 0001 |    |      | 21 | 0001
+  2 | 0002 |  2 | 0002 |    | 
+  2 | 0002 |  7 | 0002 |    | 
+  2 | 0002 | 12 | 0002 |    | 
+  2 | 0002 | 17 | 0002 |    | 
+  2 | 0002 | 22 | 0002 |    | 
+  7 | 0002 |  2 | 0002 |    | 
+  7 | 0002 |  7 | 0002 |    | 
+  7 | 0002 | 12 | 0002 |    | 
+  7 | 0002 | 17 | 0002 |    | 
+  7 | 0002 | 22 | 0002 |    | 
+ 12 | 0002 |  2 | 0002 |    | 
+ 12 | 0002 |  7 | 0002 |    | 
+ 12 | 0002 | 12 | 0002 |    | 
+ 12 | 0002 | 17 | 0002 |    | 
+ 12 | 0002 | 22 | 0002 |    | 
+ 17 | 0002 |  2 | 0002 |    | 
+ 17 | 0002 |  7 | 0002 |    | 
+ 17 | 0002 | 12 | 0002 |    | 
+ 17 | 0002 | 17 | 0002 |    | 
+ 17 | 0002 | 22 | 0002 |    | 
+ 22 | 0002 |  2 | 0002 |    | 
+ 22 | 0002 |  7 | 0002 |    | 
+ 22 | 0002 | 12 | 0002 |    | 
+ 22 | 0002 | 17 | 0002 |    | 
+ 22 | 0002 | 22 | 0002 |    | 
+(55 rows)
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+DROP TABLE plt3_adv;
+-- Tests for multi-level partitioned tables
+CREATE TABLE alpha (a double precision, b int, c text) PARTITION BY RANGE (a);
+CREATE TABLE alpha_neg PARTITION OF alpha FOR VALUES FROM ('-Infinity') TO (0) PARTITION BY RANGE (b);
+CREATE TABLE alpha_pos PARTITION OF alpha FOR VALUES FROM (0) TO (10.0) PARTITION BY LIST (c);
+CREATE TABLE alpha_neg_p1 PARTITION OF alpha_neg FOR VALUES FROM (100) TO (200);
+CREATE TABLE alpha_neg_p2 PARTITION OF alpha_neg FOR VALUES FROM (200) TO (300);
+CREATE TABLE alpha_neg_p3 PARTITION OF alpha_neg FOR VALUES FROM (300) TO (400);
+CREATE TABLE alpha_pos_p1 PARTITION OF alpha_pos FOR VALUES IN ('0001', '0003');
+CREATE TABLE alpha_pos_p2 PARTITION OF alpha_pos FOR VALUES IN ('0004', '0006');
+CREATE TABLE alpha_pos_p3 PARTITION OF alpha_pos FOR VALUES IN ('0008', '0009');
+INSERT INTO alpha_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+INSERT INTO alpha_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+ANALYZE alpha;
+CREATE TABLE beta (a double precision, b int, c text) PARTITION BY RANGE (a);
+CREATE TABLE beta_neg PARTITION OF beta FOR VALUES FROM (-10.0) TO (0) PARTITION BY RANGE (b);
+CREATE TABLE beta_pos PARTITION OF beta FOR VALUES FROM (0) TO ('Infinity') PARTITION BY LIST (c);
+CREATE TABLE beta_neg_p1 PARTITION OF beta_neg FOR VALUES FROM (100) TO (150);
+CREATE TABLE beta_neg_p2 PARTITION OF beta_neg FOR VALUES FROM (200) TO (300);
+CREATE TABLE beta_neg_p3 PARTITION OF beta_neg FOR VALUES FROM (350) TO (500);
+CREATE TABLE beta_pos_p1 PARTITION OF beta_pos FOR VALUES IN ('0002', '0003');
+CREATE TABLE beta_pos_p2 PARTITION OF beta_pos FOR VALUES IN ('0004', '0006');
+CREATE TABLE beta_pos_p3 PARTITION OF beta_pos FOR VALUES IN ('0007', '0009');
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+ANALYZE beta;
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.b
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.b = t2_1.b))
+               ->  Seq Scan on alpha_neg_p1 t1_1
+                     Filter: ((b >= 125) AND (b < 225))
+               ->  Hash
+                     ->  Seq Scan on beta_neg_p1 t2_1
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.b = t1_2.b))
+               ->  Seq Scan on beta_neg_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on alpha_neg_p2 t1_2
+                           Filter: ((b >= 125) AND (b < 225))
+         ->  Hash Join
+               Hash Cond: ((t2_4.a = t1_4.a) AND (t2_4.b = t1_4.b))
+               ->  Append
+                     ->  Seq Scan on beta_pos_p1 t2_4
+                     ->  Seq Scan on beta_pos_p2 t2_5
+                     ->  Seq Scan on beta_pos_p3 t2_6
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on alpha_pos_p1 t1_4
+                                 Filter: ((b >= 125) AND (b < 225))
+                           ->  Seq Scan on alpha_pos_p2 t1_5
+                                 Filter: ((b >= 125) AND (b < 225))
+                           ->  Seq Scan on alpha_pos_p3 t1_6
+                                 Filter: ((b >= 125) AND (b < 225))
+(29 rows)
+
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b;
+ a  |  b  |  c   | a  |  b  |  c   
+----+-----+------+----+-----+------
+ -1 | 126 | 0006 | -1 | 126 | 0006
+ -1 | 129 | 0009 | -1 | 129 | 0009
+ -1 | 133 | 0003 | -1 | 133 | 0003
+ -1 | 134 | 0004 | -1 | 134 | 0004
+ -1 | 136 | 0006 | -1 | 136 | 0006
+ -1 | 139 | 0009 | -1 | 139 | 0009
+ -1 | 143 | 0003 | -1 | 143 | 0003
+ -1 | 144 | 0004 | -1 | 144 | 0004
+ -1 | 146 | 0006 | -1 | 146 | 0006
+ -1 | 149 | 0009 | -1 | 149 | 0009
+ -1 | 203 | 0003 | -1 | 203 | 0003
+ -1 | 204 | 0004 | -1 | 204 | 0004
+ -1 | 206 | 0006 | -1 | 206 | 0006
+ -1 | 209 | 0009 | -1 | 209 | 0009
+ -1 | 213 | 0003 | -1 | 213 | 0003
+ -1 | 214 | 0004 | -1 | 214 | 0004
+ -1 | 216 | 0006 | -1 | 216 | 0006
+ -1 | 219 | 0009 | -1 | 219 | 0009
+ -1 | 223 | 0003 | -1 | 223 | 0003
+ -1 | 224 | 0004 | -1 | 224 | 0004
+  1 | 126 | 0006 |  1 | 126 | 0006
+  1 | 129 | 0009 |  1 | 129 | 0009
+  1 | 133 | 0003 |  1 | 133 | 0003
+  1 | 134 | 0004 |  1 | 134 | 0004
+  1 | 136 | 0006 |  1 | 136 | 0006
+  1 | 139 | 0009 |  1 | 139 | 0009
+  1 | 143 | 0003 |  1 | 143 | 0003
+  1 | 144 | 0004 |  1 | 144 | 0004
+  1 | 146 | 0006 |  1 | 146 | 0006
+  1 | 149 | 0009 |  1 | 149 | 0009
+  1 | 203 | 0003 |  1 | 203 | 0003
+  1 | 204 | 0004 |  1 | 204 | 0004
+  1 | 206 | 0006 |  1 | 206 | 0006
+  1 | 209 | 0009 |  1 | 209 | 0009
+  1 | 213 | 0003 |  1 | 213 | 0003
+  1 | 214 | 0004 |  1 | 214 | 0004
+  1 | 216 | 0006 |  1 | 216 | 0006
+  1 | 219 | 0009 |  1 | 219 | 0009
+  1 | 223 | 0003 |  1 | 223 | 0003
+  1 | 224 | 0004 |  1 | 224 | 0004
+(40 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b, t2.b;
+                                                              QUERY PLAN                                                              
+--------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.b, t2.b
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Append
+                     ->  Seq Scan on alpha_neg_p1 t1_2
+                           Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+                     ->  Seq Scan on alpha_neg_p2 t1_3
+                           Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on beta_neg_p1 t2_2
+                                 Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+                           ->  Seq Scan on beta_neg_p2 t2_3
+                                 Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Nested Loop
+               Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+               ->  Seq Scan on alpha_pos_p2 t1_4
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Seq Scan on beta_pos_p2 t2_4
+                     Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Nested Loop
+               Join Filter: ((t1_5.a = t2_5.a) AND (t1_5.c = t2_5.c))
+               ->  Seq Scan on alpha_pos_p3 t1_5
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Seq Scan on beta_pos_p3 t2_5
+                     Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+(28 rows)
+
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b, t2.b;
+ a  |  b  |  c   | a  |  b  |  c   
+----+-----+------+----+-----+------
+ -1 | 104 | 0004 | -1 | 104 | 0004
+ -1 | 104 | 0004 | -1 | 204 | 0004
+ -1 | 109 | 0009 | -1 | 109 | 0009
+ -1 | 109 | 0009 | -1 | 209 | 0009
+ -1 | 204 | 0004 | -1 | 104 | 0004
+ -1 | 204 | 0004 | -1 | 204 | 0004
+ -1 | 209 | 0009 | -1 | 109 | 0009
+ -1 | 209 | 0009 | -1 | 209 | 0009
+  1 | 104 | 0004 |  1 | 104 | 0004
+  1 | 104 | 0004 |  1 | 204 | 0004
+  1 | 109 | 0009 |  1 | 109 | 0009
+  1 | 109 | 0009 |  1 | 209 | 0009
+  1 | 204 | 0004 |  1 | 104 | 0004
+  1 | 204 | 0004 |  1 | 204 | 0004
+  1 | 209 | 0009 |  1 | 109 | 0009
+  1 | 209 | 0009 |  1 | 209 | 0009
 (16 rows)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;
+                                                           QUERY PLAN                                                           
+--------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.b
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on alpha_neg_p1 t1_1
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Hash
+                     ->  Seq Scan on beta_neg_p1 t2_1
+                           Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Hash Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on alpha_neg_p2 t1_2
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Hash
+                     ->  Seq Scan on beta_neg_p2 t2_2
+                           Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Nested Loop
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.b = t2_3.b) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on alpha_pos_p2 t1_3
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Seq Scan on beta_pos_p2 t2_3
+                     Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Nested Loop
+               Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.b = t2_4.b) AND (t1_4.c = t2_4.c))
+               ->  Seq Scan on alpha_pos_p3 t1_4
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Seq Scan on beta_pos_p3 t2_4
+                     Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+(29 rows)
+
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;
+ a  |  b  |  c   | a  |  b  |  c   
+----+-----+------+----+-----+------
+ -1 | 104 | 0004 | -1 | 104 | 0004
+ -1 | 109 | 0009 | -1 | 109 | 0009
+ -1 | 204 | 0004 | -1 | 204 | 0004
+ -1 | 209 | 0009 | -1 | 209 | 0009
+  1 | 104 | 0004 |  1 | 104 | 0004
+  1 | 109 | 0009 |  1 | 109 | 0009
+  1 | 204 | 0004 |  1 | 204 | 0004
+  1 | 209 | 0009 |  1 | 209 | 0009
+(8 rows)
+
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 575ba7b8d4..df30f851e8 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -445,11 +445,638 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
 
--- partitionwise join can not be applied if only one of joining tables has
--- default partition
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
-ANALYZE prt2;
 
+--
+-- Test advanced partition-matching algorithm for partitioned join
+--
+
+-- Tests for range-partitioned tables
+CREATE TABLE prt1_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_adv_p1 PARTITION OF prt1_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_adv_p2 PARTITION OF prt1_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_adv_p3 PARTITION OF prt1_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_adv_a_idx ON prt1_adv (a);
+INSERT INTO prt1_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_adv;
+
+CREATE TABLE prt2_adv (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_adv_p1 PARTITION OF prt2_adv FOR VALUES FROM (100) TO (150);
+CREATE TABLE prt2_adv_p2 PARTITION OF prt2_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt2_adv_p3 PARTITION OF prt2_adv FOR VALUES FROM (350) TO (500);
+CREATE INDEX prt2_adv_b_idx ON prt2_adv (b);
+INSERT INTO prt2_adv_p1 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 149) i;
+INSERT INTO prt2_adv_p2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(200, 299) i;
+INSERT INTO prt2_adv_p3 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_adv;
+
+-- inner join
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+-- Test cases where one side has an extra partition
+CREATE TABLE prt2_adv_extra PARTITION OF prt2_adv FOR VALUES FROM (500) TO (MAXVALUE);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(500, 599) i;
+ANALYZE prt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt1_adv t2 WHERE t1.b = t2.a) AND t1.a = 0 ORDER BY t1.b;
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+-- 3-way join where not every pair of relations can do partitioned join
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c, t3.a, t3.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) INNER JOIN prt1_adv t3 ON (t1.b = t3.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a, t3.a;
+SELECT t1.b, t1.c, t2.a, t2.c, t3.a, t3.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) INNER JOIN prt1_adv t3 ON (t1.b = t3.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a, t3.a;
+
+DROP TABLE prt2_adv_extra;
+
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in such cases
+ALTER TABLE prt2_adv DETACH PARTITION prt2_adv_p3;
+-- Split prt2_adv_p3 into two partitions so that prt1_adv_p3 matches both
+CREATE TABLE prt2_adv_p3_1 PARTITION OF prt2_adv FOR VALUES FROM (350) TO (375);
+CREATE TABLE prt2_adv_p3_2 PARTITION OF prt2_adv FOR VALUES FROM (375) TO (500);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+DROP TABLE prt2_adv_p3_1;
+DROP TABLE prt2_adv_p3_2;
+ANALYZE prt2_adv;
+
+-- Test default partitions
+ALTER TABLE prt1_adv DETACH PARTITION prt1_adv_p1;
+-- Change prt1_adv_p1 to the default partition
+ALTER TABLE prt1_adv ATTACH PARTITION prt1_adv_p1 DEFAULT;
+ALTER TABLE prt1_adv DETACH PARTITION prt1_adv_p3;
+ANALYZE prt1_adv;
+
+-- We can do partitioned join even if only one of relations has the default
+-- partition
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- Restore prt1_adv_p3
+ALTER TABLE prt1_adv ATTACH PARTITION prt1_adv_p3 FOR VALUES FROM (300) TO (400);
+ANALYZE prt1_adv;
+
+-- Restore prt2_adv_p3
+ALTER TABLE prt2_adv ATTACH PARTITION prt2_adv_p3 FOR VALUES FROM (350) TO (500);
+ANALYZE prt2_adv;
+
+-- Partitioned join can't be applied because the default partition of prt1_adv
+-- matches prt2_adv_p1 and prt2_adv_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+ALTER TABLE prt2_adv DETACH PARTITION prt2_adv_p3;
+-- Change prt2_adv_p3 to the default partition
+ALTER TABLE prt2_adv ATTACH PARTITION prt2_adv_p3 DEFAULT;
+ANALYZE prt2_adv;
+
+-- Partitioned join can't be applied because the default partition of prt1_adv
+-- matches prt2_adv_p1 and prt2_adv_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt1_adv_p3;
+ANALYZE prt1_adv;
+
+DROP TABLE prt2_adv_p3;
+ANALYZE prt2_adv;
+
+CREATE TABLE prt3_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt3_adv_p1 PARTITION OF prt3_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt3_adv_p2 PARTITION OF prt3_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt3_adv_a_idx ON prt3_adv (a);
+INSERT INTO prt3_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(200, 399) i;
+ANALYZE prt3_adv;
+
+-- 3-way join to test the default partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) LEFT JOIN prt3_adv t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) LEFT JOIN prt3_adv t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+
+DROP TABLE prt1_adv;
+DROP TABLE prt2_adv;
+DROP TABLE prt3_adv;
+
+-- Test interaction of partitioned join with partition pruning
+CREATE TABLE prt1_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_adv_p1 PARTITION OF prt1_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_adv_p2 PARTITION OF prt1_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_adv_p3 PARTITION OF prt1_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_adv_a_idx ON prt1_adv (a);
+INSERT INTO prt1_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_adv;
+
+CREATE TABLE prt2_adv (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_adv_p1 PARTITION OF prt2_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt2_adv_p2 PARTITION OF prt2_adv FOR VALUES FROM (200) TO (400);
+CREATE INDEX prt2_adv_b_idx ON prt2_adv (b);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt1_adv_p3;
+CREATE TABLE prt1_adv_default PARTITION OF prt1_adv DEFAULT;
+ANALYZE prt1_adv;
+
+CREATE TABLE prt2_adv_default PARTITION OF prt2_adv DEFAULT;
+ANALYZE prt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt1_adv;
+DROP TABLE prt2_adv;
+
+
+-- Tests for list-partitioned tables
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0001', '0003');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt1_adv_p3 PARTITION OF plt1_adv FOR VALUES IN ('0008', '0009');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+ANALYZE plt1_adv;
+
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt2_adv_p3 PARTITION OF plt2_adv FOR VALUES IN ('0007', '0009');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+-- Test cases where one side has an extra partition
+CREATE TABLE plt2_adv_extra PARTITION OF plt2_adv FOR VALUES IN ('0000');
+INSERT INTO plt2_adv_extra VALUES (0, 0, '0000');
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2_adv t1 LEFT JOIN plt1_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt2_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt1_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt2_adv_extra;
+
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in such cases
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2;
+-- Split plt2_adv_p2 into two partitions so that plt1_adv_p2 matches both
+CREATE TABLE plt2_adv_p2_1 PARTITION OF plt2_adv FOR VALUES IN ('0004');
+CREATE TABLE plt2_adv_p2_2 PARTITION OF plt2_adv FOR VALUES IN ('0006');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6);
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt2_adv_p2_1;
+DROP TABLE plt2_adv_p2_2;
+-- Restore plt2_adv_p2
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2 FOR VALUES IN ('0004', '0006');
+
+-- Test NULL partitions
+ALTER TABLE plt1_adv DETACH PARTITION plt1_adv_p1;
+-- Change plt1_adv_p1 to the NULL partition
+CREATE TABLE plt1_adv_p1_null PARTITION OF plt1_adv FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p3;
+-- Change plt2_adv_p3 to the NULL partition
+CREATE TABLE plt2_adv_p3_null PARTITION OF plt2_adv FOR VALUES IN (NULL, '0007', '0009');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (7, 9);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt1_adv_p1_null;
+-- Restore plt1_adv_p1
+ALTER TABLE plt1_adv ATTACH PARTITION plt1_adv_p1 FOR VALUES IN ('0001', '0003');
+-- Add to plt1_adv the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt1_adv_extra PARTITION OF plt1_adv FOR VALUES IN (NULL);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+
+DROP TABLE plt2_adv_p3_null;
+-- Restore plt2_adv_p3
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p3 FOR VALUES IN ('0007', '0009');
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+-- Add to plt2_adv the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt2_adv_extra PARTITION OF plt2_adv FOR VALUES IN (NULL);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+-- 3-way join to test the NULL partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt1_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt1_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt1_adv_extra;
+DROP TABLE plt2_adv_extra;
+
+-- Test default partitions
+ALTER TABLE plt1_adv DETACH PARTITION plt1_adv_p1;
+-- Change plt1_adv_p1 to the default partition
+ALTER TABLE plt1_adv ATTACH PARTITION plt1_adv_p1 DEFAULT;
+DROP TABLE plt1_adv_p3;
+ANALYZE plt1_adv;
+
+DROP TABLE plt2_adv_p3;
+ANALYZE plt2_adv;
+
+-- We can do partitioned join even if only one of relations has the default
+-- partition
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2;
+-- Change plt2_adv_p2 to contain '0005' in addition to '0004' and '0006' as
+-- the key values
+CREATE TABLE plt2_adv_p2_ext PARTITION OF plt2_adv FOR VALUES IN ('0004', '0005', '0006');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 5, 6);
+ANALYZE plt2_adv;
+
+-- Partitioned join can't be applied because the default partition of plt1_adv
+-- matches plt2_adv_p1 and plt2_adv_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2_ext;
+-- Change plt2_adv_p2_ext to the default partition
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2_ext DEFAULT;
+ANALYZE plt2_adv;
+
+-- Partitioned join can't be applied because the default partition of plt1_adv
+-- matches plt2_adv_p1 and plt2_adv_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt2_adv_p2_ext;
+-- Restore plt2_adv_p2
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_adv;
+
+CREATE TABLE plt3_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_adv_p1 PARTITION OF plt3_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt3_adv_p2 PARTITION OF plt3_adv FOR VALUES IN ('0007', '0009');
+INSERT INTO plt3_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6, 7, 9);
+ANALYZE plt3_adv;
+
+-- 3-way join to test the default partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Test cases where one side has the default partition while the other side
+-- has the NULL partition
+DROP TABLE plt2_adv_p1;
+-- Add the NULL partition to plt2_adv
+CREATE TABLE plt2_adv_p1_null PARTITION OF plt2_adv FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt2_adv_p1_null;
+-- Add the NULL partition that contains only NULL values as the key values
+CREATE TABLE plt2_adv_p1_null PARTITION OF plt2_adv FOR VALUES IN (NULL);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+DROP TABLE plt3_adv;
+
+-- Test interaction of partitioned join with partition pruning
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0001');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0002');
+CREATE TABLE plt1_adv_p3 PARTITION OF plt1_adv FOR VALUES IN ('0003');
+CREATE TABLE plt1_adv_p4 PARTITION OF plt1_adv FOR VALUES IN (NULL, '0004', '0005');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN (NULL);
+CREATE TABLE plt2_adv_p3 PARTITION OF plt2_adv FOR VALUES IN ('0003');
+CREATE TABLE plt2_adv_p4 PARTITION OF plt2_adv FOR VALUES IN ('0004', '0005');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+
+CREATE TABLE plt1_adv_default PARTITION OF plt1_adv DEFAULT;
+ANALYZE plt1_adv;
+
+CREATE TABLE plt2_adv_default PARTITION OF plt2_adv DEFAULT;
+ANALYZE plt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+
+-- Test the process_outer_partition() code path
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1_adv;
+
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0002');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2_adv;
+
+CREATE TABLE plt3_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_adv_p1 PARTITION OF plt3_adv FOR VALUES IN ('0001');
+CREATE TABLE plt3_adv_p2 PARTITION OF plt3_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3_adv;
+
+-- This tests that when merging partitions from plt1_adv and plt2_adv in
+-- merge_list_bounds(), process_outer_partition() returns an already-assigned
+-- merged partition when re-called with plt1_adv_p1 for the second list value
+-- '0001' of that partitin
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.c = t2.c)) FULL JOIN plt3_adv t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.c = t2.c)) FULL JOIN plt3_adv t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+DROP TABLE plt3_adv;
+
+
+-- Tests for multi-level partitioned tables
+CREATE TABLE alpha (a double precision, b int, c text) PARTITION BY RANGE (a);
+CREATE TABLE alpha_neg PARTITION OF alpha FOR VALUES FROM ('-Infinity') TO (0) PARTITION BY RANGE (b);
+CREATE TABLE alpha_pos PARTITION OF alpha FOR VALUES FROM (0) TO (10.0) PARTITION BY LIST (c);
+CREATE TABLE alpha_neg_p1 PARTITION OF alpha_neg FOR VALUES FROM (100) TO (200);
+CREATE TABLE alpha_neg_p2 PARTITION OF alpha_neg FOR VALUES FROM (200) TO (300);
+CREATE TABLE alpha_neg_p3 PARTITION OF alpha_neg FOR VALUES FROM (300) TO (400);
+CREATE TABLE alpha_pos_p1 PARTITION OF alpha_pos FOR VALUES IN ('0001', '0003');
+CREATE TABLE alpha_pos_p2 PARTITION OF alpha_pos FOR VALUES IN ('0004', '0006');
+CREATE TABLE alpha_pos_p3 PARTITION OF alpha_pos FOR VALUES IN ('0008', '0009');
+INSERT INTO alpha_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+INSERT INTO alpha_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+ANALYZE alpha;
+
+CREATE TABLE beta (a double precision, b int, c text) PARTITION BY RANGE (a);
+CREATE TABLE beta_neg PARTITION OF beta FOR VALUES FROM (-10.0) TO (0) PARTITION BY RANGE (b);
+CREATE TABLE beta_pos PARTITION OF beta FOR VALUES FROM (0) TO ('Infinity') PARTITION BY LIST (c);
+CREATE TABLE beta_neg_p1 PARTITION OF beta_neg FOR VALUES FROM (100) TO (150);
+CREATE TABLE beta_neg_p2 PARTITION OF beta_neg FOR VALUES FROM (200) TO (300);
+CREATE TABLE beta_neg_p3 PARTITION OF beta_neg FOR VALUES FROM (350) TO (500);
+CREATE TABLE beta_pos_p1 PARTITION OF beta_pos FOR VALUES IN ('0002', '0003');
+CREATE TABLE beta_pos_p2 PARTITION OF beta_pos FOR VALUES IN ('0004', '0006');
+CREATE TABLE beta_pos_p3 PARTITION OF beta_pos FOR VALUES IN ('0007', '0009');
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+ANALYZE beta;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b;
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b, t2.b;
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b, t2.b;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;
-- 
2.17.1

0004-Fix-some-white-space-errors-in-the-previous-commits.patchtext/x-patch; charset=US-ASCII; name=0004-Fix-some-white-space-errors-in-the-previous-commits.patchDownload
From 6005eb2766c0c78f56c97223cd15b693e3bb5133 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@2ndquadrant.com>
Date: Wed, 1 Apr 2020 15:44:38 +0530
Subject: [PATCH 4/5] Fix some white space errors in the previous commits.

---
 src/backend/partitioning/partbounds.c | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index ae426c1a30..ede16cba15 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -2046,13 +2046,13 @@ merge_null_partitions(PartitionMap *outer_map,
 	{
 		Assert(outer_null >= 0 && outer_null < outer_map->nparts);
 		if (outer_map->merged_indexes[outer_null] == -1)
-		 	consider_outer_null = true;
+			consider_outer_null = true;
 	}
 	if (inner_has_null)
 	{
 		Assert(inner_null >= 0 && inner_null < inner_map->nparts);
 		if (inner_map->merged_indexes[inner_null] == -1)
-		 	consider_inner_null = true;
+			consider_inner_null = true;
 	}
 
 	/* If both flags are set false, we don't need to do anything. */
-- 
2.17.1

0005-Address-Tomas-s-comments.patchtext/x-patch; charset=US-ASCII; name=0005-Address-Tomas-s-comments.patchDownload
From af0e27b033082bed836cb49cdf0b2614ebf9bba1 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@2ndquadrant.com>
Date: Wed, 1 Apr 2020 22:17:26 +0530
Subject: [PATCH 5/5] Address Tomas's comments.

---
 doc/src/sgml/config.sgml              |   6 +-
 src/backend/nodes/outfuncs.c          |   2 +-
 src/backend/optimizer/README          |  37 +++++--
 src/backend/optimizer/path/joinrels.c |  31 ++++--
 src/backend/optimizer/util/relnode.c  |  30 ++++--
 src/backend/partitioning/partbounds.c | 143 +++++++++++++++++++-------
 src/include/nodes/pathnodes.h         |  19 +++-
 7 files changed, 200 insertions(+), 68 deletions(-)

diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 2de21903a1..d61a01c156 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4693,9 +4693,9 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
         which allows a join between partitioned tables to be performed by
         joining the matching partitions.  Partitionwise join currently applies
         only when the join conditions include all the partition keys, which
-        must be of the same data type and have exactly matching sets of child
-        partitions.  Because partitionwise join planning can use significantly
-        more CPU time and memory during planning, the default is
+        must be of the same data type and when there is exactly one pair of
+        matching partitions.  Because partitionwise join planning can use
+        significantly more CPU time and memory during planning, the default is
         <literal>off</literal>.
        </para>
       </listitem>
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 79c768409c..f3361780f1 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -2288,7 +2288,7 @@ _outRelOptInfo(StringInfo str, const RelOptInfo *node)
 	WRITE_BOOL_FIELD(has_eclass_joins);
 	WRITE_BOOL_FIELD(consider_partitionwise_join);
 	WRITE_BITMAPSET_FIELD(top_parent_relids);
-	WRITE_BOOL_FIELD(merged);
+	WRITE_BOOL_FIELD(partbounds_merged);
 	WRITE_BITMAPSET_FIELD(all_partrels);
 	WRITE_NODE_FIELD(partitioned_child_rels);
 }
diff --git a/src/backend/optimizer/README b/src/backend/optimizer/README
index 89ce373d5e..137b060ac6 100644
--- a/src/backend/optimizer/README
+++ b/src/backend/optimizer/README
@@ -1092,13 +1092,26 @@ Partitionwise joins
 -------------------
 
 A join between two similarly partitioned tables can be broken down into joins
-between their matching partitions if there exists an equi-join condition
-between the partition keys of the joining tables. The equi-join between
-partition keys implies that all join partners for a given row in one
-partitioned table must be in the corresponding partition of the other
-partitioned table. Because of this the join between partitioned tables to be
-broken into joins between the matching partitions. The resultant join is
-partitioned in the same way as the joining relations, thus allowing an N-way
+between their matching partitions under the following conditions
+
+1. there exists an equi-join condition between the partition keys of the
+joining tables.
+
+2. at most one partition on one side of the join matches a given partition on
+the other side of join for each side.
+
+3. In case of LEFT, RIGHT and FULL OUTER joins, a given partition on the outer
+side has exactly one matching partition on the inner side. For FULL OUTER join
+both the sides act as outer sides.
+
+The equi-join between partition keys implies that all join partners for a given
+row in one partitioned table must be in the matching partition of the other
+partitioned table. Because of this the join between partitioned tables can be
+broken into joins between the matching partitions. The resultant join has the
+same partitioning scheme as the joining relations and has the partition bounds
+resulting from the merging the partition bounds from matching partitions. The
+algorithm to find out matching partitions and resulting partition bounds is
+implemented in partition_bounds_merge() and its minions.  This allows an N-way
 join between similarly partitioned tables having equi-join condition between
 their partition keys to be broken down into N-way joins between their matching
 partitions. This technique of breaking down a join between partitioned tables
@@ -1106,6 +1119,16 @@ into joins between their partitions is called partitionwise join. We will use
 term "partitioned relation" for either a partitioned table or a join between
 compatibly partitioned tables.
 
+A default partition of a joining relation may have rows which match rows in any
+of the partitions of the other joining relation that do not have a matching
+partition in the first joining relation and vice versa. Thus when a partition
+from one joining relation which does not have a matching partition in the other
+joining relation, the default partition from the other joining relation becomes
+its matching partition and vice versa. If both the relations have default
+partitions, they form a matching pair. This might lead a default partition from
+one joining relation to be matched to multiple partitions from the other
+joining relation violating second condition mentioned above.
+
 The partitioning properties of a partitioned relation are stored in its
 RelOptInfo.  The information about data types of partition keys are stored in
 PartitionSchemeData structure. The planner maintains a list of canonical
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 314b0267c3..8cee4f6cd6 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -43,6 +43,10 @@ static void try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1,
 								   RelOptInfo *rel2, RelOptInfo *joinrel,
 								   SpecialJoinInfo *parent_sjinfo,
 								   List *parent_restrictlist);
+static void compute_partition_bounds(PlannerInfo *root, RelOptInfo *rel1,
+						 RelOptInfo *rel2, RelOptInfo *joinrel,
+						 SpecialJoinInfo *parent_sjinfo,
+						 List **parts1, List **parts2);
 static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 												SpecialJoinInfo *parent_sjinfo,
 												Relids left_relids, Relids right_relids);
@@ -1333,6 +1337,11 @@ restriction_is_constant_false(List *restrictlist,
 	return false;
 }
 
+/*
+ * Compute partition bounds for given join relation based on the given pair of
+ * joining relations. If the bounds have been already computed, find the pairs
+ * of partitions to be joined.
+ */
 static void
 compute_partition_bounds(PlannerInfo *root, RelOptInfo *rel1,
 						 RelOptInfo *rel2, RelOptInfo *joinrel,
@@ -1362,8 +1371,8 @@ compute_partition_bounds(PlannerInfo *root, RelOptInfo *rel1,
 		 * it would be possible for both the bounds to be exactly the same, but
 		 * it seems unlikely to be worth the cycles to check.
 		 */
-		if (!rel1->merged &&
-			!rel2->merged &&
+		if (!rel1->partbounds_merged &&
+			!rel2->partbounds_merged &&
 			rel1->nparts == rel2->nparts &&
 			partition_bounds_equal(part_scheme->partnatts,
 								   part_scheme->parttyplen,
@@ -1388,7 +1397,7 @@ compute_partition_bounds(PlannerInfo *root, RelOptInfo *rel1,
 				return;
 			}
 			nparts = list_length(*parts1);
-			joinrel->merged = true;
+			joinrel->partbounds_merged = true;
 		}
 
 		Assert(nparts > 0);
@@ -1404,13 +1413,15 @@ compute_partition_bounds(PlannerInfo *root, RelOptInfo *rel1,
 		Assert(joinrel->part_rels);
 
 		/*
-		 * If the join rel's merged flag is true, it means inputs are not
+		 * If the join relation's bounds were computed by merging the bounds of
+		 * one of the previous joining pairs, it means inputs may not be
 		 * guaranteed to have the same partition bounds, therefore we can't
 		 * assume that the partitions at the same cardinal positions form the
 		 * pairs; let get_matching_part_pairs() generate the pairs.  Otherwise,
-		 * nothing to do since we can assume that.
+		 * nothing to do since matching partitions are at the same positions in
+		 * both the part_rels array.
 		 */
-		if (joinrel->merged)
+		if (joinrel->partbounds_merged)
 		{
 			get_matching_part_pairs(root, joinrel, rel1, rel2,
 									parts1, parts2);
@@ -1457,7 +1468,7 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
+	if (IS_JOINREL_NOT_PARTITITIONED(joinrel))
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
@@ -1483,12 +1494,12 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	Assert(joinrel->part_scheme == rel1->part_scheme &&
 		   joinrel->part_scheme == rel2->part_scheme);
 
-	Assert(!(joinrel->merged && (joinrel->nparts <= 0)));
+	Assert(!(joinrel->partbounds_merged && (joinrel->nparts <= 0)));
 
 	compute_partition_bounds(root, rel1, rel2, joinrel, parent_sjinfo,
 							 &parts1, &parts2);
 
-	if (joinrel->merged)
+	if (joinrel->partbounds_merged)
 	{
 		lcr1 = list_head(parts1);
 		lcr2 = list_head(parts2);
@@ -1512,7 +1523,7 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
-		if (joinrel->merged)
+		if (joinrel->partbounds_merged)
 		{
 			child_rel1 = lfirst_node(RelOptInfo, lcr1);
 			child_rel2 = lfirst_node(RelOptInfo, lcr2);
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 0e4944ac8e..7df76a22ed 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -242,7 +242,7 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->part_scheme = NULL;
 	rel->nparts = -1;
 	rel->boundinfo = NULL;
-	rel->merged = false;
+	rel->partbounds_merged = false;
 	rel->partition_qual = NIL;
 	rel->part_rels = NULL;
 	rel->all_partrels = NULL;
@@ -657,7 +657,7 @@ build_join_rel(PlannerInfo *root,
 	joinrel->part_scheme = NULL;
 	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
-	joinrel->merged = false;
+	joinrel->partbounds_merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
 	joinrel->all_partrels = NULL;
@@ -835,7 +835,7 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->part_scheme = NULL;
 	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
-	joinrel->merged = false;
+	joinrel->partbounds_merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
 	joinrel->all_partrels = NULL;
@@ -1638,12 +1638,22 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * We can only consider this join as an input to further partitionwise
 	 * joins if (a) the input relations are partitioned and have
 	 * consider_partitionwise_join=true, (b) the partition schemes match, and
-	 * (c) we can identify an equi-join between the partition keys.  Note that
-	 * if it were possible for have_partkey_equi_join to return different
-	 * answers for the same joinrel depending on which join ordering we try
-	 * first, this logic would break.  That shouldn't happen, though, because
-	 * of the way the query planner deduces implied equalities and reorders
-	 * the joins.  Please see optimizer/README for details.
+	 * (c) we can identify an equi-join between the partition keys.
+	 *
+	 * Note that if it were possible for have_partkey_equi_join to return
+	 * different answers for the same joinrel depending on which join ordering
+	 * we try first, this logic would break.  That shouldn't happen, though,
+	 * because of the way the query planner deduces implied equalities and
+	 * reorders the joins.  Please see optimizer/README for details.
+	 *
+	 * It might be possible though that one or both relations in the given pair
+	 * of joining relations do not have partition bounds set but a later pair
+	 * does. This is possible if partitionwise join was not possible for one of
+	 * the joining relation (which in itself is a join relation) because we
+	 * could not merge bounds for none of its joining pairs. Hence we just
+	 * check existence of partition scheme for the joining relations and let
+	 * try_partitionwise_join() handle the rest for each of the joining pairs
+	 * of this join relation.
 	 */
 	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
@@ -1653,6 +1663,8 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 								jointype, restrictlist))
 	{
 		Assert(!IS_PARTITIONED_REL(joinrel));
+		/* Join is not partitioned. */
+		joinrel->nparts = 0;
 		return;
 	}
 
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index ede16cba15..d5f6eac4be 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -69,13 +69,23 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
-/* Per-partitioned-relation data for merge_list_bounds()/merge_range_bounds() */
+/*
+ * A mapping between partitions of a joining relation and the partitions of the
+ * join (a.k.a merged partitions.).
+ */
 typedef struct PartitionMap
 {
 	int			nparts;			/* number of partitions */
-	int		   *merged_indexes;	/* indexes of merged partitions */
-	bool	   *merged;			/* flags to indicate whether partitions are
-								 * merged with non-dummy partitions */
+	int		   *merged_indexes;	/* ith entry in this array gives the index of
+								 * merged partition to which ith partition
+								 * (of relation for which this map is
+								 * maintained) is mapped. */
+	bool	   *matches_non_dummy;	/* ith entry in the array indicates
+									 * whether the ith partition (of an
+									 * outer relation) matches non-dummy
+									 * partition of the other joining
+									 * relation.
+									 */
 	bool		did_remapping;	/* did we re-map partitions? */
 	int		   *old_indexes;	/* old indexes of merged partitions if
 								 * did_remapping */
@@ -1086,6 +1096,38 @@ partition_bounds_merge(int partnatts,
  * merge_list_bounds
  *		Create the partition bounds for a join relation between list
  *		partitioned tables, if possible
+ *
+ * For each list item from either joining relations the partitions from both
+ * the sides containing that list item form a pair of matching partitions.  The
+ * partition resulting from joining these two partitions will contain that list
+ * item in its bounds in the join. If a list item from the inner side of the
+ * join is missing from the outer side the rows with that list item as the
+ * partition key do not appear in the join and hence that list item does not
+ * appear in the bounds of any of the partition of the join. Corresponding
+ * partition from the inner side may not appear in the join if none of its list
+ * items appear in the other relation. Even if a list item from the outer side
+ * of the join is missing from the inner side, the corresponding rows and hence
+ * the partition appear in the join. So these list items are included in the
+ * partition bounds of the join.
+ *
+ * If a list item from one relation doesn't appear in the partition bounds of
+ * the other relation explicitly, it may be covered by the default partition of
+ * the other relation. In such a case, the corresponding partition from the
+ * first relation forms a matching pair with the default partition from the
+ * other relation.
+ *
+ * Since the list items appear in the ascending order, an algorithm similar to
+ * merge join is used to find the matching partitions and compute the list
+ * items that will be part of the join.
+ *
+ * If there are multiple partitions from one side matching a given partition on
+ * the other side, the algorithm bails out since we do not have machinary for
+ * joining one partition with mulitple partitions. It might happen that any of
+ * the list items of a partition from the outer relation do not appear in the
+ * inner relation and there is no default partition in the inner relation. Such
+ * a partition from the outer side will have no matching partition on the inner
+ * side. The algorithm will bail out in such a case since we do not have a
+ * mechanism to perform a join with a non-existing relation.
  */
 static PartitionBoundInfo
 merge_list_bounds(FmgrInfo *partsupfunc, Oid *partcollation,
@@ -1381,6 +1423,35 @@ cleanup:
  * merge_range_bounds
  *		Create the partition bounds for a join relation between range
  *		partitioned tables, if possible
+ *
+ * For each range from either joining relations the partitions from both the
+ * sides containing that range form a pair of matching partitions.  The
+ * partition resulting from joining these two partitions will cover the range
+ * produced by merging the ranges (See get_merged_range_bounds()) of the
+ * corresponding partitions. If a range from the inner side of the join is
+ * missing from the outer side the rows in that range do not appear in the join
+ * and hence that range does not appears in the join. Even if a range from the
+ * outer side of the join is missing from the inner side, the corresponding
+ * rows and hence the range appears in the join.
+ *
+ * If a range from one relation doesn't appear in the other relation
+ * explicitly, it may be covered by the default partition of the other
+ * relation. In such a case, the corresponding partition from the first
+ * relation forms a matching pair with the default partition from the other
+ * relation.
+ *
+ * Since the ranges appear in the ascending order, an algorithm similar to
+ * merge join is used to find the matching partitions and compute the ranges of
+ * partitions of the join.
+ *
+ * If there are multiple partitions from one side matching a given partition on
+ * the other side, the algorithm bails out since we do not have machinary for
+ * joining one partition with mulitple partitions. It might happen that any of
+ * the ranges of a partition from the outer relation do not appear in the inner
+ * relation and there is no default partition in the inner relation. Such a
+ * partition from the outer side will have no matching partition on the inner
+ * side. The algorithm will bail out in such a case since we do not have a
+ * mechanism to perform a join with a non-existing relation.
  */
 static PartitionBoundInfo
 merge_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
@@ -1490,10 +1561,10 @@ merge_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
 			/* Both partitions should not have been merged yet. */
 			Assert(outer_index >= 0);
 			Assert(outer_map.merged_indexes[outer_index] == -1 &&
-				   outer_map.merged[outer_index] == false);
+				   outer_map.matches_non_dummy[outer_index] == false);
 			Assert(inner_index >= 0);
 			Assert(inner_map.merged_indexes[inner_index] == -1 &&
-				   inner_map.merged[inner_index] == false);
+				   inner_map.matches_non_dummy[inner_index] == false);
 
 			/*
 			 * Get the index of the merged partition.  Both partitions aren't
@@ -1555,7 +1626,7 @@ merge_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
 			/* The outer partition should not have been merged yet. */
 			Assert(outer_index >= 0);
 			Assert(outer_map.merged_indexes[outer_index] == -1 &&
-				   outer_map.merged[outer_index] == false);
+				   outer_map.matches_non_dummy[outer_index] == false);
 
 			/*
 			 * If the inner side has the default partition, or this is an outer
@@ -1592,7 +1663,7 @@ merge_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
 			/* The inner partition should not have been merged yet. */
 			Assert(inner_index >= 0);
 			Assert(inner_map.merged_indexes[inner_index] == -1 &&
-				   inner_map.merged[inner_index] == false);
+				   inner_map.matches_non_dummy[inner_index] == false);
 
 			/*
 			 * If the outer side has the default partition, or this is a FULL
@@ -1695,13 +1766,13 @@ init_partition_map(RelOptInfo *rel, PartitionMap *map)
 
 	map->nparts = nparts;
 	map->merged_indexes = (int *) palloc(sizeof(int) * nparts);
-	map->merged = (bool *) palloc(sizeof(bool) * nparts);
+	map->matches_non_dummy = (bool *) palloc(sizeof(bool) * nparts);
 	map->did_remapping = false;
 	map->old_indexes = (int *) palloc(sizeof(int) * nparts);
 	for (i = 0; i < nparts; i++)
 	{
 		map->merged_indexes[i] = map->old_indexes[i] = -1;
-		map->merged[i] = false;
+		map->matches_non_dummy[i] = false;
 	}
 }
 
@@ -1712,7 +1783,7 @@ static void
 free_partition_map(PartitionMap *map)
 {
 	pfree(map->merged_indexes);
-	pfree(map->merged);
+	pfree(map->matches_non_dummy);
 	pfree(map->old_indexes);
 }
 
@@ -1744,15 +1815,15 @@ merge_matching_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
 {
 	int 		outer_merged_index;
 	int 		inner_merged_index;
-	bool 		outer_merged;
-	bool 		inner_merged;
+	bool 		outer_matches_non_dummy;
+	bool 		inner_matches_non_dummy;
 
 	Assert(outer_index >= 0 && outer_index < outer_map->nparts);
 	outer_merged_index = outer_map->merged_indexes[outer_index];
-	outer_merged = outer_map->merged[outer_index];
+	outer_matches_non_dummy = outer_map->matches_non_dummy[outer_index];
 	Assert(inner_index >= 0 && inner_index < inner_map->nparts);
 	inner_merged_index = inner_map->merged_indexes[inner_index];
-	inner_merged = inner_map->merged[inner_index];
+	inner_matches_non_dummy = inner_map->matches_non_dummy[inner_index];
 
 	/*
 	 * Handle cases where we have already assigned a merged partition to each
@@ -1769,11 +1840,11 @@ merge_matching_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
 		 */
 		if (outer_merged_index == inner_merged_index)
 		{
-			Assert(outer_merged);
-			Assert(inner_merged);
+			Assert(outer_matches_non_dummy);
+			Assert(inner_matches_non_dummy);
 			return outer_merged_index;
 		}
-		if (!outer_merged && !inner_merged)
+		if (!outer_matches_non_dummy && !inner_matches_non_dummy)
 		{
 			/*
 			 * This can only happen for a list-partitioning case.  We re-map
@@ -1784,18 +1855,18 @@ merge_matching_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
 			 */
 			if (outer_merged_index < inner_merged_index)
 			{
-				outer_map->merged[outer_index] = true;
+				outer_map->matches_non_dummy[outer_index] = true;
 				inner_map->merged_indexes[inner_index] = outer_merged_index;
-				inner_map->merged[inner_index] = true;
+				inner_map->matches_non_dummy[inner_index] = true;
 				inner_map->did_remapping = true;
 				inner_map->old_indexes[inner_index] = inner_merged_index;
 				return outer_merged_index;
 			}
 			else
 			{
-				inner_map->merged[inner_index] = true;
+				inner_map->matches_non_dummy[inner_index] = true;
 				outer_map->merged_indexes[outer_index] = inner_merged_index;
-				outer_map->merged[outer_index] = true;
+				outer_map->matches_non_dummy[outer_index] = true;
 				outer_map->did_remapping = true;
 				outer_map->old_indexes[outer_index] = outer_merged_index;
 				return inner_merged_index;
@@ -1817,31 +1888,31 @@ merge_matching_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
 	{
 		int 		merged_index = *next_index;
 
-		Assert(!outer_merged);
-		Assert(!inner_merged);
+		Assert(!outer_matches_non_dummy);
+		Assert(!inner_matches_non_dummy);
 		outer_map->merged_indexes[outer_index] = merged_index;
-		outer_map->merged[outer_index] = true;
+		outer_map->matches_non_dummy[outer_index] = true;
 		inner_map->merged_indexes[inner_index] = merged_index;
-		inner_map->merged[inner_index] = true;
+		inner_map->matches_non_dummy[inner_index] = true;
 		*next_index = *next_index + 1;
 		return merged_index;
 	}
-	if (outer_merged_index >= 0 && !outer_map->merged[outer_index])
+	if (outer_merged_index >= 0 && !outer_map->matches_non_dummy[outer_index])
 	{
 		Assert(inner_merged_index == -1);
-		Assert(!inner_merged);
+		Assert(!inner_matches_non_dummy);
 		inner_map->merged_indexes[inner_index] = outer_merged_index;
-		inner_map->merged[inner_index] = true;
-		outer_map->merged[outer_index] = true;
+		inner_map->matches_non_dummy[inner_index] = true;
+		outer_map->matches_non_dummy[outer_index] = true;
 		return outer_merged_index;
 	}
-	if (inner_merged_index >= 0 && !inner_map->merged[inner_index])
+	if (inner_merged_index >= 0 && !inner_map->matches_non_dummy[inner_index])
 	{
 		Assert(outer_merged_index == -1);
-		Assert(!outer_merged);
+		Assert(!outer_matches_non_dummy);
 		outer_map->merged_indexes[outer_index] = inner_merged_index;
-		outer_map->merged[outer_index] = true;
-		inner_map->merged[inner_index] = true;
+		outer_map->matches_non_dummy[outer_index] = true;
+		inner_map->matches_non_dummy[inner_index] = true;
 		return inner_merged_index;
 	}
 	return -1;
@@ -2237,7 +2308,7 @@ merge_default_partitions(PartitionMap *outer_map,
  *
  * Note: The caller assumes that the given partition doesn't have a non-dummy
  * matching partition on the other side, but if the given partition finds the
- * matchig partition later, we will adjust the assignment.
+ * matching partition later, we will adjust the assignment.
  */
 static int
 merge_partition_with_dummy(PartitionMap *map, int index, int *next_index)
@@ -2246,7 +2317,7 @@ merge_partition_with_dummy(PartitionMap *map, int index, int *next_index)
 
 	Assert(index >= 0 && index < map->nparts);
 	Assert(map->merged_indexes[index] == -1);
-	Assert(!map->merged[index]);
+	Assert(!map->matches_non_dummy[index]);
 	map->merged_indexes[index] = merged_index;
 	/* Leave the merged flag alone! */
 	*next_index = *next_index + 1;
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index 622ea2bf63..2c6935d81d 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -723,8 +723,8 @@ typedef struct RelOptInfo
 	int			nparts;			/* number of partitions; 0 = not partitioned;
 								 * -1 = not yet set */
 	struct PartitionBoundInfoData *boundinfo;	/* Partition bounds */
-	bool		merged;			/* true if partition bounds were created by
-								 * partition_bounds_merge() */
+	bool		partbounds_merged;	/* true if partition bounds were created
+									 * by partition_bounds_merge() */
 	List	   *partition_qual; /* partition constraint */
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of partitions,
 									 * stored in the same order of bounds */
@@ -754,6 +754,21 @@ typedef struct RelOptInfo
 	((rel)->part_scheme && (rel)->boundinfo && (rel)->nparts > 0 && \
 	 (rel)->part_rels && (rel)->partexprs && (rel)->nullable_partexprs)
 
+/*
+ * Is given join relation deemed to be unpartitioned certainly (by
+ * build_joinrel_partition_info() and try_partitionwise_join())? A join is
+ * considered to be partitioned if it can be computed using partitionwise join
+ * technique. If that's possible, we will have both partition scheme and number
+ * of partitions set in the join relation. If the partition schemes of the
+ * joining relations match but the partition bounds can not be merged, number
+ * of partitions will be set to 0.
+ *
+ * The macro is expected to be called only on a join relation, but we don't
+ * check that explicitly here.
+ */
+#define IS_JOINREL_NOT_PARTITITIONED(joinrel) \
+	((joinrel)->part_scheme == NULL || (joinrel)->nparts == 0)
+
 /*
  * IndexOptInfo
  *		Per-index information for planning/optimization
-- 
2.17.1

#128Ashutosh Bapat
ashutosh.bapat@2ndquadrant.com
In reply to: Tomas Vondra (#125)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Thu, 26 Mar 2020 at 00:35, Tomas Vondra <tomas.vondra@2ndquadrant.com>
wrote:

Hi,

I've started reviewing the patch a couple days ago. I haven't done any
extensive testing, but I do have a bunch of initial comments that I can
share now.

1) I wonder if this needs to update src/backend/optimizer/README, which
does have a section about partitionwise joins. It seems formulated in a
way that that probably covers even this more advanced algorithm, but
maybe it should mention handling of default partitions etc.?

Done. Please check the wording. It might need some word smithy.

There certainly needs to be some description of the algorithm somewhere,
either in a README or before a suitable function. It doesn't have to be
particularly detailed, a rough outline of the matching would be enough,
so that readers don't have to rebuild the knowledge from pieces
scattered around various comments.

The algorithm for list and range partitioned tables is slightly different.
So, I have added separate prologue to each list_merge_bounds() and
range_merge_bounds(). Please check if that serves the purpose.

2) Do we need another GUC enabling this more complex algorithm? In PG11
the partitionwise join is disabled by default, on the grounds that it's
expensive and not worth it by default. How much more expensive is this?
Maybe it makes sense to allow enabling only the "simple" approach?

We have reduced the complexity of merging bounds quite a bit so this
shouldn't be costly. Further more we handle the usual case of equal bounds
quickly using the merge flag so most of the cases should be fine. It's only
when two partitioned tables with same partition scheme are joined but do
not have merge-able bounds that this algorithm would not yield useful
result - but that would be rare in the field. enable_partitionwise_join =
false should take care of such scenarios easily. I am not in favour of
adding another GUC which we set to false by default and then take another
few releases to make it true by default.

3) This comment in try_partitionwise_join is now incorrect, because the
condition may be true even for partitioned tables with (nparts == 0).

/* Nothing to do, if the join relation is not partitioned. */
if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
return;

If part_scheme = NULL, npart should be 0, fixed that in
build_joinrel_partition_info(). If partscheme != NULL but bounds can not be
merged, nparts = 0. So this condition is correct. Encapsulated in a macro
IS_JOINREL_NOT_PARTITITIONED(). and added comments for the macro. Given
that the macro is used exactly at one place, it may not be necessary to
define it but it looks *nice*.

Moreover, the condition used to be

if (!IS_PARTITIONED_REL(joinrel))
return;

which is way more readable. I think it's net negative to replace these
"nice" macros with clear meaning with complex conditions. If needed, we
can invent new macros. There are many other places where the patch
replaces macros with less readable conditions.

The only other place where we have replaced a *nice* macro is in
build_joinrel_partition_info(). But I think it's a legit replacement. I
have added a comment there.

4) I'm a bit puzzled how we could get here with non-partitioned rels?

/*
* We can not perform partitionwise join if either of the joining
relations
* is not partitioned.
*/
if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
return;

See the comment I have added in build_joinrel_partition_info(). Not all
joining pairs for a given relation are partitioned.

5) I find the "merged" flag in RelOptInfo rather unclear, because it
does not clearly indicate what was merged. Maybe something like
partbounds_merged would be better?

Done.

6) The try_partitionwise_join function is getting a bit too long and
harder to understand. The whole block in

if (joinrel->nparts == -1)
{
...
}

seems rather well isolated, so I propose to move it to a separate
function responsible only for the merging. We can simply call it on the
joinrel, and make it return right away if (joinrel->nparts == -1).

Looks like you have already taken care of this one in one of your patches.

7) I'd suggest not to reference exact functions in comments unless
abolutely necessary, because it's harder to maintain and it does not
really explain purpose of the struct/code. E.g. consider this:

/* Per-partitioned-relation data for
merge_list_bounds()/merge_range_bounds() */
typedef struct PartitionMap
{ ... }

Why does it matter where is the struct used? That's pretty trivial to
find using 'git grep' or something. Instead the comment should explain
the purpose of the struct.

Adjusted names and comments a bit.

--
Best Wishes,
Ashutosh

#129Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Ashutosh Bapat (#127)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Hi Tomas and Ashutosh,

On Thu, Apr 2, 2020 at 1:51 AM Ashutosh Bapat
<ashutosh.bapat@2ndquadrant.com> wrote:

On Thu, 26 Mar 2020 at 05:47, Tomas Vondra <tomas.vondra@2ndquadrant.com> wrote:

three more comments after eye-balling the code for a bit longer.

1) The patch probably needs to tweak config.sgml which says this about
the enable_partitionwise_join GUC:

.. Partitionwise join currently applies only when the join conditions
include all the partition keys, which must be of the same data type
and have exactly matching sets of child partitions. ..

Done. Actually this wasn't updated when partition pruning was introduced, which could cause a partitionwise join to be not used even when those conditions were met. Similarly when a query involved whole row reference. It's hard to explain all the conditions under which partitionwise join technique will be used. But I have tried to keep it easy to understand.

Which is probably incorrect, because the point of this patch is not to
require exact match of the partitions, right?

2) Do we really need the 'merged' flag in try_partitionwise_join? Can't
we simply use the joinrel->merged flag directly? ISTM the we always
start with joinrel->merged=false, and then only ever set it to true in
some cases. I've tried doing that, per the attached 0002 patch. The
regression tests seem to work fine.

Thanks. I just added a small prologue to compute_partition_bounds().

I noticed this because I've tried moving part of the function into a
separate function, and removing the variable makes that simpler.

The patch also does a couple additional minor tweaks.

3) I think the for nparts comment is somewhat misleading:

int nparts; /* number of partitions; 0 = not partitioned;
* -1 = not yet set */

which says that nparts=0 means not partitioned. But then there are
conditions like this:

/* Nothing to do, if the join relation is not partitioned. */
if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
return;

which (ignoring the obsolete comment) seems to say we can have nparts==0
even for partitioned tables, no?

See my previous mail.

Anyway, attached is the original patch (0001) and two patches with
proposed changes. 0002 removes the "merged" flag as explained in (2),
0003 splits the try_partitionwise_join() function into two parts.

I'm saying these changes have to happen and it's a bit crude (and it
might be a bit of a bikeshedding).

I have added 0005 with the changes I described in this as well as the previous mail. 0004 is just some white space fixes.

Thanks for the comments, Tomas! Thanks for the patch, Ashutosh! I
will look at the patch.

Best regards,
Etsuro Fujita

#130Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Ashutosh Bapat (#128)
2 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Hi,

On Thu, Apr 2, 2020 at 2:12 AM Ashutosh Bapat
<ashutosh.bapat@2ndquadrant.com> wrote:

On Thu, 26 Mar 2020 at 00:35, Tomas Vondra <tomas.vondra@2ndquadrant.com> wrote:

I've started reviewing the patch a couple days ago. I haven't done any
extensive testing, but I do have a bunch of initial comments that I can
share now.

1) I wonder if this needs to update src/backend/optimizer/README, which
does have a section about partitionwise joins. It seems formulated in a
way that that probably covers even this more advanced algorithm, but
maybe it should mention handling of default partitions etc.?

Done. Please check the wording. It might need some word smithy.

You heavily changed the existing documentation about PWJ, but I don't
think we really need to do so. Also, IMO I think the description
about default partitions you added is needed in README. I think it
would be better to put such a description in source files. How about
something like the attached, instead? I wrote part of this based on
the commit message in the original versions of the patch you posted.

There certainly needs to be some description of the algorithm somewhere,
either in a README or before a suitable function. It doesn't have to be
particularly detailed, a rough outline of the matching would be enough,
so that readers don't have to rebuild the knowledge from pieces
scattered around various comments.

The algorithm for list and range partitioned tables is slightly different. So, I have added separate prologue to each list_merge_bounds() and range_merge_bounds(). Please check if that serves the purpose.

Too detailed to me. In this:

+ * If there are multiple partitions from one side matching a given partition on
+ * the other side, the algorithm bails out since we do not have machinary for
+ * joining one partition with mulitple partitions. It might happen that any of
+ * the list items of a partition from the outer relation do not appear in the
+ * inner relation and there is no default partition in the inner relation. Such
+ * a partition from the outer side will have no matching partition on the inner
+ * side. The algorithm will bail out in such a case since we do not have a
+ * mechanism to perform a join with a non-existing relation.

I don't think the last comment is correct; that would apply to the old
versions of this function IIRC, but not to the latest version. How
about something much simpler like the attached, instead?

2) Do we need another GUC enabling this more complex algorithm? In PG11
the partitionwise join is disabled by default, on the grounds that it's
expensive and not worth it by default. How much more expensive is this?
Maybe it makes sense to allow enabling only the "simple" approach?

We have reduced the complexity of merging bounds quite a bit so this shouldn't be costly. Further more we handle the usual case of equal bounds quickly using the merge flag so most of the cases should be fine. It's only when two partitioned tables with same partition scheme are joined but do not have merge-able bounds that this algorithm would not yield useful result - but that would be rare in the field. enable_partitionwise_join = false should take care of such scenarios easily. I am not in favour of adding another GUC which we set to false by default and then take another few releases to make it true by default.

I agree with Ashutosh.

3) This comment in try_partitionwise_join is now incorrect, because the
condition may be true even for partitioned tables with (nparts == 0).

/* Nothing to do, if the join relation is not partitioned. */
if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
return;

If part_scheme = NULL, npart should be 0, fixed that in build_joinrel_partition_info(). If partscheme != NULL but bounds can not be merged, nparts = 0. So this condition is correct. Encapsulated in a macro IS_JOINREL_NOT_PARTITITIONED(). and added comments for the macro. Given that the macro is used exactly at one place, it may not be necessary to define it but it looks *nice*.

I don't think it would be a good idea to add such a macro for only one place.

Moreover, the condition used to be

if (!IS_PARTITIONED_REL(joinrel))
return;

which is way more readable. I think it's net negative to replace these
"nice" macros with clear meaning with complex conditions. If needed, we
can invent new macros. There are many other places where the patch
replaces macros with less readable conditions.

The only other place where we have replaced a *nice* macro is in build_joinrel_partition_info(). But I think it's a legit replacement. I have added a comment there.

IIUC, this is the existing issue, so I think it would be better to
leave this for another improvement patch.

6) The try_partitionwise_join function is getting a bit too long and
harder to understand. The whole block in

if (joinrel->nparts == -1)
{
...
}

seems rather well isolated, so I propose to move it to a separate
function responsible only for the merging. We can simply call it on the
joinrel, and make it return right away if (joinrel->nparts == -1).

That's a good idea, so +1.

7) I'd suggest not to reference exact functions in comments unless
abolutely necessary, because it's harder to maintain and it does not
really explain purpose of the struct/code. E.g. consider this:

/* Per-partitioned-relation data for merge_list_bounds()/merge_range_bounds() */
typedef struct PartitionMap
{ ... }

Why does it matter where is the struct used? That's pretty trivial to
find using 'git grep' or something. Instead the comment should explain
the purpose of the struct.

Adjusted names and comments a bit.

I modified the comments a bit further. I don't think we need to
change the name of a variable, so I kept it as is.

On Thu, Apr 2, 2020 at 1:51 AM Ashutosh Bapat
<ashutosh.bapat@2ndquadrant.com> wrote:

On Thu, 26 Mar 2020 at 05:47, Tomas Vondra <tomas.vondra@2ndquadrant.com> wrote:

three more comments after eye-balling the code for a bit longer.

1) The patch probably needs to tweak config.sgml which says this about
the enable_partitionwise_join GUC:

.. Partitionwise join currently applies only when the join conditions
include all the partition keys, which must be of the same data type
and have exactly matching sets of child partitions. ..

Done. Actually this wasn't updated when partition pruning was introduced, which could cause a partitionwise join to be not used even when those conditions were met. Similarly when a query involved whole row reference. It's hard to explain all the conditions under which partitionwise join technique will be used. But I have tried to keep it easy to understand.

IMO I think your words "there is exactly one pair of matching
partitions." is a bit misleading, because that sounds like that PWJ
doesn't allow multiply-segmented join. How about s/exact
matching/one-to-one matching/ in the existing documentation, instead?

2) Do we really need the 'merged' flag in try_partitionwise_join? Can't
we simply use the joinrel->merged flag directly? ISTM the we always
start with joinrel->merged=false, and then only ever set it to true in
some cases. I've tried doing that, per the attached 0002 patch. The
regression tests seem to work fine.

I introduced the flag "merged", because I thought it would be verbose
to write "joiners->merged" in many places, but I think removing that
would make the code simpler, so +1 for that change.

Thanks. I just added a small prologue to compute_partition_bounds().

I tweaked that a bit.

I noticed this because I've tried moving part of the function into a
separate function, and removing the variable makes that simpler.

The patch also does a couple additional minor tweaks.

    /*
     * Currently, this function is called only from
try_partitionwise_join(),
     * so the join type should be INNER, LEFT, FULL, SEMI, or ANTI.
+    *
+    * XXX Maybe an assert would be more appropriate? Or maybe just
+    * bail out by returning NULL? Not sure.
     */
    if (jointype != JOIN_INNER && jointype != JOIN_LEFT &&
        jointype != JOIN_FULL && jointype != JOIN_SEMI &&
        jointype != JOIN_ANTI)

I kept this as originally proposed, but I agree that an assertion
would be more appropriate, because it would save cycles a bit in a
non-assertion-enabled build. I modified this as such. I modified the
test below this as well.

3) I think the for nparts comment is somewhat misleading:

int nparts; /* number of partitions; 0 = not partitioned;
* -1 = not yet set */

which says that nparts=0 means not partitioned. But then there are
conditions like this:

/* Nothing to do, if the join relation is not partitioned. */
if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
return;

which (ignoring the obsolete comment) seems to say we can have nparts==0
even for partitioned tables, no?

Yeah, I think I was a bit hasty. I fixed this.

Attached is the original patch (0001) and one patch (0002) with
changes including those by Tomas and Ashutosh. In the 0002 patch I
fixed my many typos as well. :-( Many of them were found by Justin
Pryzby. One of them was found by Ashutosh. Thank you!

Anyway, attached is the original patch (0001) and two patches with
proposed changes. 0002 removes the "merged" flag as explained in (2),
0003 splits the try_partitionwise_join() function into two parts.

Thanks for the patches, Tomas!

I have added 0005 with the changes I described in this as well as the previous mail. 0004 is just some white space fixes.

Thanks for the patches, Ashutosh! I fixed the white space issues.

Best regards,
Etsuro Fujita

Attachments:

0001-Improve-partition-matching-for-partitionwise-join.patchapplication/octet-stream; name=0001-Improve-partition-matching-for-partitionwise-join.patchDownload
From 650f614cef5c098709b955996ee982a0027c3c24 Mon Sep 17 00:00:00 2001
From: Etsuro Fujita <etsuro@EtsuronoMacBook-Air.local>
Date: Tue, 31 Mar 2020 18:06:47 +0900
Subject: [PATCH 1/2] Improve partition matching for partitionwise join.

---
 src/backend/nodes/outfuncs.c                 |    2 +
 src/backend/optimizer/path/joinrels.c        |  242 ++-
 src/backend/optimizer/util/inherit.c         |    2 +
 src/backend/optimizer/util/relnode.c         |   43 +-
 src/backend/partitioning/partbounds.c        | 1838 ++++++++++++++++++
 src/include/nodes/pathnodes.h                |    8 +-
 src/include/partitioning/partbounds.h        |    9 +
 src/test/regress/expected/partition_join.out | 2608 +++++++++++++++++++++++++-
 src/test/regress/sql/partition_join.sql      |  639 ++++++-
 9 files changed, 5315 insertions(+), 76 deletions(-)

diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index eb168ffd6d..55dfa27157 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -2288,6 +2288,8 @@ _outRelOptInfo(StringInfo str, const RelOptInfo *node)
 	WRITE_BOOL_FIELD(has_eclass_joins);
 	WRITE_BOOL_FIELD(consider_partitionwise_join);
 	WRITE_BITMAPSET_FIELD(top_parent_relids);
+	WRITE_BOOL_FIELD(merged);
+	WRITE_BITMAPSET_FIELD(all_partrels);
 	WRITE_NODE_FIELD(partitioned_child_rels);
 }
 
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index a21c295b99..530ebed245 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -48,6 +48,9 @@ static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 												Relids left_relids, Relids right_relids);
 static int	match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel,
 										 bool strict_op);
+static void get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2);
 
 
 /*
@@ -1357,25 +1360,30 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	bool		merged = false;
+	List	   *parts1 = NIL;
+	List	   *parts2 = NIL;
+	ListCell   *lcr1 = NULL;
+	ListCell   *lcr2 = NULL;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partitionwise join if either of the joining relations
+	 * is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1390,34 +1398,104 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		   joinrel->part_scheme == rel2->part_scheme);
 
 	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
+	 * If we don't have the partition bounds for the join rel yet, try to
+	 * compute those along with pairs of partitions to be joined.
 	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	if (joinrel->nparts == -1)
+	{
+		PartitionScheme part_scheme = joinrel->part_scheme;
+		PartitionBoundInfo boundinfo = NULL;
+		int			nparts = 0;
+
+		Assert(joinrel->boundinfo == NULL);
+		Assert(joinrel->part_rels == NULL);
+
+		/*
+		 * See if the partition bounds for inputs are exactly the same, in
+		 * which case we don't need to work hard: the join rel have the same
+		 * partition bounds as inputs, and the partitions with the same
+		 * cardinal positions form the pairs.
+		 *
+		 * Note: even in cases where one or both inputs have merged bounds,
+		 * it would be possible for both the bounds to be exactly the same, but
+		 * it seems unlikely to be worth the cycles to check.
+		 */
+		if (!rel1->merged &&
+			!rel2->merged &&
+			rel1->nparts == rel2->nparts &&
+			partition_bounds_equal(part_scheme->partnatts,
+								   part_scheme->parttyplen,
+								   part_scheme->parttypbyval,
+								   rel1->boundinfo, rel2->boundinfo))
+		{
+			boundinfo = rel1->boundinfo;
+			nparts = rel1->nparts;
+		}
+		else
+		{
+			/* Try merging the partition bounds for inputs. */
+			boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											   part_scheme->partsupfunc,
+											   part_scheme->partcollation,
+											   rel1, rel2,
+											   parent_sjinfo->jointype,
+											   &parts1, &parts2);
+			if (boundinfo == NULL)
+			{
+				joinrel->nparts = 0;
+				return;
+			}
+			nparts = list_length(parts1);
+			merged = true;
+		}
+
+		Assert(nparts > 0);
+		joinrel->boundinfo = boundinfo;
+		joinrel->merged = merged;
+		joinrel->nparts = nparts;
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+	}
+	else
+	{
+		Assert(joinrel->nparts > 0);
+		Assert(joinrel->boundinfo);
+		Assert(joinrel->part_rels);
+
+		/*
+		 * If the join rel's merged flag is true, it means inputs are not
+		 * guaranteed to have the same partition bounds, therefore we can't
+		 * assume that the partitions at the same cardinal positions form the
+		 * pairs; let get_matching_part_pairs() generate the pairs.  Otherwise,
+		 * nothing to do since we can assume that.
+		 */
+		if (joinrel->merged)
+		{
+			get_matching_part_pairs(root, joinrel, rel1, rel2,
+									&parts1, &parts2);
+			Assert(list_length(parts1) == joinrel->nparts);
+			Assert(list_length(parts2) == joinrel->nparts);
+			merged = true;
+		}
+	}
 
-	nparts = joinrel->nparts;
+	if (merged)
+	{
+		lcr1 = list_head(parts1);
+		lcr2 = list_head(parts2);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
-		bool		rel1_empty = (child_rel1 == NULL ||
-								  IS_DUMMY_REL(child_rel1));
-		bool		rel2_empty = (child_rel2 == NULL ||
-								  IS_DUMMY_REL(child_rel2));
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		bool		rel1_empty;
+		bool		rel2_empty;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1425,6 +1503,22 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		if (merged)
+		{
+			child_rel1 = lfirst_node(RelOptInfo, lcr1);
+			child_rel2 = lfirst_node(RelOptInfo, lcr2);
+			lcr1 = lnext(parts1, lcr1);
+			lcr2 = lnext(parts2, lcr2);
+		}
+		else
+		{
+			child_rel1 = rel1->part_rels[cnt_parts];
+			child_rel2 = rel2->part_rels[cnt_parts];
+		}
+
+		rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1));
+		rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2));
+
 		/*
 		 * Check for cases where we can prove that this segment of the join
 		 * returns no rows, due to one or both inputs being empty (including
@@ -1522,6 +1616,8 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 												 child_sjinfo,
 												 child_sjinfo->jointype);
 			joinrel->part_rels[cnt_parts] = child_joinrel;
+			joinrel->all_partrels = bms_add_members(joinrel->all_partrels,
+													child_joinrel->relids);
 		}
 
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
@@ -1738,3 +1834,99 @@ match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op)
 
 	return -1;
 }
+
+/*
+ * get_matching_part_pairs
+ *		Generate pairs of partitions to be joined from the two inputs
+ */
+static void
+get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2)
+{
+	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
+	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
+	int 		cnt_parts;
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
+	{
+		RelOptInfo *child_joinrel = joinrel->part_rels[cnt_parts];
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		Relids		child_relids1;
+		Relids		child_relids2;
+
+		/*
+		 * If this segment of the join is empty, it means that this segment
+		 * was ignored when previously creating child-join paths for it in
+		 * try_partitionwise_join() as it would not contribute to the join
+		 * result, due to one or both inputs being empty; add NULL to each of
+		 * the given lists so that this segment will be ignored again in that
+		 * function.
+		 */
+		if (!child_joinrel)
+		{
+			*parts1 = lappend(*parts1, NULL);
+			*parts2 = lappend(*parts2, NULL);
+			continue;
+		}
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel1 side.
+		 */
+		child_relids1 = bms_intersect(child_joinrel->relids,
+									  rel1->all_partrels);
+		Assert(bms_num_members(child_relids1) == bms_num_members(rel1->relids));
+
+		/*
+		 * Get a child rel for rel1 with the relids.  Note that we should have
+		 * the child rel even if rel1 is a join rel, because in that case the
+		 * partitions specified in the relids would have matching/overlapping
+		 * boundaries, so the specified partitions should be considered as ones
+		 * to be joined when planning partitionwise joins of rel1, meaning that
+		 * the child rel would have been built by the time we get here.
+		 */
+		if (rel1_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids1);
+
+			child_rel1 = find_base_rel(root, varno);
+		}
+		else
+			child_rel1 = find_join_rel(root, child_relids1);
+		Assert(child_rel1);
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel2 side.
+		 */
+		child_relids2 = bms_intersect(child_joinrel->relids,
+									  rel2->all_partrels);
+		Assert(bms_num_members(child_relids2) == bms_num_members(rel2->relids));
+
+		/*
+		 * Get a child rel for rel2 with the relids.  See above comments.
+		 */
+		if (rel2_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids2);
+
+			child_rel2 = find_base_rel(root, varno);
+		}
+		else
+			child_rel2 = find_join_rel(root, child_relids2);
+		Assert(child_rel2);
+
+		/*
+		 * The join of rel1 and rel2 is legal, so is the join of the child
+		 * rels obtained above; add them to the given lists as a join pair
+		 * producing this join segment.
+		 */
+		*parts1 = lappend(*parts1, child_rel1);
+		*parts2 = lappend(*parts2, child_rel2);
+	}
+}
diff --git a/src/backend/optimizer/util/inherit.c b/src/backend/optimizer/util/inherit.c
index 7db67fdf34..3132fd35a5 100644
--- a/src/backend/optimizer/util/inherit.c
+++ b/src/backend/optimizer/util/inherit.c
@@ -376,6 +376,8 @@ expand_partitioned_rtentry(PlannerInfo *root, RelOptInfo *relinfo,
 		/* Create the otherrel RelOptInfo too. */
 		childrelinfo = build_simple_rel(root, childRTindex, relinfo);
 		relinfo->part_rels[i] = childrelinfo;
+		relinfo->all_partrels = bms_add_members(relinfo->all_partrels,
+												childrelinfo->relids);
 
 		/* If this child is itself partitioned, recurse */
 		if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 374f93890b..0e4944ac8e 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -240,10 +240,12 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->has_eclass_joins = false;
 	rel->consider_partitionwise_join = false;	/* might get changed later */
 	rel->part_scheme = NULL;
-	rel->nparts = 0;
+	rel->nparts = -1;
 	rel->boundinfo = NULL;
+	rel->merged = false;
 	rel->partition_qual = NIL;
 	rel->part_rels = NULL;
+	rel->all_partrels = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->partitioned_child_rels = NIL;
@@ -653,10 +655,12 @@ build_join_rel(PlannerInfo *root,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -829,10 +833,12 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -1639,7 +1645,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1652,24 +1658,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partition scheme, partition bounds, partition key
@@ -1680,18 +1668,17 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 		   !joinrel->boundinfo);
 
 	/*
-	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * If the join relation is partitioned, it use the same partitioning scheme
+	 * as the joining relations.
+	 *
+	 * Note: we calculate the partition bounds, number of partitions, and
+	 * child-join relations of the join relation in try_partitionwise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
 	partnatts = joinrel->part_scheme->partnatts;
 	joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
 	joinrel->nullable_partexprs =
 		(List **) palloc0(sizeof(List *) * partnatts);
-	joinrel->nparts = outer_rel->nparts;
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
 
 	/*
 	 * Set the consider_partitionwise_join flag.
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 4c47f54a57..24dbc2c8f3 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -26,6 +26,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/pathnodes.h"
 #include "parser/parse_coerce.h"
 #include "partitioning/partbounds.h"
 #include "partitioning/partdesc.h"
@@ -68,6 +69,25 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+/* Per-partitioned-relation data for merge_list_bounds()/merge_range_bounds() */
+typedef struct PartitionMap
+{
+	int			nparts;			/* number of partitions */
+	int		   *merged_indexes;	/* indexes of merged partitions */
+	bool	   *merged;			/* flags to indicate whether partitions are
+								 * merged with non-dummy partitions */
+	bool		did_remapping;	/* did we re-map partitions? */
+	int		   *old_indexes;	/* old indexes of merged partitions if
+								 * did_remapping */
+} PartitionMap;
+
+/* Macro for comparing two range bounds */
+#define compare_range_bounds(partnatts, partsupfunc, partcollations, \
+							 bound1, bound2) \
+	(partition_rbound_cmp(partnatts, partsupfunc, partcollations, \
+						  (bound1)->datums, (bound1)->kind, (bound1)->lower, \
+						  bound2))
+
 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);
@@ -79,6 +99,116 @@ static PartitionBoundInfo create_list_bounds(PartitionBoundSpec **boundspecs,
 											 int nparts, PartitionKey key, int **mapping);
 static PartitionBoundInfo create_range_bounds(PartitionBoundSpec **boundspecs,
 											  int nparts, PartitionKey key, int **mapping);
+static PartitionBoundInfo merge_list_bounds(FmgrInfo *partsupfunc,
+											Oid *collations,
+											RelOptInfo *outer_rel,
+											RelOptInfo *inner_rel,
+											JoinType jointype,
+											List **outer_parts,
+											List **inner_parts);
+static PartitionBoundInfo merge_range_bounds(int partnatts,
+											 FmgrInfo *partsupfuncs,
+											 Oid *partcollations,
+											 RelOptInfo *outer_rel,
+											 RelOptInfo *inner_rel,
+											 JoinType jointype,
+											 List **outer_parts,
+											 List **inner_parts);
+static void init_partition_map(RelOptInfo *rel, PartitionMap *map);
+static void free_partition_map(PartitionMap *map);
+static bool is_dummy_partition(RelOptInfo *rel, int part_index);
+static int merge_matching_partitions(PartitionMap *outer_map,
+									 PartitionMap *inner_map,
+									 int outer_part,
+									 int inner_part,
+									 int *next_index);
+static int process_outer_partition(PartitionMap *outer_map,
+								   PartitionMap *inner_map,
+								   bool outer_has_default,
+								   bool inner_has_default,
+								   int outer_index,
+								   int inner_default,
+								   JoinType jointype,
+								   int *next_index,
+								   int *default_index);
+static int process_inner_partition(PartitionMap *outer_map,
+								   PartitionMap *inner_map,
+								   bool outer_has_default,
+								   bool inner_has_default,
+								   int inner_index,
+								   int outer_default,
+								   JoinType jointype,
+								   int *next_index,
+								   int *default_index);
+static void merge_null_partitions(PartitionMap *outer_map,
+								  PartitionMap *inner_map,
+								  bool outer_has_null,
+								  bool inner_has_null,
+								  int outer_null,
+								  int inner_null,
+								  JoinType jointype,
+								  int *next_index,
+								  int *null_index);
+static void merge_default_partitions(PartitionMap *outer_map,
+									 PartitionMap *inner_map,
+									 bool outer_has_default,
+									 bool inner_has_default,
+									 int outer_default,
+									 int inner_default,
+									 JoinType jointype,
+									 int *next_index,
+									 int *default_index);
+static int merge_partition_with_dummy(PartitionMap *map, int index,
+									  int *next_index);
+static void fix_merged_indexes(PartitionMap *outer_map,
+							   PartitionMap *inner_map,
+							   int nmerged, List *merged_indexes);
+static void generate_matching_part_pairs(RelOptInfo *outer_rel,
+										 RelOptInfo *inner_rel,
+										 PartitionMap *outer_map,
+										 PartitionMap *inner_map,
+										 int nmerged,
+										 List **outer_parts,
+										 List **inner_parts);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+														List *merged_datums,
+														List *merged_kinds,
+														List *merged_indexes,
+														int null_index,
+														int default_index);
+static int get_range_partition(RelOptInfo *rel,
+							   PartitionBoundInfo bi,
+							   int *lb_pos,
+							   PartitionRangeBound *lb,
+							   PartitionRangeBound *ub);
+static int get_range_partition_internal(PartitionBoundInfo bi,
+										int *lb_pos,
+										PartitionRangeBound *lb,
+										PartitionRangeBound *ub);
+static bool compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+									 Oid *partcollations,
+									 PartitionRangeBound *outer_lb,
+									 PartitionRangeBound *outer_ub,
+									 PartitionRangeBound *inner_lb,
+									 PartitionRangeBound *inner_ub,
+									 int *lb_cmpval, int *ub_cmpval);
+static void get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+									Oid *partcollations, JoinType jointype,
+									PartitionRangeBound *outer_lb,
+									PartitionRangeBound *outer_ub,
+									PartitionRangeBound *inner_lb,
+									PartitionRangeBound *inner_ub,
+									int	lb_cmpval, int ub_cmpval,
+									PartitionRangeBound *merged_lb,
+									PartitionRangeBound *merged_ub);
+static void add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+									Oid *partcollations,
+									PartitionRangeBound *merged_lb,
+									PartitionRangeBound *merged_ub,
+									int merged_index,
+									List **merged_datums,
+									List **merged_kinds,
+									List **merged_indexes);
 static PartitionRangeBound *make_one_partition_rbound(PartitionKey key, int index,
 													  List *datums, bool lower);
 static int32 partition_hbound_cmp(int modulus1, int remainder1, int modulus2,
@@ -866,6 +996,1714 @@ partition_bounds_copy(PartitionBoundInfo src,
 	return dest;
 }
 
+/*
+ * partition_bounds_merge
+ *
+ * This function builds and returns the partition bounds for a join relation
+ * between input relations, creating two lists of partitions, which are
+ * returned to *outer_parts and *inner_parts respectively.  The lists contain
+ * the same number of partitions, and the partitions at the same positions in
+ * the lists indicate join pairs used for partitioned join.
+ *
+ * This function returns NULL, setting *outer_parts and *inner_parts to NIL,
+ * if a partition on one side matches multiple partitions on the other side,
+ * in which case we currently don't support partitioned join.
+ */
+PartitionBoundInfo
+partition_bounds_merge(int partnatts,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype,
+					   List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo outer_binfo = outer_rel->boundinfo;
+	PartitionBoundInfo inner_binfo = inner_rel->boundinfo;
+	char		strategy;
+
+	/*
+	 * Currently, this function is called only from try_partitionwise_join(),
+	 * so the join type should be INNER, LEFT, FULL, SEMI, or ANTI.
+	 */
+	if (jointype != JOIN_INNER && jointype != JOIN_LEFT && 
+		jointype != JOIN_FULL && jointype != JOIN_SEMI &&
+		jointype != JOIN_ANTI)
+		elog(ERROR, "unrecognized join type: %d", (int) jointype);
+
+	/* Bail out if the partitioning strategies are different. */
+	if (outer_binfo->strategy != inner_binfo->strategy)
+		return NULL;
+
+	strategy = outer_binfo->strategy;
+	*outer_parts = *inner_parts = NIL;
+	switch (strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+
+			/*
+			 * For hash partitioned tables, we currently support partitioned
+			 * join only when the partition bounds for them exactly match.
+			 *
+			 * XXX: it might be possible to relax the restriction to support
+			 * cases where hash partitioned tables have missing partitions
+			 * and/or different moduli, but it's not clear if it would be
+			 * useful to support the former case since it's unusual to have
+			 * missing partitions.  On the other hand, it would be useful to
+			 * support the latter case, but in that case, there is a high
+			 * probability that a partition on one side will match multiple
+			 * partitions on the other side, which is the scenario the current
+			 * implementation of partitioned join can't handle.
+			 */
+			return NULL;
+
+		case PARTITION_STRATEGY_LIST:
+			return merge_list_bounds(partsupfunc,
+									 partcollation,
+									 outer_rel,
+									 inner_rel,
+									 jointype,
+									 outer_parts,
+									 inner_parts);
+
+		case PARTITION_STRATEGY_RANGE:
+			return merge_range_bounds(partnatts,
+									  partsupfunc,
+									  partcollation,
+									  outer_rel,
+									  inner_rel,
+									  jointype,
+									  outer_parts,
+									  inner_parts);
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d", (int) strategy);
+			return NULL;				/* keep compiler quiet */
+	}
+}
+
+/*
+ * merge_list_bounds
+ *		Create the partition bounds for a join relation between list
+ *		partitioned tables, if possible
+ */
+static PartitionBoundInfo
+merge_list_bounds(FmgrInfo *partsupfunc, Oid *partcollation,
+				  RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+				  JoinType jointype,
+				  List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			outer_pos;
+	int			inner_pos;
+	int			next_index = 0;
+	int			null_index = -1;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+	/* List partitioning doesn't require kinds. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * If the default partitions (if any) have been proven empty, deem them
+	 * non-existent.
+	 */
+	if (outer_has_default && is_dummy_partition(outer_rel, outer_default))
+		outer_has_default = false;
+	if (inner_has_default && is_dummy_partition(inner_rel, inner_default))
+		inner_has_default = false;
+
+	/*
+	 * Merge partitions from both sides.  In each iteration we compare a pair
+	 * of list values, one from each side, and decide whether the corresponding
+	 * partitions match or not.  If the two values match exactly, move to the
+	 * next pair of list values, otherwise move to the next list value on the
+	 * side with a smaller list value.
+	 */
+	outer_pos = inner_pos = 0;
+	while (outer_pos < outer_bi->ndatums || inner_pos < inner_bi->ndatums)
+	{
+		int			outer_index = -1;
+		int			inner_index = -1;
+		Datum	   *outer_datums;
+		Datum	   *inner_datums;
+		int			cmpval;
+		Datum	   *merged_datum = NULL;
+		int			merged_index = -1;
+
+		if (outer_pos < outer_bi->ndatums)
+		{
+			/*
+			 * If the partition on the outer side has been proven empty, ignore
+			 * it and move to the next datum on the outer side.
+			 */
+			outer_index = outer_bi->indexes[outer_pos];
+			if (is_dummy_partition(outer_rel, outer_index))
+			{
+				outer_pos++;
+				continue;
+			}
+		}
+		if (inner_pos < inner_bi->ndatums)
+		{
+			/*
+			 * If the partition on the inner side has been proven empty, ignore
+			 * it and move to the next datum on the inner side.
+			 */
+			inner_index = inner_bi->indexes[inner_pos];
+			if (is_dummy_partition(inner_rel, inner_index))
+			{
+				inner_pos++;
+				continue;
+			}
+		}
+
+		/* Get the list values. */
+		outer_datums = outer_pos < outer_bi->ndatums ?
+			outer_bi->datums[outer_pos] : NULL;
+		inner_datums = inner_pos < inner_bi->ndatums ?
+			inner_bi->datums[inner_pos] : NULL;
+
+		/*
+		 * We run this loop till both sides finish.  This allows us to avoid
+		 * duplicating code to handle the remaining values on the side which
+		 * finishes later.  For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra value higher than any other value on the unfinished side.
+		 * That way we advance the values on the unfinished side till all of
+		 * its values are exhausted.
+		 */
+		if (outer_pos >= outer_bi->ndatums)
+			cmpval = 1;
+		else if (inner_pos >= inner_bi->ndatums)
+			cmpval = -1;
+		else
+		{
+			Assert(outer_datums != NULL && inner_datums != NULL);
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 outer_datums[0],
+													 inner_datums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			/* Two list values match exactly. */
+			Assert(outer_pos < outer_bi->ndatums);
+			Assert(inner_pos < inner_bi->ndatums);
+			Assert(outer_index >= 0);
+			Assert(inner_index >= 0);
+
+			/*
+			 * Try merging both paritions.  If successful, add the list value
+			 * and index of the merged partition below.
+			 */
+			merged_index = merge_matching_partitions(&outer_map, &inner_map,
+													 outer_index, inner_index,
+													 &next_index);
+			if (merged_index == -1)
+				goto cleanup;
+
+			merged_datum = outer_datums;
+
+			/* Move to the next pair of list values. */
+			outer_pos++;
+			inner_pos++;
+		}
+		else if (cmpval < 0)
+		{
+			/* A list value missing from the inner side. */
+			Assert(outer_pos < outer_bi->ndatums);
+
+			/*
+			 * If the inner side has the default partition, or this is an outer
+			 * join, try to assign a merged partition to the outer partition
+			 * (see process_outer_partition()).  Otherwise, the outer partition
+			 * will not contribute to the result.
+			 */
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				/* Get the outer partition. */
+				outer_index = outer_bi->indexes[outer_pos];
+				Assert(outer_index >= 0);
+				merged_index = process_outer_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   outer_index,
+													   inner_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+				merged_datum = outer_datums;
+			}
+
+			/* Move to the next list value on the outer side. */
+			outer_pos++;
+		}
+		else
+		{
+			/* A list value missing from the outer side. */
+			Assert(cmpval > 0);
+			Assert(inner_pos < inner_bi->ndatums);
+
+			/*
+			 * If the outer side has the default partition, or this is a FULL
+			 * join, try to assign a merged partition to the inner partition
+			 * (see process_inner_partition()).  Otherwise, the inner partition
+			 * will not contribute to the result.
+			 */
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				/* Get the inner partition. */
+				inner_index = inner_bi->indexes[inner_pos];
+				Assert(inner_index >= 0);
+				merged_index = process_inner_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   inner_index,
+													   outer_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+				merged_datum = inner_datums;
+			}
+
+			/* Move to the next list value on the inner side. */
+			inner_pos++;
+		}
+
+		/*
+		 * If we assigned a merged partition, add the list value and index of
+		 * the merged partition if appropriate.
+		 */
+		if (merged_index >= 0 && merged_index != default_index)
+		{
+			merged_datums = lappend(merged_datums, merged_datum);
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+		}
+	}
+
+	/*
+	 * If the NULL partitions (if any) have been proven empty, deem them
+	 * non-existent.
+	 */
+	if (outer_has_null &&
+		is_dummy_partition(outer_rel, outer_bi->null_index))
+		outer_has_null = false;
+	if (inner_has_null &&
+		is_dummy_partition(inner_rel, inner_bi->null_index))
+		inner_has_null = false;
+
+	/* Merge the NULL partitions if any. */
+	if (outer_has_null || inner_has_null)
+		merge_null_partitions(&outer_map, &inner_map,
+							  outer_has_null, inner_has_null,
+							  outer_bi->null_index, inner_bi->null_index,
+							  jointype, &next_index, &null_index);
+	else
+		Assert(null_index == -1);
+
+	/* Merge the default partitions if any. */
+	if (outer_has_default || inner_has_default)
+		merge_default_partitions(&outer_map, &inner_map,
+								 outer_has_default, inner_has_default,
+								 outer_default, inner_default,
+								 jointype, &next_index, &default_index);
+	else
+		Assert(default_index == -1);
+
+	/* If we have merged partitions, create the partition bounds. */
+	if (next_index > 0)
+	{
+		/* Fix the merged_indexes list if necessary. */
+		if (outer_map.did_remapping || inner_map.did_remapping)
+		{
+			Assert(jointype == JOIN_FULL);
+			fix_merged_indexes(&outer_map, &inner_map,
+							   next_index, merged_indexes);
+		}
+
+		/* Use maps to match partitions from inputs. */
+		generate_matching_part_pairs(outer_rel, inner_rel,
+									 &outer_map, &inner_map,
+									 next_index,
+									 outer_parts, inner_parts);
+		Assert(*outer_parts != NIL);
+		Assert(*inner_parts != NIL);
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) <= next_index);
+
+		/* Make a PartitionBoundInfo struct to return. */
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  NIL,
+													  merged_indexes,
+													  null_index,
+													  default_index);
+		Assert(merged_bounds);
+	}
+
+cleanup:
+	/* Free local memory before returning. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+
+	return merged_bounds;
+}
+
+/*
+ * merge_range_bounds
+ *		Create the partition bounds for a join relation between range
+ *		partitioned tables, if possible
+ */
+static PartitionBoundInfo
+merge_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+				   Oid *partcollations,
+				   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+				   JoinType jointype,
+				   List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			outer_index;
+	int			inner_index;
+	int			outer_lb_pos;
+	int			inner_lb_pos;
+	PartitionRangeBound outer_lb;
+	PartitionRangeBound outer_ub;
+	PartitionRangeBound inner_lb;
+	PartitionRangeBound inner_ub;
+	int			next_index = 0;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_kinds = NIL;
+	List	   *merged_indexes = NIL;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * If the default partitions (if any) have been proven empty, deem them
+	 * non-existent.
+	 */
+	if (outer_has_default && is_dummy_partition(outer_rel, outer_default))
+		outer_has_default = false;
+	if (inner_has_default && is_dummy_partition(inner_rel, inner_default))
+		inner_has_default = false;
+
+	/*
+	 * Merge partitions from both sides.  In each iteration we compare a pair
+	 * of ranges, one from each side, and decide whether the corresponding
+	 * partitions match or not.  If the two ranges overlap, move to the next
+	 * pair of ranges, otherwise move to the next range on the side with a
+	 * lower range.  outer_lb_pos/inner_lb_pos keep track of the positions of
+	 * lower bounds in the datums arrays in the outer/inner PartitionBoundInfos
+	 * respectively.
+	 */
+	outer_lb_pos = inner_lb_pos = 0;
+	outer_index = get_range_partition(outer_rel, outer_bi, &outer_lb_pos,
+									  &outer_lb, &outer_ub);
+	inner_index = get_range_partition(inner_rel, inner_bi, &inner_lb_pos,
+									  &inner_lb, &inner_ub);
+	while (outer_index >= 0 || inner_index >= 0)
+	{
+		bool		overlap;
+		int			ub_cmpval;
+		int			lb_cmpval;
+		PartitionRangeBound merged_lb = {-1, NULL, NULL, true};
+		PartitionRangeBound merged_ub = {-1, NULL, NULL, false};
+		int			merged_index = -1;
+
+		/*
+		 * We run this loop till both sides finish.  This allows us to avoid
+		 * duplicating code to handle the remaining ranges on the side which
+		 * finishes later.  For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra range higher than any other range on the unfinished side.
+		 * That way we advance the ranges on the unfinished side till all of
+		 * its ranges are exhausted.
+		 */
+		if (outer_index == -1)
+		{
+			overlap = false;
+			lb_cmpval = 1;
+			ub_cmpval = 1;
+		}
+		else if (inner_index == -1)
+		{
+			overlap = false;
+			lb_cmpval = -1;
+			ub_cmpval = -1;
+		}
+		else
+			overlap = compare_range_partitions(partnatts, partsupfuncs,
+											   partcollations,
+											   &outer_lb, &outer_ub,
+											   &inner_lb, &inner_ub,
+											   &lb_cmpval, &ub_cmpval);
+
+		if (overlap)
+		{
+			/* Two ranges overlap; form a join pair. */
+
+			PartitionRangeBound save_outer_ub;
+			PartitionRangeBound save_inner_ub;
+
+			/* Both partitions should not have been merged yet. */
+			Assert(outer_index >= 0);
+			Assert(outer_map.merged_indexes[outer_index] == -1 &&
+				   outer_map.merged[outer_index] == false);
+			Assert(inner_index >= 0);
+			Assert(inner_map.merged_indexes[inner_index] == -1 &&
+				   inner_map.merged[inner_index] == false);
+
+			/*
+			 * Get the index of the merged partition.  Both partitions aren't
+			 * merged yet, so the partitions should be merged successfully.
+			 */
+			merged_index = merge_matching_partitions(&outer_map, &inner_map,
+													 outer_index, inner_index,
+													 &next_index);
+			Assert(merged_index >= 0);
+
+			/* Get the range of the merged partition. */
+			get_merged_range_bounds(partnatts, partsupfuncs,
+									partcollations, jointype,
+									&outer_lb, &outer_ub,
+									&inner_lb, &inner_ub,
+									lb_cmpval, ub_cmpval,
+									&merged_lb, &merged_ub);
+
+			/* Save the upper bounds of both partitions for use below. */
+			save_outer_ub = outer_ub;
+			save_inner_ub = inner_ub;
+
+			/* Move to the next pair of ranges. */
+			outer_index = get_range_partition(outer_rel, outer_bi, &outer_lb_pos,
+											  &outer_lb, &outer_ub);
+			inner_index = get_range_partition(inner_rel, inner_bi, &inner_lb_pos,
+											  &inner_lb, &inner_ub);
+
+			/*
+			 * If the range of a partition on one side overlaps the range of
+			 * the next partition on the other side, that will cause the
+			 * partition on one side to match at least two partitions on the
+			 * other side, which is the case that we currently don't support
+			 * partitioned join for; give up.
+			 */
+			if (ub_cmpval > 0 && inner_index >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &save_outer_ub, &inner_lb) > 0)
+				goto cleanup;
+			if (ub_cmpval < 0 && outer_index >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &outer_lb, &save_inner_ub) < 0)
+				goto cleanup;
+
+			/*
+			 * A row from a non-overlapping portion (if any) of a partition
+			 * on one side might find its join partner in the default
+			 * partition (if any) on the other side, causing the same
+			 * situation as above; give up in that case.
+			 */
+			if ((outer_has_default && (lb_cmpval > 0 || ub_cmpval < 0)) ||
+				(inner_has_default && (lb_cmpval < 0 || ub_cmpval > 0)))
+				goto cleanup;
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* A non-overlapping outer range. */
+
+			/* The outer partition should not have been merged yet. */
+			Assert(outer_index >= 0);
+			Assert(outer_map.merged_indexes[outer_index] == -1 &&
+				   outer_map.merged[outer_index] == false);
+
+			/*
+			 * If the inner side has the default partition, or this is an outer
+			 * join, try to assign a merged partition to the outer partition
+			 * (see process_outer_partition()).  Otherwise, the outer partition
+			 * will not contribute to the result.
+			 */
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				merged_index = process_outer_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   outer_index,
+													   inner_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+				merged_lb = outer_lb;
+				merged_ub = outer_ub;
+			}
+
+			/* Move to the next range on the outer side. */
+			outer_index = get_range_partition(outer_rel, outer_bi, &outer_lb_pos,
+											  &outer_lb, &outer_ub);
+		}
+		else
+		{
+			/* A non-overlapping inner range. */
+			Assert(ub_cmpval > 0);
+
+			/* The inner partition should not have been merged yet. */
+			Assert(inner_index >= 0);
+			Assert(inner_map.merged_indexes[inner_index] == -1 &&
+				   inner_map.merged[inner_index] == false);
+
+			/*
+			 * If the outer side has the default partition, or this is a FULL
+			 * join, try to assign a merged partition to the inner partition
+			 * (see process_inner_partition()).  Otherwise, the inner partition
+			 * will not contribute to the result.
+			 */
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				merged_index = process_inner_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   inner_index,
+													   outer_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+				merged_lb = inner_lb;
+				merged_ub = inner_ub;
+			}
+
+			/* Move to the next range on the inner side. */
+			inner_index = get_range_partition(inner_rel, inner_bi, &inner_lb_pos,
+											  &inner_lb, &inner_ub);
+		}
+
+		/*
+		 * If we assigned a merged partition, add the range bounds and index of
+		 * the merged partition if appropriate.
+		 */
+		if (merged_index >= 0 && merged_index != default_index)
+			add_merged_range_bounds(partnatts, partsupfuncs, partcollations,
+									&merged_lb, &merged_ub, merged_index,
+									&merged_datums, &merged_kinds,
+									&merged_indexes);
+	}
+
+	/* Merge the default partitions if any. */
+	if (outer_has_default || inner_has_default)
+		merge_default_partitions(&outer_map, &inner_map,
+								 outer_has_default, inner_has_default,
+								 outer_default, inner_default,
+								 jointype, &next_index, &default_index);
+	else
+		Assert(default_index == -1);
+
+	/* If we have merged partitions, create the partition bounds. */
+	if (next_index > 0)
+	{
+		/*
+		 * Unlike the case of list partitioning, we wouldn't have re-merged
+		 * partitions, so did_remapping should be left alone.
+		 */
+		Assert(!outer_map.did_remapping);
+		Assert(!inner_map.did_remapping);
+
+		/* Use maps to match partitions from inputs. */
+		generate_matching_part_pairs(outer_rel, inner_rel,
+									 &outer_map, &inner_map,
+									 next_index,
+									 outer_parts, inner_parts);
+		Assert(*outer_parts != NIL);
+		Assert(*inner_parts != NIL);
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+
+		/* Make a PartitionBoundInfo struct to return. */
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_kinds,
+													  merged_indexes,
+													  -1,
+													  default_index);
+		Assert(merged_bounds);
+	}
+
+cleanup:
+	/* Free local memory before returning. */
+	list_free(merged_datums);
+	list_free(merged_kinds);
+	list_free(merged_indexes);
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+
+	return merged_bounds;
+}
+
+/*
+ * init_partition_map
+ *		Initialize a PartitionMap struct for given relation
+ */
+static void
+init_partition_map(RelOptInfo *rel, PartitionMap *map)
+{
+	int			nparts = rel->nparts;
+	int			i;
+
+	map->nparts = nparts;
+	map->merged_indexes = (int *) palloc(sizeof(int) * nparts);
+	map->merged = (bool *) palloc(sizeof(bool) * nparts);
+	map->did_remapping = false;
+	map->old_indexes = (int *) palloc(sizeof(int) * nparts);
+	for (i = 0; i < nparts; i++)
+	{
+		map->merged_indexes[i] = map->old_indexes[i] = -1;
+		map->merged[i] = false;
+	}
+}
+
+/*
+ * free_partition_map
+ */
+static void
+free_partition_map(PartitionMap *map)
+{
+	pfree(map->merged_indexes);
+	pfree(map->merged);
+	pfree(map->old_indexes);
+}
+
+/*
+ * is_dummy_partition --- has partition been proven empty?
+ */
+static bool
+is_dummy_partition(RelOptInfo *rel, int part_index)
+{
+	RelOptInfo *part_rel;
+
+	Assert(part_index >= 0);
+	part_rel = rel->part_rels[part_index];
+	if (part_rel == NULL || IS_DUMMY_REL(part_rel))
+		return true;
+	return false;
+}
+
+/*
+ * merge_matching_partitions
+ *		Try to merge given outer/inner partitions, and return the index of a
+ *		merged partition produced from them if successful, -1 otherwise
+ *
+ * If the merged partition is newly created, *next_index is incremented.
+ */
+static int
+merge_matching_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						  int outer_index, int inner_index, int *next_index)
+{
+	int 		outer_merged_index;
+	int 		inner_merged_index;
+	bool 		outer_merged;
+	bool 		inner_merged;
+
+	Assert(outer_index >= 0 && outer_index < outer_map->nparts);
+	outer_merged_index = outer_map->merged_indexes[outer_index];
+	outer_merged = outer_map->merged[outer_index];
+	Assert(inner_index >= 0 && inner_index < inner_map->nparts);
+	inner_merged_index = inner_map->merged_indexes[inner_index];
+	inner_merged = inner_map->merged[inner_index];
+
+	/*
+	 * Handle cases where we have already assigned a merged partition to each
+	 * of the given partitions.
+	 */
+	if (outer_merged_index >= 0 && inner_merged_index >= 0)
+	{
+		/*
+		 * If the mereged partitions are the same, no need to do anything;
+		 * return the index of the merged partitions.  Otherwise, if each of
+		 * the given partitions has been merged with a dummy partition on the
+		 * other side, re-map them to either of the two merged partitions.
+		 * Otherwise, they can't be merged, so return -1.
+		 */
+		if (outer_merged_index == inner_merged_index)
+		{
+			Assert(outer_merged);
+			Assert(inner_merged);
+			return outer_merged_index;
+		}
+		if (!outer_merged && !inner_merged)
+		{
+			/*
+			 * This can only happen for a list-partitioning case.  We re-map
+			 * them to the merged partition with the smaller of the two merged
+			 * indexes to preserve the property that the canonical order of
+			 * list partitions is determined by the indexes assigned to the
+			 * smallest list value of each partition.
+			 */
+			if (outer_merged_index < inner_merged_index)
+			{
+				outer_map->merged[outer_index] = true;
+				inner_map->merged_indexes[inner_index] = outer_merged_index;
+				inner_map->merged[inner_index] = true;
+				inner_map->did_remapping = true;
+				inner_map->old_indexes[inner_index] = inner_merged_index;
+				return outer_merged_index;
+			}
+			else
+			{
+				inner_map->merged[inner_index] = true;
+				outer_map->merged_indexes[outer_index] = inner_merged_index;
+				outer_map->merged[outer_index] = true;
+				outer_map->did_remapping = true;
+				outer_map->old_indexes[outer_index] = outer_merged_index;
+				return inner_merged_index;
+			}
+		}
+		return -1;
+	}
+
+	/* At least one of the given partitions should not have yet been merged. */
+	Assert(outer_merged_index == -1 || inner_merged_index == -1);
+
+	/*
+	 * If neither of them has been merged, merge them.  Otherwise, if one has
+	 * been merged with a dummy relation on the other side (and the other
+	 * hasn't yet been merged with anything), re-merge them.  Otherwise, they
+	 * can't be merged, so return -1.
+	 */
+	if (outer_merged_index == -1 && inner_merged_index == -1)
+	{
+		int 		merged_index = *next_index;
+
+		Assert(!outer_merged);
+		Assert(!inner_merged);
+		outer_map->merged_indexes[outer_index] = merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged_indexes[inner_index] = merged_index;
+		inner_map->merged[inner_index] = true;
+		*next_index = *next_index + 1;
+		return merged_index;
+	}
+	if (outer_merged_index >= 0 && !outer_map->merged[outer_index])
+	{
+		Assert(inner_merged_index == -1);
+		Assert(!inner_merged);
+		inner_map->merged_indexes[inner_index] = outer_merged_index;
+		inner_map->merged[inner_index] = true;
+		outer_map->merged[outer_index] = true;
+		return outer_merged_index;
+	}
+	if (inner_merged_index >= 0 && !inner_map->merged[inner_index])
+	{
+		Assert(outer_merged_index == -1);
+		Assert(!outer_merged);
+		outer_map->merged_indexes[outer_index] = inner_merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged[inner_index] = true;
+		return inner_merged_index;
+	}
+	return -1;
+}
+
+/*
+ * process_outer_partition
+ *		Try to assign given outer partition a merged partition, and return the
+ *		index of the merged partition if successful, -1 otherwise
+ *
+ * If the partition is newly created, *next_index is incremented.  Also, if it
+ * is the default partition of the join relation, *default_partition is set to
+ * the index if not already done.
+ */
+static int
+process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index)
+{
+	int 		merged_index = -1;
+
+	Assert(outer_index >= 0);
+
+	/*
+	 * If the inner side has the default partition, a row from the outer
+	 * partition might find its join partner in the default partition; try
+	 * merging the outer partition with the default partition.  Otherwise, this
+	 * should be an outer join, in which case the outer partition has to be
+	 * scanned all the way anyway; merge the outer partition with a dummy
+	 * partition on the other side.
+	 */
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0);
+
+		/*
+		 * If the outer side has the default partition as well, the default
+		 * partition on the inner side will have two matching partitions on the
+		 * other side: the outer partition and the default partition on the
+		 * outer side.  Partitionwise join doesn't handle this scenario yet.
+		 */
+		if (outer_has_default)
+			return -1;
+
+		merged_index = merge_matching_partitions(outer_map, inner_map,
+												 outer_index, inner_default,
+												 next_index);
+		if (merged_index == -1)
+			return -1;
+
+		/*
+		 * If this is a FULL join, the default partition on the inner side
+		 * has to be scanned all the way anyway, so the resulting partition
+		 * will contain all key values from the default partition, which any
+		 * other partition of the join relation will not contain.  Thus the
+		 * resutling partition will act as the default partition of the join
+		 * relation; record the index in *default_index if not already done.
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (*default_index == -1)
+				*default_index = merged_index;
+			else
+				Assert(*default_index == merged_index);
+		}
+	}
+	else
+	{
+		Assert(IS_OUTER_JOIN(jointype));
+		Assert(jointype != JOIN_RIGHT);
+
+		/* If we have already assigned a partition, no need to do anything. */
+		merged_index = outer_map->merged_indexes[outer_index];
+		if (merged_index == -1)
+			merged_index = merge_partition_with_dummy(outer_map, outer_index,
+													  next_index);
+	}
+	return merged_index;
+}
+
+/*
+ * process_inner_partition
+ *		Try to assign given inner partition a merged partition, and return the
+ *		index of the merged partition if successful, -1 otherwise
+ *
+ * If the partition is newly created, *next_index is incremented.  Also, if it
+ * is the default partition of the join relation, *default_partition is set to
+ * the index if not already done.
+ */
+static int
+process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index)
+{
+	int 		merged_index = -1;
+
+	Assert(inner_index >= 0);
+
+	/*
+	 * If the outer side has the default partition, a row from the inner
+	 * partition might find its join partner in the default partition; try
+	 * merging the inner partition with the default partition.  Otherwise, this
+	 * should be a FULL join, in which case the inner partition has to be
+	 * scanned all the way anyway; merge the inner partition with a dummy
+	 * partition on the other side.
+	 */
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0);
+
+		/*
+		 * If the inner side has the default partition as well, the default
+		 * partition on the outer side will have two matching partitions on the
+		 * other side: the inner partition and the default partition on the
+		 * inner side.  Partitionwise join doesn't handle this scenario yet.
+		 */
+		if (inner_has_default)
+			return -1;
+
+		merged_index = merge_matching_partitions(outer_map, inner_map,
+												 outer_default, inner_index,
+												 next_index);
+		if (merged_index == -1)
+			return -1;
+
+		/*
+		 * If this is an outer join, the default partition on the outer side
+		 * has to be scanned all the way anyway, so the resulting partition
+		 * will contain all key values from the default partition, which any
+		 * other partition of the join relation will not contain.  Thus the
+		 * resutling partition will act as the default partition of the join
+		 * relation; record the index in *default_index if not already done.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			if (*default_index == -1)
+				*default_index = merged_index;
+			else
+				Assert(*default_index == merged_index);
+		}
+	}
+	else
+	{
+		Assert(jointype == JOIN_FULL);
+
+		/* If we have already assigned a partition, no need to do anything. */
+		merged_index = inner_map->merged_indexes[inner_index];
+		if (merged_index == -1)
+			merged_index = merge_partition_with_dummy(inner_map, inner_index,
+													  next_index);
+	}
+	return merged_index;
+}
+
+/*
+ * merge_null_partitions
+ *		Merge the NULL partitions from a join's outer and inner sides.
+ *
+ * If the merged partition produced from them is the NULL partition of the join
+ * relation, *null_index is set to the index of the merged partition.
+ *
+ * Note: We assume here that the join clause for a partitioned join is strict
+ * because have_partkey_equi_join() requires that the corresponding operator
+ * be mergejoinable, and we currently assume that mergejoinable operators are
+ * strict (see MJEvalOuterValues()/MJEvalInnerValues()).
+ */
+static void
+merge_null_partitions(PartitionMap *outer_map,
+					  PartitionMap *inner_map,
+					  bool outer_has_null,
+					  bool inner_has_null,
+					  int outer_null,
+					  int inner_null,
+					  JoinType jointype,
+					  int *next_index,
+					  int *null_index)
+{
+	bool 		consider_outer_null = false;
+	bool 		consider_inner_null = false;
+
+	Assert(outer_has_null || inner_has_null);
+	Assert(*null_index == -1);
+
+	/*
+	 * Check whether the NULL partitions have already been merged and if so,
+	 * set the consider_outer_null/consider_inner_null flags.
+	 */
+	if (outer_has_null)
+	{
+		Assert(outer_null >= 0 && outer_null < outer_map->nparts);
+		if (outer_map->merged_indexes[outer_null] == -1)
+		 	consider_outer_null = true;
+	}
+	if (inner_has_null)
+	{
+		Assert(inner_null >= 0 && inner_null < inner_map->nparts);
+		if (inner_map->merged_indexes[inner_null] == -1)
+		 	consider_inner_null = true;
+	}
+
+	/* If both flags are set false, we don't need to do anything. */
+	if (!consider_outer_null && !consider_inner_null)
+		return;
+
+	if (consider_outer_null && !consider_inner_null)
+	{
+		Assert(outer_has_null);
+
+		/*
+		 * If this is an outer join, the NULL partition on the outer side has
+		 * to be scanned all the way anyway; merge the NULL partition with a
+		 * dummy partition on the other side.  In that case consider_outer_null
+		 * means that the NULL partition only contains NULL values as the key
+		 * values, so the merged partition will do so; treat it as the NULL
+		 * partition of the join relation.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			*null_index = merge_partition_with_dummy(outer_map, outer_null,
+													 next_index);
+		}
+	}
+	else if (!consider_outer_null && consider_inner_null)
+	{
+		Assert(inner_has_null);
+
+		/*
+		 * If this is a FULL join, the NULL partition on the inner side has
+		 * to be scanned all the way anyway; merge the NULL partition with a
+		 * dummy partition on the other side.  In that case consider_inner_null
+		 * means that the NULL partition only contains NULL values as the key
+		 * values, so the merged partition will do so; treat it as the NULL
+		 * partition of the join relation.
+		 */
+		if (jointype == JOIN_FULL)
+			*null_index = merge_partition_with_dummy(inner_map, inner_null,
+													 next_index);
+	}
+	else
+	{
+		Assert(consider_outer_null && consider_inner_null);
+		Assert(outer_has_null);
+		Assert(inner_has_null);
+
+		/*
+		 * If this is an outer join, the NULL partition on the outer side (and
+		 * that on the inner side if this is a FULL join) have to be scanned
+		 * all the way anyway, so merge them.  Note that each of the NULL
+		 * partitions isn't merged yet, so they should be merged successfully.
+		 * Like the above, each of the NULL partitions only contains NULL
+		 * values as the key values, so the merged partition will do so; treat
+		 * it as the NULL partition of the join relation.
+		 *
+		 * Note: if this an INNER/SEMI join, the join clause will never be
+		 * satisfied by two NULL values (see comments above), so both the NULL
+		 * partitions can be eliminated.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			*null_index = merge_matching_partitions(outer_map, inner_map,
+													outer_null, inner_null,
+													next_index);
+			Assert(*null_index >= 0);
+		}
+	}
+}
+
+/*
+ * merge_default_partitions
+ *		Merge the default partitions from a join's outer and inner sides.
+ *
+ * If the merged partition produced from them is the default partition of the
+ * join relation, *default_index is set to the index of the merged partition.
+ */
+static void
+merge_default_partitions(PartitionMap *outer_map,
+						 PartitionMap *inner_map,
+						 bool outer_has_default,
+						 bool inner_has_default,
+						 int outer_default,
+						 int inner_default,
+						 JoinType jointype,
+						 int *next_index,
+						 int *default_index)
+{
+	int 		outer_merged_index = -1;
+	int 		inner_merged_index = -1;
+
+	Assert(outer_has_default || inner_has_default);
+
+	/* Get the merged partition indexes for the default partitions. */
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0 && outer_default < outer_map->nparts);
+		outer_merged_index = outer_map->merged_indexes[outer_default];
+	}
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0 && inner_default < inner_map->nparts);
+		inner_merged_index = inner_map->merged_indexes[inner_default];
+	}
+
+	if (outer_has_default && !inner_has_default)
+	{
+		/*
+		 * If this is an outer join, the default partition on the outer side
+		 * has to be scanned all the way anyway; if we have not yet assigned a
+		 * partition, merge the default partition with a dummy partition on the
+		 * other side.  The merged partition will act as the default partition
+		 * of the join relation (see comments in process_inner_partition()).
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			if (outer_merged_index == -1)
+			{
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(outer_map,
+															outer_default,
+															next_index);
+			}
+			else
+				Assert(*default_index == outer_merged_index);
+		}
+		else
+			Assert(*default_index == -1);
+	}
+	else if (!outer_has_default && inner_has_default)
+	{
+		/*
+		 * If this is a FULL join, the default partition on the inner side
+		 * has to be scanned all the way anyway; if we have not yet assigned a
+		 * partition, merge the default partition with a dummy partition on the
+		 * other side.  The merged partition will act as the default partition
+		 * of the join relation (see comments in process_outer_partition()).
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (inner_merged_index == -1)
+			{
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(inner_map,
+															inner_default,
+															next_index);
+			}
+			else
+				Assert(*default_index == inner_merged_index);
+		}
+		else
+			Assert(*default_index == -1);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		/*
+		 * The default partitions have to be joined with each other, so merge
+		 * them.  Note that each of the default partitions isn't merged yet
+		 * (see, process_outer_partition()/process_innerer_partition()), so
+		 * they should be merged successfully.  The merged partition will act
+		 * as the default partition of the join relation.
+		 */
+		Assert(outer_merged_index == -1);
+		Assert(inner_merged_index == -1);
+		Assert(*default_index == -1);
+		*default_index = merge_matching_partitions(outer_map,
+												   inner_map,
+												   outer_default,
+												   inner_default,
+												   next_index);
+		Assert(*default_index >= 0);
+	}
+}
+
+/*
+ * merge_partition_with_dummy
+ *		Assign given partition a new partition of a join relation
+ *
+ * Note: The caller assumes that the given partition doesn't have a non-dummy
+ * matching partition on the other side, but if the given partition finds the
+ * matchig partition later, we will adjust the assignment.
+ */
+static int
+merge_partition_with_dummy(PartitionMap *map, int index, int *next_index)
+{
+	int 		merged_index = *next_index;
+
+	Assert(index >= 0 && index < map->nparts);
+	Assert(map->merged_indexes[index] == -1);
+	Assert(!map->merged[index]);
+	map->merged_indexes[index] = merged_index;
+	/* Leave the merged flag alone! */
+	*next_index = *next_index + 1;
+	return merged_index;
+}
+
+/*
+ * fix_merged_indexes
+ *		Adjust merged indexes of re-merged partitions
+ */
+static void
+fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes)
+{
+	int		   *new_indexes;
+	int			merged_index;
+	int			i;
+	ListCell   *lc;
+
+	Assert(nmerged > 0);
+
+	new_indexes = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		new_indexes[i] = -1;
+
+	/* Build the mapping of old merged indexes to new merged indexes. */
+	if (outer_map->did_remapping)
+	{
+		for (i = 0; i < outer_map->nparts; i++)
+		{
+			merged_index = outer_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = outer_map->merged_indexes[i];
+		}
+	}
+	if (inner_map->did_remapping)
+	{
+		for (i = 0; i < inner_map->nparts; i++)
+		{
+			merged_index = inner_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = inner_map->merged_indexes[i];
+		}
+	}
+
+	/* Fix the merged_indexes list using the mapping. */
+	foreach(lc, merged_indexes)
+	{
+		merged_index = lfirst_int(lc);
+		Assert(merged_index >= 0);
+		if (new_indexes[merged_index] >= 0)
+			lfirst_int(lc) = new_indexes[merged_index];
+	}
+
+	pfree(new_indexes);
+}
+
+/*
+ * generate_matching_part_pairs
+ *		Generate a pair of lists of partitions that produce merged partitions
+ *
+ * The lists of partitions are built in the order of merged partition indexes,
+ * and returned in *outer_parts and *inner_parts.
+ */
+static void
+generate_matching_part_pairs(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 PartitionMap *outer_map, PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_parts, List **inner_parts)
+{
+	int			outer_nparts = outer_map->nparts;
+	int			inner_nparts = inner_map->nparts;
+	int		   *outer_indexes;
+	int		   *inner_indexes;
+	int			max_nparts;
+	int 		i;
+
+	Assert(nmerged > 0);
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	outer_indexes = (int *) palloc(sizeof(int) * nmerged);
+	inner_indexes = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		outer_indexes[i] = inner_indexes[i] = -1;
+
+	/* Set pairs of matching partitions. */
+	Assert(outer_nparts == outer_rel->nparts);
+	Assert(inner_nparts == inner_rel->nparts);
+	max_nparts = Max(outer_nparts, inner_nparts);
+	for (i = 0; i < max_nparts; i++)
+	{
+		if (i < outer_nparts)
+		{
+			int 		merged_index = outer_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				outer_indexes[merged_index] = i;
+			}
+		}
+		if (i < inner_nparts)
+		{
+			int 		merged_index = inner_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				inner_indexes[merged_index] = i;
+			}
+		}
+	}
+
+	/* Build the list pairs. */
+	for (i = 0; i < nmerged; i++)
+	{
+		int			outer_index = outer_indexes[i];
+		int			inner_index = inner_indexes[i];
+
+		/*
+		 * If both partitions are dummy, it means the merged partition that had
+		 * been assigned to the outer/inner partition was removed when
+		 * re-merging the outer/inner partition in merge_matching_partitions();
+		 * ignore the merged partition.
+		 */
+		if (outer_index == -1 && inner_index == -1)
+			continue;
+
+		*outer_parts = lappend(*outer_parts, outer_index >= 0 ?
+							   outer_rel->part_rels[outer_index] : NULL);
+		*inner_parts = lappend(*inner_parts, inner_index >= 0 ?
+							   inner_rel->part_rels[inner_index] : NULL);
+	}
+
+	pfree(outer_indexes);
+	pfree(inner_indexes);
+}
+
+/*
+ * build_merged_partition_bounds
+ *		Create a PartitionBoundInfo struct from merged partition bounds
+ */
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_kinds, List *merged_indexes,
+							  int null_index, int default_index)
+{
+	PartitionBoundInfo merged_bounds;
+	int			ndatums = list_length(merged_datums);
+	int			pos;
+	ListCell   *lc;
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = ndatums;
+
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) * ndatums);
+	pos = 0;
+	foreach(lc, merged_datums)
+		merged_bounds->datums[pos++] = (Datum *) lfirst(lc);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_kinds) == ndatums);
+		merged_bounds->kind = (PartitionRangeDatumKind **)
+			palloc(sizeof(PartitionRangeDatumKind *) * ndatums);
+		pos = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[pos++] = (PartitionRangeDatumKind *) lfirst(lc);
+
+		/* There are ndatums+1 indexes in the case of range partitioning. */
+		merged_indexes = lappend_int(merged_indexes, -1);
+		ndatums++;
+	}
+	else
+	{
+		Assert(strategy == PARTITION_STRATEGY_LIST);
+		Assert(merged_kinds == NIL);
+		merged_bounds->kind = NULL;
+	}
+
+	Assert(list_length(merged_indexes) == ndatums);
+	merged_bounds->indexes = (int *) palloc(sizeof(int) * ndatums);
+	pos = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[pos++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * get_range_partition
+ *		Get the next non-dummy partition of a range-partitioned relation,
+ *		returning the index of that partition
+ *
+ * *lb and *ub are set to the lower and upper bounds of that partition
+ * respectively, and *lb_pos is advanced to the next lower bound, if any.
+ */
+static int
+get_range_partition(RelOptInfo *rel,
+					PartitionBoundInfo bi,
+					int *lb_pos,
+					PartitionRangeBound *lb,
+					PartitionRangeBound *ub)
+{
+	int			part_index;
+
+	Assert(bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	do {
+		part_index = get_range_partition_internal(bi, lb_pos, lb, ub);
+		if (part_index == -1)
+			return -1;
+	} while (is_dummy_partition(rel, part_index));
+
+	return part_index;
+}
+
+static int
+get_range_partition_internal(PartitionBoundInfo bi,
+							 int *lb_pos,
+							 PartitionRangeBound *lb,
+							 PartitionRangeBound *ub)
+{
+	/* Return the index as -1 if we've exhausted all lower bounds. */
+	if (*lb_pos >= bi->ndatums)
+		return -1;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(*lb_pos + 1 < bi->ndatums);
+
+	/* Set the lower bound. */
+	lb->index = bi->indexes[*lb_pos];
+	lb->datums = bi->datums[*lb_pos];
+	lb->kind = bi->kind[*lb_pos];
+	lb->lower = true;
+	/* Set the upper bound. */
+	ub->index = bi->indexes[*lb_pos + 1];
+	ub->datums = bi->datums[*lb_pos + 1];
+	ub->kind = bi->kind[*lb_pos + 1];
+	ub->lower = false;
+
+	/* The index assigned to an upper bound should be valid. */
+	Assert(ub->index >= 0);
+
+	/*
+	 * Advance the position to the next lower bound.  If there are no bounds
+	 * left beyond the upper bound, we have reached the last lower bound.
+	 */
+	if (*lb_pos + 2 >= bi->ndatums)
+		*lb_pos = bi->ndatums;
+	else
+	{
+		/*
+		 * If the index assigned to the bound next to the upper bound isn't
+		 * valid, that is the next lower bound; else, the upper bound is also
+		 * the lower bound of the next range partition.
+		 */
+		if (bi->indexes[*lb_pos + 2] < 0)
+			*lb_pos = *lb_pos + 2;
+		else
+			*lb_pos = *lb_pos + 1;
+	}
+
+	return ub->index;
+}
+
+/*
+ * compare_range_partitions
+ *		Compare the bounds of two range partitions, and return true if the
+ *		two partitions overlap, false otherwise
+ *
+ * *lb_cmpval is set to -1, 0, or 1 if the outer partition's lower bound is
+ * lower than, equal to, or higher than the inner partition's lower bound
+ * respectively.  Likewise, *ub_cmpval is set to -1, 0, or 1 if the outer
+ * partition's upper bound is lower than, equal to, or higher than the inner
+ * partition's upper bound respectively.
+ */
+static bool
+compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+						 Oid *partcollations,
+						 PartitionRangeBound *outer_lb,
+						 PartitionRangeBound *outer_ub,
+						 PartitionRangeBound *inner_lb,
+						 PartitionRangeBound *inner_ub,
+						 int *lb_cmpval, int *ub_cmpval)
+{
+	/*
+	 * Check if the outer partition's upper bound is lower than the inner
+	 * partition's lower bound; if so the partitions aren't overlapping.
+	 */
+	if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+							 outer_ub, inner_lb) < 0)
+	{
+		*lb_cmpval = -1;
+		*ub_cmpval = -1;
+		return false;
+	}
+
+	/*
+	 * Check if the outer partition's lower bound is higher than the inner
+	 * partition's upper bound; if so the partitions aren't overlapping.
+	 */
+	if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+							 outer_lb, inner_ub) > 0)
+	{
+		*lb_cmpval = 1;
+		*ub_cmpval = 1;
+		return false;
+	}
+
+	/* All other cases indicate overlapping partitions. */
+	*lb_cmpval = compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									  outer_lb, inner_lb);
+	*ub_cmpval = compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									  outer_ub, inner_ub);
+	return true;
+}
+
+/*
+ * get_merged_range_bounds
+ *		Given the bounds of range partitions to be joined, determine the bounds
+ *		of a merged partition produced from the range partitions
+ *
+ * *merged_lb and *merged_ub are set to the lower and upper bounds of the
+ * merged partition.
+ */
+static void
+get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations, JoinType jointype,
+						PartitionRangeBound *outer_lb,
+						PartitionRangeBound *outer_ub,
+						PartitionRangeBound *inner_lb,
+						PartitionRangeBound *inner_ub,
+						int	lb_cmpval, int ub_cmpval,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub)
+{
+	Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								outer_lb, inner_lb) == lb_cmpval);
+	Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								outer_ub, inner_ub) == ub_cmpval);
+
+	switch (jointype)
+	{
+		case JOIN_INNER:
+		case JOIN_SEMI:
+
+			/*
+			 * An INNER/SEMI join will have the rows that fit both sides, so
+			 * the lower bound of the merged partition will be the higher of
+			 * the two lower bounds, and the upper bound of the merged
+			 * partition will be the lower of the two upper bounds.
+			 */
+			*merged_lb = (lb_cmpval > 0) ? *outer_lb : *inner_lb;
+			*merged_ub = (ub_cmpval < 0) ? *outer_ub : *inner_ub;
+			break;
+
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+
+			/*
+			 * A LEFT/ANTI join will have all the rows from the outer side, so
+			 * the bounds of the merged partition will be the same as the outer
+			 * bounds.
+			 */
+			*merged_lb = *outer_lb;
+			*merged_ub = *outer_ub;
+			break;
+
+		case JOIN_FULL:
+
+			/*
+			 * A FULL join will have all the rows from both sides, so the lower
+			 * bound of the merged partition will be the lower of the two lower
+			 * bounds, and the upper bound of the merged partition will be the
+			 * higher of the two upper bounds.
+			 */
+			*merged_lb = (lb_cmpval < 0) ? *outer_lb : *inner_lb;
+			*merged_ub = (ub_cmpval > 0) ? *outer_ub : *inner_ub;
+			break;
+
+		default:
+			elog(ERROR, "unrecognized join type: %d", (int) jointype);
+	}
+}
+
+/*
+ * add_merged_range_bounds
+ *		Add the bounds of a merged partition to the lists of range bounds
+ */
+static void
+add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub,
+						int merged_index,
+						List **merged_datums,
+						List **merged_kinds,
+						List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		/* First merged partition */
+		Assert(!*merged_kinds);
+		Assert(!*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		Assert(*merged_datums);
+		Assert(*merged_kinds);
+		Assert(*merged_indexes);
+
+		/* Get the last upper bound. */
+		prev_ub.index = llast_int(*merged_indexes);
+		prev_ub.datums = (Datum *) llast(*merged_datums);
+		prev_ub.kind = (PartitionRangeDatumKind *) llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		/*
+		 * We pass to partition_rbound_cmp() lower1 as false to prevent it
+		 * from considering the last upper bound to be smaller than the lower
+		 * bound of the merged partition when the values of the two range
+		 * bounds compare equal.
+		 */
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  merged_lb->datums, merged_lb->kind,
+									  false, &prev_ub);
+		Assert(cmpval >= 0);
+	}
+
+	/*
+	 * If the lower bound is higher than the last upper bound, add the lower
+	 * bound with the index as -1 indicating that that is a lower bound; else,
+	 * the last upper bound will be reused as the lower bound of the merged
+	 * partition, so skip this.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, merged_lb->datums);
+		*merged_kinds = lappend(*merged_kinds, merged_lb->kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	/* Add the upper bound and index of the merged partition. */
+	*merged_datums = lappend(*merged_datums, merged_ub->datums);
+	*merged_kinds = lappend(*merged_kinds, merged_ub->kind);
+	*merged_indexes = lappend_int(*merged_indexes, merged_index);
+}
+
 /*
  * partitions_are_ordered
  *		Determine whether the partitions described by 'boundinfo' are ordered,
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index 5334a73b53..7f9c4ab1f1 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -579,8 +579,10 @@ typedef struct PartitionSchemeData *PartitionScheme;
  *		part_scheme - Partitioning scheme of the relation
  *		nparts - Number of partitions
  *		boundinfo - Partition bounds
+ *		merged - true if partition bounds are merged ones
  *		partition_qual - Partition constraint if not the root
  *		part_rels - RelOptInfos for each partition
+ *		all_partrels - Relids set of all partition relids
  *		partexprs, nullable_partexprs - Partition key expressions
  *		partitioned_child_rels - RT indexes of unpruned partitions of
  *								 this relation that are partitioned tables
@@ -718,11 +720,15 @@ typedef struct RelOptInfo
 
 	/* used for partitioned relations */
 	PartitionScheme part_scheme;	/* Partitioning scheme. */
-	int			nparts;			/* number of partitions */
+	int			nparts;			/* number of partitions; 0 = not partitioned;
+								 * -1 = not yet set */
 	struct PartitionBoundInfoData *boundinfo;	/* Partition bounds */
+	bool		merged;			/* true if partition bounds were created by
+								 * partition_bounds_merge() */
 	List	   *partition_qual; /* partition constraint */
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of partitions,
 									 * stored in the same order of bounds */
+	Relids		all_partrels;	/* Relids set of all partition relids */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 	List	   *partitioned_child_rels; /* List of RT indexes. */
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index be940b5192..dfc720720b 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -16,6 +16,7 @@
 #include "nodes/pg_list.h"
 #include "partitioning/partdefs.h"
 #include "utils/relcache.h"
+struct RelOptInfo;				/* avoid including pathnodes.h here */
 
 
 /*
@@ -87,6 +88,14 @@ extern bool partition_bounds_equal(int partnatts, int16 *parttyplen,
 								   PartitionBoundInfo b2);
 extern PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src,
 												PartitionKey key);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+												 FmgrInfo *partsupfunc,
+												 Oid *partcollation,
+												 struct RelOptInfo *outer_rel,
+												 struct RelOptInfo *inner_rel,
+												 JoinType jointype,
+												 List **outer_parts,
+												 List **inner_parts);
 extern bool partitions_are_ordered(PartitionBoundInfo boundinfo, int nparts);
 extern void check_new_partition_bound(char *relname, Relation parent,
 									  PartitionBoundSpec *spec);
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index b3fbe47bde..27588c883e 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -2055,30 +2055,2606 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
                ->  Seq Scan on prt1_n_p2 t1_2
 (10 rows)
 
--- partitionwise join can not be applied if only one of joining tables has
--- default partition
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
-ANALYZE prt2;
+--
+-- Test advanced partition-matching algorithm for partitioned join
+--
+-- Tests for range-partitioned tables
+CREATE TABLE prt1_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_adv_p1 PARTITION OF prt1_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_adv_p2 PARTITION OF prt1_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_adv_p3 PARTITION OF prt1_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_adv_a_idx ON prt1_adv (a);
+INSERT INTO prt1_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_adv;
+CREATE TABLE prt2_adv (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_adv_p1 PARTITION OF prt2_adv FOR VALUES FROM (100) TO (150);
+CREATE TABLE prt2_adv_p2 PARTITION OF prt2_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt2_adv_p3 PARTITION OF prt2_adv FOR VALUES FROM (350) TO (500);
+CREATE INDEX prt2_adv_b_idx ON prt2_adv (b);
+INSERT INTO prt2_adv_p1 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 149) i;
+INSERT INTO prt2_adv_p2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(200, 299) i;
+INSERT INTO prt2_adv_p3 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_adv;
+-- inner join
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
  Sort
    Sort Key: t1.a
-   ->  Hash Join
-         Hash Cond: (t2.b = t1.a)
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 100 | 0 | 0100
+ 125 | 0 | 0125
+ 200 | 0 | 0200
+ 225 | 0 | 0225
+ 250 | 0 | 0250
+ 275 | 0 | 0275
+ 350 | 0 | 0350
+ 375 | 0 | 0375
+(8 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 |     | 
+ 175 | 0175 |     | 
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 300 | 0300 |     | 
+ 325 | 0325 |     | 
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(12 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 150 | 0 | 0150
+ 175 | 0 | 0175
+ 300 | 0 | 0300
+ 325 | 0 | 0325
+(4 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                                QUERY PLAN                                
+--------------------------------------------------------------------------
+ Sort
+   Sort Key: prt1_adv.a, prt2_adv.b
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_adv_1.a = prt2_adv_1.b)
+               Filter: (((175) = prt1_adv_1.a) OR ((425) = prt2_adv_1.b))
+               ->  Seq Scan on prt1_adv_p1 prt1_adv_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 prt2_adv_1
+                           Filter: (a = 0)
+         ->  Hash Full Join
+               Hash Cond: (prt1_adv_2.a = prt2_adv_2.b)
+               Filter: (((175) = prt1_adv_2.a) OR ((425) = prt2_adv_2.b))
+               ->  Seq Scan on prt1_adv_p2 prt1_adv_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 prt2_adv_2
+                           Filter: (a = 0)
+         ->  Hash Full Join
+               Hash Cond: (prt2_adv_3.b = prt1_adv_3.a)
+               Filter: (((175) = prt1_adv_3.a) OR ((425) = prt2_adv_3.b))
+               ->  Seq Scan on prt2_adv_p3 prt2_adv_3
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 prt1_adv_3
+                           Filter: (b = 0)
+(27 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 175 | 0175 |     | 
+     |      | 425 | 0425
+(2 rows)
+
+-- Test cases where one side has an extra partition
+CREATE TABLE prt2_adv_extra PARTITION OF prt2_adv FOR VALUES FROM (500) TO (MAXVALUE);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(500, 599) i;
+ANALYZE prt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 100 | 0 | 0100
+ 125 | 0 | 0125
+ 200 | 0 | 0200
+ 225 | 0 | 0225
+ 250 | 0 | 0250
+ 275 | 0 | 0275
+ 350 | 0 | 0350
+ 375 | 0 | 0375
+(8 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 |     | 
+ 175 | 0175 |     | 
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 300 | 0300 |     | 
+ 325 | 0325 |     | 
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(12 rows)
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t2.a
+   ->  Hash Right Join
+         Hash Cond: (t2.a = t1.b)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2_1
-               ->  Seq Scan on prt2_p2 t2_2
-               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt1_adv_p1 t2_1
+               ->  Seq Scan on prt1_adv_p2 t2_2
+               ->  Seq Scan on prt1_adv_p3 t2_3
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt2_adv_p1 t1_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_adv_p2 t1_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_adv_p3 t1_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_adv_extra t1_4
+                           Filter: (a = 0)
+(18 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 150 | 0 | 0150
+ 175 | 0 | 0175
+ 300 | 0 | 0300
+ 325 | 0 | 0325
+(4 rows)
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt1_adv t2 WHERE t1.b = t2.a) AND t1.a = 0 ORDER BY t1.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_extra t1_4
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 t2_1
+                     ->  Seq Scan on prt1_adv_p2 t2_2
+                     ->  Seq Scan on prt1_adv_p3 t2_3
+(18 rows)
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                           QUERY PLAN                           
+----------------------------------------------------------------
+ Sort
+   Sort Key: prt1_adv.a, prt2_adv.b
+   ->  Hash Full Join
+         Hash Cond: (prt2_adv.b = prt1_adv.a)
+         Filter: (((175) = prt1_adv.a) OR ((425) = prt2_adv.b))
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 prt2_adv_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p2 prt2_adv_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p3 prt2_adv_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_extra prt2_adv_4
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 prt1_adv_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_adv_p2 prt1_adv_2
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_adv_p3 prt1_adv_3
                            Filter: (b = 0)
+(22 rows)
+
+-- 3-way join where not every pair of relations can do partitioned join
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c, t3.a, t3.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) INNER JOIN prt1_adv t3 ON (t1.b = t3.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a, t3.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t2.a
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_adv_p1 t1_1
+                           Filter: (a = 0)
+                     ->  Index Scan using prt1_adv_p1_a_idx on prt1_adv_p1 t3_1
+                           Index Cond: (a = t1_1.b)
+               ->  Index Scan using prt1_adv_p1_a_idx on prt1_adv_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.a = t1_2.b)
+               ->  Seq Scan on prt1_adv_p2 t2_2
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_2.a = t1_2.b)
+                           ->  Seq Scan on prt1_adv_p2 t3_2
+                           ->  Hash
+                                 ->  Seq Scan on prt2_adv_p2 t1_2
+                                       Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_3.a = t1_3.b)
+               ->  Seq Scan on prt1_adv_p3 t2_3
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_3.a = t1_3.b)
+                           ->  Seq Scan on prt1_adv_p3 t3_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_adv_p3 t1_3
+                                       Filter: (a = 0)
+(31 rows)
+
+SELECT t1.b, t1.c, t2.a, t2.c, t3.a, t3.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) INNER JOIN prt1_adv t3 ON (t1.b = t3.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a, t3.a;
+  b  |  c   |  a  |  c   |  a  |  c   
+-----+------+-----+------+-----+------
+ 100 | 0100 | 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275 | 275 | 0275
+ 350 | 0350 | 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375 | 375 | 0375
+(8 rows)
+
+DROP TABLE prt2_adv_extra;
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in such cases
+ALTER TABLE prt2_adv DETACH PARTITION prt2_adv_p3;
+-- Split prt2_adv_p3 into two partitions so that prt1_adv_p3 matches both
+CREATE TABLE prt2_adv_p3_1 PARTITION OF prt2_adv FOR VALUES FROM (350) TO (375);
+CREATE TABLE prt2_adv_p3_2 PARTITION OF prt2_adv FOR VALUES FROM (375) TO (500);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Seq Scan on prt2_adv_p3_1 t2_3
+               ->  Seq Scan on prt2_adv_p3_2 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+                     ->  Seq Scan on prt2_adv_p3_1 t2_3
+                     ->  Seq Scan on prt2_adv_p3_2 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Seq Scan on prt2_adv_p3_1 t2_3
+               ->  Seq Scan on prt2_adv_p3_2 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(17 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+                     ->  Seq Scan on prt2_adv_p3_1 t2_3
+                     ->  Seq Scan on prt2_adv_p3_2 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                           QUERY PLAN                           
+----------------------------------------------------------------
+ Sort
+   Sort Key: prt1_adv.a, prt2_adv.b
+   ->  Hash Full Join
+         Hash Cond: (prt2_adv.b = prt1_adv.a)
+         Filter: (((175) = prt1_adv.a) OR ((425) = prt2_adv.b))
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 prt2_adv_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p2 prt2_adv_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p3_1 prt2_adv_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p3_2 prt2_adv_4
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 prt1_adv_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p2 prt1_adv_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 prt1_adv_3
+                           Filter: (b = 0)
+(22 rows)
+
+DROP TABLE prt2_adv_p3_1;
+DROP TABLE prt2_adv_p3_2;
+ANALYZE prt2_adv;
+-- Test default partitions
+ALTER TABLE prt1_adv DETACH PARTITION prt1_adv_p1;
+-- Change prt1_adv_p1 to the default partition
+ALTER TABLE prt1_adv ATTACH PARTITION prt1_adv_p1 DEFAULT;
+ALTER TABLE prt1_adv DETACH PARTITION prt1_adv_p3;
+ANALYZE prt1_adv;
+-- We can do partitioned join even if only one of relations has the default
+-- partition
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_1
+                           Filter: (b = 0)
+(15 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+(6 rows)
+
+-- Restore prt1_adv_p3
+ALTER TABLE prt1_adv ATTACH PARTITION prt1_adv_p3 FOR VALUES FROM (300) TO (400);
+ANALYZE prt1_adv;
+-- Restore prt2_adv_p3
+ALTER TABLE prt2_adv ATTACH PARTITION prt2_adv_p3 FOR VALUES FROM (350) TO (500);
+ANALYZE prt2_adv;
+-- Partitioned join can't be applied because the default partition of prt1_adv
+-- matches prt2_adv_p1 and prt2_adv_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Seq Scan on prt2_adv_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p2 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p1 t1_3
+                           Filter: (b = 0)
+(16 rows)
+
+ALTER TABLE prt2_adv DETACH PARTITION prt2_adv_p3;
+-- Change prt2_adv_p3 to the default partition
+ALTER TABLE prt2_adv ATTACH PARTITION prt2_adv_p3 DEFAULT;
+ANALYZE prt2_adv;
+-- Partitioned join can't be applied because the default partition of prt1_adv
+-- matches prt2_adv_p1 and prt2_adv_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Seq Scan on prt2_adv_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p2 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p1 t1_3
+                           Filter: (b = 0)
+(16 rows)
+
+DROP TABLE prt1_adv_p3;
+ANALYZE prt1_adv;
+DROP TABLE prt2_adv_p3;
+ANALYZE prt2_adv;
+CREATE TABLE prt3_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt3_adv_p1 PARTITION OF prt3_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt3_adv_p2 PARTITION OF prt3_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt3_adv_a_idx ON prt3_adv (a);
+INSERT INTO prt3_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(200, 399) i;
+ANALYZE prt3_adv;
+-- 3-way join to test the default partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) LEFT JOIN prt3_adv t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b, t3.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t3_1.a = t1_1.a)
+               ->  Seq Scan on prt3_adv_p1 t3_1
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_2.b = t1_1.a)
+                           ->  Seq Scan on prt2_adv_p2 t2_2
+                           ->  Hash
+                                 ->  Seq Scan on prt1_adv_p2 t1_1
+                                       Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t3_2.a = t1_2.a)
+               ->  Seq Scan on prt3_adv_p2 t3_2
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_1.b = t1_2.a)
+                           ->  Seq Scan on prt2_adv_p1 t2_1
+                           ->  Hash
+                                 ->  Seq Scan on prt1_adv_p1 t1_2
+                                       Filter: (b = 0)
+(23 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) LEFT JOIN prt3_adv t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+  a  |  c   |  b  |  c   |  a  |  c   
+-----+------+-----+------+-----+------
+ 100 | 0100 | 100 | 0100 |     | 
+ 125 | 0125 | 125 | 0125 |     | 
+ 150 | 0150 |     |      |     | 
+ 175 | 0175 |     |      |     | 
+ 200 | 0200 | 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275 | 275 | 0275
+(8 rows)
+
+DROP TABLE prt1_adv;
+DROP TABLE prt2_adv;
+DROP TABLE prt3_adv;
+-- Test interaction of partitioned join with partition pruning
+CREATE TABLE prt1_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_adv_p1 PARTITION OF prt1_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_adv_p2 PARTITION OF prt1_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_adv_p3 PARTITION OF prt1_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_adv_a_idx ON prt1_adv (a);
+INSERT INTO prt1_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_adv;
+CREATE TABLE prt2_adv (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_adv_p1 PARTITION OF prt2_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt2_adv_p2 PARTITION OF prt2_adv FOR VALUES FROM (200) TO (400);
+CREATE INDEX prt2_adv_b_idx ON prt2_adv (b);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+                        QUERY PLAN                         
+-----------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: ((a < 300) AND (b = 0))
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: ((a < 300) AND (b = 0))
+(15 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 | 150 | 0150
+ 175 | 0175 | 175 | 0175
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+(8 rows)
+
+DROP TABLE prt1_adv_p3;
+CREATE TABLE prt1_adv_default PARTITION OF prt1_adv DEFAULT;
+ANALYZE prt1_adv;
+CREATE TABLE prt2_adv_default PARTITION OF prt2_adv DEFAULT;
+ANALYZE prt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+                                QUERY PLAN                                
+--------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: ((a >= 100) AND (a < 300) AND (b = 0))
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: ((a >= 100) AND (a < 300) AND (b = 0))
+(15 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 | 150 | 0150
+ 175 | 0175 | 175 | 0175
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+(8 rows)
+
+DROP TABLE prt1_adv;
+DROP TABLE prt2_adv;
+-- Tests for list-partitioned tables
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0001', '0003');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt1_adv_p3 PARTITION OF plt1_adv FOR VALUES IN ('0008', '0009');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+ANALYZE plt1_adv;
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt2_adv_p3 PARTITION OF plt2_adv FOR VALUES IN ('0007', '0009');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+(6 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 1 | 1 | 0001
+ 8 | 8 | 0008
+(2 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+   |      | 2 | 0002
+   |      | 7 | 0007
+(8 rows)
+
+-- Test cases where one side has an extra partition
+CREATE TABLE plt2_adv_extra PARTITION OF plt2_adv FOR VALUES IN ('0000');
+INSERT INTO plt2_adv_extra VALUES (0, 0, '0000');
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+(6 rows)
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2_adv t1 LEFT JOIN plt1_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt1_adv_p1 t2_1
+               ->  Seq Scan on plt1_adv_p2 t2_2
+               ->  Seq Scan on plt1_adv_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_adv_extra t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_adv_p1 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_adv_p2 t1_3
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_adv_p3 t1_4
+                           Filter: (b < 10)
+(18 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 1 | 1 | 0001
+ 8 | 8 | 0008
+(2 rows)
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt2_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt1_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Anti Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_extra t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t1_4
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t2_1
+                     ->  Seq Scan on plt1_adv_p2 t2_2
+                     ->  Seq Scan on plt1_adv_p3 t2_3
+(18 rows)
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt2_adv_extra t2_1
+               ->  Seq Scan on plt2_adv_p1 t2_2
+               ->  Seq Scan on plt2_adv_p2 t2_3
+               ->  Seq Scan on plt2_adv_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+(15 rows)
+
+DROP TABLE plt2_adv_extra;
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in such cases
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2;
+-- Split plt2_adv_p2 into two partitions so that plt1_adv_p2 matches both
+CREATE TABLE plt2_adv_p2_1 PARTITION OF plt2_adv FOR VALUES IN ('0004');
+CREATE TABLE plt2_adv_p2_2 PARTITION OF plt2_adv FOR VALUES IN ('0006');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6);
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_1 t2_2
+               ->  Seq Scan on plt2_adv_p2_2 t2_3
+               ->  Seq Scan on plt2_adv_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Semi Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+                     ->  Seq Scan on plt2_adv_p2_1 t2_2
+                     ->  Seq Scan on plt2_adv_p2_2 t2_3
+                     ->  Seq Scan on plt2_adv_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_1 t2_2
+               ->  Seq Scan on plt2_adv_p2_2 t2_3
+               ->  Seq Scan on plt2_adv_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(17 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Anti Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+                     ->  Seq Scan on plt2_adv_p2_1 t2_2
+                     ->  Seq Scan on plt2_adv_p2_2 t2_3
+                     ->  Seq Scan on plt2_adv_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_1 t2_2
+               ->  Seq Scan on plt2_adv_p2_2 t2_3
+               ->  Seq Scan on plt2_adv_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+(15 rows)
+
+DROP TABLE plt2_adv_p2_1;
+DROP TABLE plt2_adv_p2_2;
+-- Restore plt2_adv_p2
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2 FOR VALUES IN ('0004', '0006');
+-- Test NULL partitions
+ALTER TABLE plt1_adv DETACH PARTITION plt1_adv_p1;
+-- Change plt1_adv_p1 to the NULL partition
+CREATE TABLE plt1_adv_p1_null PARTITION OF plt1_adv FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p3;
+-- Change plt2_adv_p3 to the NULL partition
+CREATE TABLE plt2_adv_p3_null PARTITION OF plt2_adv FOR VALUES IN (NULL, '0007', '0009');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (7, 9);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1_null t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1_null t1_1
+                     Filter: (b < 10)
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+(19 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1_null t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a  |  c   | a |  c   
+----+------+---+------
+ -1 |      |   | 
+  1 | 0001 |   | 
+  3 | 0003 | 3 | 0003
+  4 | 0004 | 4 | 0004
+  6 | 0006 | 6 | 0006
+  8 | 0008 |   | 
+  9 | 0009 | 9 | 0009
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1_null t1_1
+                     Filter: (b < 10)
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+(19 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a  | b  |  c   
+----+----+------
+ -1 | -1 | 
+  1 |  1 | 0001
+  8 |  8 | 0008
+(3 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p1_null t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+ -1 |      |    | 
+  1 | 0001 |    | 
+  3 | 0003 |  3 | 0003
+  4 | 0004 |  4 | 0004
+  6 | 0006 |  6 | 0006
+  8 | 0008 |    | 
+  9 | 0009 |  9 | 0009
+    |      | -1 | 
+    |      |  2 | 0002
+    |      |  7 | 0007
+(10 rows)
+
+DROP TABLE plt1_adv_p1_null;
+-- Restore plt1_adv_p1
+ALTER TABLE plt1_adv ATTACH PARTITION plt1_adv_p1 FOR VALUES IN ('0001', '0003');
+-- Add to plt1_adv the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt1_adv_extra PARTITION OF plt1_adv FOR VALUES IN (NULL);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+DROP TABLE plt2_adv_p3_null;
+-- Restore plt2_adv_p3
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p3 FOR VALUES IN ('0007', '0009');
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Seq Scan on plt2_adv_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_extra t1_4
+                           Filter: (b < 10)
+(18 rows)
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt1_adv_p1 t1_1
+               ->  Seq Scan on plt1_adv_p2 t1_2
+               ->  Seq Scan on plt1_adv_p3 t1_3
+               ->  Seq Scan on plt1_adv_extra t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+                     ->  Seq Scan on plt2_adv_p2 t2_2
+                     ->  Seq Scan on plt2_adv_p3 t2_3
+(15 rows)
+
+-- Add to plt2_adv the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt2_adv_extra PARTITION OF plt2_adv FOR VALUES IN (NULL);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+         ->  Nested Loop Left Join
+               Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+               ->  Seq Scan on plt1_adv_extra t1_4
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_extra t2_4
+(26 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a  |  c   | a |  c   
+----+------+---+------
+ -1 |      |   | 
+  1 | 0001 |   | 
+  3 | 0003 | 3 | 0003
+  4 | 0004 | 4 | 0004
+  6 | 0006 | 6 | 0006
+  8 | 0008 |   | 
+  9 | 0009 | 9 | 0009
+(7 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p3 t2_3
+         ->  Hash Full Join
+               Hash Cond: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+               Filter: ((COALESCE(t1_4.b, 0) < 10) AND (COALESCE(t2_4.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_extra t1_4
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_extra t2_4
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+ -1 |      |    | 
+  1 | 0001 |    | 
+  3 | 0003 |  3 | 0003
+  4 | 0004 |  4 | 0004
+  6 | 0006 |  6 | 0006
+  8 | 0008 |    | 
+  9 | 0009 |  9 | 0009
+    |      | -1 | 
+    |      |  2 | 0002
+    |      |  7 | 0007
+(10 rows)
+
+-- 3-way join to test the NULL partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt1_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t3_1.a = t1_1.a) AND (t3_1.c = t1_1.c))
+               ->  Seq Scan on plt1_adv_p1 t3_1
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+                           ->  Seq Scan on plt2_adv_p1 t2_1
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p1 t1_1
+                                       Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t3_2.a = t1_2.a) AND (t3_2.c = t1_2.c))
+               ->  Seq Scan on plt1_adv_p2 t3_2
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+                           ->  Seq Scan on plt2_adv_p2 t2_2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p2 t1_2
+                                       Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t3_3.a = t1_3.a) AND (t3_3.c = t1_3.c))
+               ->  Seq Scan on plt1_adv_p3 t3_3
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+                           ->  Seq Scan on plt2_adv_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p3 t1_3
+                                       Filter: (b < 10)
+         ->  Nested Loop Left Join
+               Join Filter: ((t1_4.a = t3_4.a) AND (t1_4.c = t3_4.c))
+               ->  Nested Loop Left Join
+                     Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+                     ->  Seq Scan on plt1_adv_extra t1_4
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_adv_extra t2_4
+               ->  Seq Scan on plt1_adv_extra t3_4
+(41 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt1_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a  |  c   | a |  c   | a |  c   
+----+------+---+------+---+------
+ -1 |      |   |      |   | 
+  1 | 0001 |   |      | 1 | 0001
+  3 | 0003 | 3 | 0003 | 3 | 0003
+  4 | 0004 | 4 | 0004 | 4 | 0004
+  6 | 0006 | 6 | 0006 | 6 | 0006
+  8 | 0008 |   |      | 8 | 0008
+  9 | 0009 | 9 | 0009 | 9 | 0009
+(7 rows)
+
+DROP TABLE plt1_adv_extra;
+DROP TABLE plt2_adv_extra;
+-- Test default partitions
+ALTER TABLE plt1_adv DETACH PARTITION plt1_adv_p1;
+-- Change plt1_adv_p1 to the default partition
+ALTER TABLE plt1_adv ATTACH PARTITION plt1_adv_p1 DEFAULT;
+DROP TABLE plt1_adv_p3;
+ANALYZE plt1_adv;
+DROP TABLE plt2_adv_p3;
+ANALYZE plt2_adv;
+-- We can do partitioned join even if only one of relations has the default
+-- partition
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_1
+                           Filter: (b < 10)
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(3 rows)
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2;
+-- Change plt2_adv_p2 to contain '0005' in addition to '0004' and '0006' as
+-- the key values
+CREATE TABLE plt2_adv_p2_ext PARTITION OF plt2_adv FOR VALUES IN ('0004', '0005', '0006');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 5, 6);
+ANALYZE plt2_adv;
+-- Partitioned join can't be applied because the default partition of plt1_adv
+-- matches plt2_adv_p1 and plt2_adv_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_ext t2_2
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p2 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p1 t1_2
+                           Filter: (b < 10)
+(13 rows)
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2_ext;
+-- Change plt2_adv_p2_ext to the default partition
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2_ext DEFAULT;
+ANALYZE plt2_adv;
+-- Partitioned join can't be applied because the default partition of plt1_adv
+-- matches plt2_adv_p1 and plt2_adv_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_ext t2_2
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p2 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p1 t1_2
+                           Filter: (b < 10)
+(13 rows)
+
+DROP TABLE plt2_adv_p2_ext;
+-- Restore plt2_adv_p2
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_adv;
+CREATE TABLE plt3_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_adv_p1 PARTITION OF plt3_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt3_adv_p2 PARTITION OF plt3_adv FOR VALUES IN ('0007', '0009');
+INSERT INTO plt3_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6, 7, 9);
+ANALYZE plt3_adv;
+-- 3-way join to test the default partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t3_1.a = t1_1.a) AND (t3_1.c = t1_1.c))
+               ->  Seq Scan on plt3_adv_p1 t3_1
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+                           ->  Seq Scan on plt2_adv_p2 t2_2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p2 t1_1
+                                       Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t3_2.a = t1_2.a) AND (t3_2.c = t1_2.c))
+               ->  Seq Scan on plt3_adv_p2 t3_2
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+                           ->  Seq Scan on plt2_adv_p1 t2_1
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p1 t1_2
+                                       Filter: (b < 10)
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   | a |  c   
+---+------+---+------+---+------
+ 1 | 0001 |   |      |   | 
+ 3 | 0003 | 3 | 0003 |   | 
+ 4 | 0004 | 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006 | 6 | 0006
+(4 rows)
+
+-- Test cases where one side has the default partition while the other side
+-- has the NULL partition
+DROP TABLE plt2_adv_p1;
+-- Add the NULL partition to plt2_adv
+CREATE TABLE plt2_adv_p1_null PARTITION OF plt2_adv FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p1_null t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_1
+                           Filter: (b < 10)
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 | 1 | 0001
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(4 rows)
+
+DROP TABLE plt2_adv_p1_null;
+-- Add the NULL partition that contains only NULL values as the key values
+CREATE TABLE plt2_adv_p1_null PARTITION OF plt2_adv FOR VALUES IN (NULL);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Seq Scan on plt2_adv_p2 t2
+         ->  Hash
+               ->  Seq Scan on plt1_adv_p2 t1
+                     Filter: (b < 10)
+(8 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(2 rows)
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+DROP TABLE plt3_adv;
+-- Test interaction of partitioned join with partition pruning
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0001');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0002');
+CREATE TABLE plt1_adv_p3 PARTITION OF plt1_adv FOR VALUES IN ('0003');
+CREATE TABLE plt1_adv_p4 PARTITION OF plt1_adv FOR VALUES IN (NULL, '0004', '0005');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN (NULL);
+CREATE TABLE plt2_adv_p3 PARTITION OF plt2_adv FOR VALUES IN ('0003');
+CREATE TABLE plt2_adv_p4 PARTITION OF plt2_adv FOR VALUES IN ('0004', '0005');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+                                       QUERY PLAN                                        
+-----------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p3 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_1
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p4 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p4 t1_2
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 5 | 0005 | 5 | 0005
+(3 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Seq Scan on plt2_adv_p4 t2
+         ->  Hash
+               ->  Seq Scan on plt1_adv_p4 t1
+                     Filter: ((c IS NULL) AND (b < 10))
+(8 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+ a  | c | a | c 
+----+---+---+---
+ -1 |   |   | 
+(1 row)
+
+CREATE TABLE plt1_adv_default PARTITION OF plt1_adv DEFAULT;
+ANALYZE plt1_adv;
+CREATE TABLE plt2_adv_default PARTITION OF plt2_adv DEFAULT;
+ANALYZE plt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+                                       QUERY PLAN                                        
+-----------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p3 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_1
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p4 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p4 t1_2
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 5 | 0005 | 5 | 0005
+(3 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Seq Scan on plt2_adv_p4 t2
+         ->  Hash
+               ->  Seq Scan on plt1_adv_p4 t1
+                     Filter: ((c IS NULL) AND (b < 10))
+(8 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+ a  | c | a | c 
+----+---+---+---
+ -1 |   |   | 
+(1 row)
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+-- Test the process_outer_partition() code path
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1_adv;
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0002');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2_adv;
+CREATE TABLE plt3_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_adv_p1 PARTITION OF plt3_adv FOR VALUES IN ('0001');
+CREATE TABLE plt3_adv_p2 PARTITION OF plt3_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3_adv;
+-- This tests that when merging partitions from plt1_adv and plt2_adv in
+-- merge_list_bounds(), process_outer_partition() returns an already-assigned
+-- merged partition when re-called with plt1_adv_p1 for the second list value
+-- '0001' of that partitin
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.c = t2.c)) FULL JOIN plt3_adv t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2.a, t3.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t3_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_1.c = t2_1.c)
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                     ->  Hash
+                           ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt3_adv_p1 t3_1
+         ->  Hash Full Join
+               Hash Cond: (t1_2.c = t3_2.c)
+               Filter: (((COALESCE(t1_2.a, 0) % 5) <> 3) AND ((COALESCE(t1_2.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_2.c = t2_2.c)
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                     ->  Hash
+                           ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt3_adv_p2 t3_2
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.c = t2.c)) FULL JOIN plt3_adv t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+ a  |  c   | a  |  c   | a  |  c   
+----+------+----+------+----+------
+  0 | 0000 |    |      |    | 
+  5 | 0000 |    |      |    | 
+ 10 | 0000 |    |      |    | 
+ 15 | 0000 |    |      |    | 
+ 20 | 0000 |    |      |    | 
+  1 | 0001 |    |      |  1 | 0001
+  1 | 0001 |    |      |  6 | 0001
+  1 | 0001 |    |      | 11 | 0001
+  1 | 0001 |    |      | 16 | 0001
+  1 | 0001 |    |      | 21 | 0001
+  6 | 0001 |    |      |  1 | 0001
+  6 | 0001 |    |      |  6 | 0001
+  6 | 0001 |    |      | 11 | 0001
+  6 | 0001 |    |      | 16 | 0001
+  6 | 0001 |    |      | 21 | 0001
+ 11 | 0001 |    |      |  1 | 0001
+ 11 | 0001 |    |      |  6 | 0001
+ 11 | 0001 |    |      | 11 | 0001
+ 11 | 0001 |    |      | 16 | 0001
+ 11 | 0001 |    |      | 21 | 0001
+ 16 | 0001 |    |      |  1 | 0001
+ 16 | 0001 |    |      |  6 | 0001
+ 16 | 0001 |    |      | 11 | 0001
+ 16 | 0001 |    |      | 16 | 0001
+ 16 | 0001 |    |      | 21 | 0001
+ 21 | 0001 |    |      |  1 | 0001
+ 21 | 0001 |    |      |  6 | 0001
+ 21 | 0001 |    |      | 11 | 0001
+ 21 | 0001 |    |      | 16 | 0001
+ 21 | 0001 |    |      | 21 | 0001
+  2 | 0002 |  2 | 0002 |    | 
+  2 | 0002 |  7 | 0002 |    | 
+  2 | 0002 | 12 | 0002 |    | 
+  2 | 0002 | 17 | 0002 |    | 
+  2 | 0002 | 22 | 0002 |    | 
+  7 | 0002 |  2 | 0002 |    | 
+  7 | 0002 |  7 | 0002 |    | 
+  7 | 0002 | 12 | 0002 |    | 
+  7 | 0002 | 17 | 0002 |    | 
+  7 | 0002 | 22 | 0002 |    | 
+ 12 | 0002 |  2 | 0002 |    | 
+ 12 | 0002 |  7 | 0002 |    | 
+ 12 | 0002 | 12 | 0002 |    | 
+ 12 | 0002 | 17 | 0002 |    | 
+ 12 | 0002 | 22 | 0002 |    | 
+ 17 | 0002 |  2 | 0002 |    | 
+ 17 | 0002 |  7 | 0002 |    | 
+ 17 | 0002 | 12 | 0002 |    | 
+ 17 | 0002 | 17 | 0002 |    | 
+ 17 | 0002 | 22 | 0002 |    | 
+ 22 | 0002 |  2 | 0002 |    | 
+ 22 | 0002 |  7 | 0002 |    | 
+ 22 | 0002 | 12 | 0002 |    | 
+ 22 | 0002 | 17 | 0002 |    | 
+ 22 | 0002 | 22 | 0002 |    | 
+(55 rows)
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+DROP TABLE plt3_adv;
+-- Tests for multi-level partitioned tables
+CREATE TABLE alpha (a double precision, b int, c text) PARTITION BY RANGE (a);
+CREATE TABLE alpha_neg PARTITION OF alpha FOR VALUES FROM ('-Infinity') TO (0) PARTITION BY RANGE (b);
+CREATE TABLE alpha_pos PARTITION OF alpha FOR VALUES FROM (0) TO (10.0) PARTITION BY LIST (c);
+CREATE TABLE alpha_neg_p1 PARTITION OF alpha_neg FOR VALUES FROM (100) TO (200);
+CREATE TABLE alpha_neg_p2 PARTITION OF alpha_neg FOR VALUES FROM (200) TO (300);
+CREATE TABLE alpha_neg_p3 PARTITION OF alpha_neg FOR VALUES FROM (300) TO (400);
+CREATE TABLE alpha_pos_p1 PARTITION OF alpha_pos FOR VALUES IN ('0001', '0003');
+CREATE TABLE alpha_pos_p2 PARTITION OF alpha_pos FOR VALUES IN ('0004', '0006');
+CREATE TABLE alpha_pos_p3 PARTITION OF alpha_pos FOR VALUES IN ('0008', '0009');
+INSERT INTO alpha_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+INSERT INTO alpha_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+ANALYZE alpha;
+CREATE TABLE beta (a double precision, b int, c text) PARTITION BY RANGE (a);
+CREATE TABLE beta_neg PARTITION OF beta FOR VALUES FROM (-10.0) TO (0) PARTITION BY RANGE (b);
+CREATE TABLE beta_pos PARTITION OF beta FOR VALUES FROM (0) TO ('Infinity') PARTITION BY LIST (c);
+CREATE TABLE beta_neg_p1 PARTITION OF beta_neg FOR VALUES FROM (100) TO (150);
+CREATE TABLE beta_neg_p2 PARTITION OF beta_neg FOR VALUES FROM (200) TO (300);
+CREATE TABLE beta_neg_p3 PARTITION OF beta_neg FOR VALUES FROM (350) TO (500);
+CREATE TABLE beta_pos_p1 PARTITION OF beta_pos FOR VALUES IN ('0002', '0003');
+CREATE TABLE beta_pos_p2 PARTITION OF beta_pos FOR VALUES IN ('0004', '0006');
+CREATE TABLE beta_pos_p3 PARTITION OF beta_pos FOR VALUES IN ('0007', '0009');
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+ANALYZE beta;
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.b
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.b = t2_1.b))
+               ->  Seq Scan on alpha_neg_p1 t1_1
+                     Filter: ((b >= 125) AND (b < 225))
+               ->  Hash
+                     ->  Seq Scan on beta_neg_p1 t2_1
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.b = t1_2.b))
+               ->  Seq Scan on beta_neg_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on alpha_neg_p2 t1_2
+                           Filter: ((b >= 125) AND (b < 225))
+         ->  Hash Join
+               Hash Cond: ((t2_4.a = t1_4.a) AND (t2_4.b = t1_4.b))
+               ->  Append
+                     ->  Seq Scan on beta_pos_p1 t2_4
+                     ->  Seq Scan on beta_pos_p2 t2_5
+                     ->  Seq Scan on beta_pos_p3 t2_6
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on alpha_pos_p1 t1_4
+                                 Filter: ((b >= 125) AND (b < 225))
+                           ->  Seq Scan on alpha_pos_p2 t1_5
+                                 Filter: ((b >= 125) AND (b < 225))
+                           ->  Seq Scan on alpha_pos_p3 t1_6
+                                 Filter: ((b >= 125) AND (b < 225))
+(29 rows)
+
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b;
+ a  |  b  |  c   | a  |  b  |  c   
+----+-----+------+----+-----+------
+ -1 | 126 | 0006 | -1 | 126 | 0006
+ -1 | 129 | 0009 | -1 | 129 | 0009
+ -1 | 133 | 0003 | -1 | 133 | 0003
+ -1 | 134 | 0004 | -1 | 134 | 0004
+ -1 | 136 | 0006 | -1 | 136 | 0006
+ -1 | 139 | 0009 | -1 | 139 | 0009
+ -1 | 143 | 0003 | -1 | 143 | 0003
+ -1 | 144 | 0004 | -1 | 144 | 0004
+ -1 | 146 | 0006 | -1 | 146 | 0006
+ -1 | 149 | 0009 | -1 | 149 | 0009
+ -1 | 203 | 0003 | -1 | 203 | 0003
+ -1 | 204 | 0004 | -1 | 204 | 0004
+ -1 | 206 | 0006 | -1 | 206 | 0006
+ -1 | 209 | 0009 | -1 | 209 | 0009
+ -1 | 213 | 0003 | -1 | 213 | 0003
+ -1 | 214 | 0004 | -1 | 214 | 0004
+ -1 | 216 | 0006 | -1 | 216 | 0006
+ -1 | 219 | 0009 | -1 | 219 | 0009
+ -1 | 223 | 0003 | -1 | 223 | 0003
+ -1 | 224 | 0004 | -1 | 224 | 0004
+  1 | 126 | 0006 |  1 | 126 | 0006
+  1 | 129 | 0009 |  1 | 129 | 0009
+  1 | 133 | 0003 |  1 | 133 | 0003
+  1 | 134 | 0004 |  1 | 134 | 0004
+  1 | 136 | 0006 |  1 | 136 | 0006
+  1 | 139 | 0009 |  1 | 139 | 0009
+  1 | 143 | 0003 |  1 | 143 | 0003
+  1 | 144 | 0004 |  1 | 144 | 0004
+  1 | 146 | 0006 |  1 | 146 | 0006
+  1 | 149 | 0009 |  1 | 149 | 0009
+  1 | 203 | 0003 |  1 | 203 | 0003
+  1 | 204 | 0004 |  1 | 204 | 0004
+  1 | 206 | 0006 |  1 | 206 | 0006
+  1 | 209 | 0009 |  1 | 209 | 0009
+  1 | 213 | 0003 |  1 | 213 | 0003
+  1 | 214 | 0004 |  1 | 214 | 0004
+  1 | 216 | 0006 |  1 | 216 | 0006
+  1 | 219 | 0009 |  1 | 219 | 0009
+  1 | 223 | 0003 |  1 | 223 | 0003
+  1 | 224 | 0004 |  1 | 224 | 0004
+(40 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b, t2.b;
+                                                              QUERY PLAN                                                              
+--------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.b, t2.b
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Append
+                     ->  Seq Scan on alpha_neg_p1 t1_2
+                           Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+                     ->  Seq Scan on alpha_neg_p2 t1_3
+                           Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on beta_neg_p1 t2_2
+                                 Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+                           ->  Seq Scan on beta_neg_p2 t2_3
+                                 Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Nested Loop
+               Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+               ->  Seq Scan on alpha_pos_p2 t1_4
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Seq Scan on beta_pos_p2 t2_4
+                     Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Nested Loop
+               Join Filter: ((t1_5.a = t2_5.a) AND (t1_5.c = t2_5.c))
+               ->  Seq Scan on alpha_pos_p3 t1_5
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Seq Scan on beta_pos_p3 t2_5
+                     Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+(28 rows)
+
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b, t2.b;
+ a  |  b  |  c   | a  |  b  |  c   
+----+-----+------+----+-----+------
+ -1 | 104 | 0004 | -1 | 104 | 0004
+ -1 | 104 | 0004 | -1 | 204 | 0004
+ -1 | 109 | 0009 | -1 | 109 | 0009
+ -1 | 109 | 0009 | -1 | 209 | 0009
+ -1 | 204 | 0004 | -1 | 104 | 0004
+ -1 | 204 | 0004 | -1 | 204 | 0004
+ -1 | 209 | 0009 | -1 | 109 | 0009
+ -1 | 209 | 0009 | -1 | 209 | 0009
+  1 | 104 | 0004 |  1 | 104 | 0004
+  1 | 104 | 0004 |  1 | 204 | 0004
+  1 | 109 | 0009 |  1 | 109 | 0009
+  1 | 109 | 0009 |  1 | 209 | 0009
+  1 | 204 | 0004 |  1 | 104 | 0004
+  1 | 204 | 0004 |  1 | 204 | 0004
+  1 | 209 | 0009 |  1 | 109 | 0009
+  1 | 209 | 0009 |  1 | 209 | 0009
 (16 rows)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;
+                                                           QUERY PLAN                                                           
+--------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.b
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on alpha_neg_p1 t1_1
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Hash
+                     ->  Seq Scan on beta_neg_p1 t2_1
+                           Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Hash Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on alpha_neg_p2 t1_2
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Hash
+                     ->  Seq Scan on beta_neg_p2 t2_2
+                           Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Nested Loop
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.b = t2_3.b) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on alpha_pos_p2 t1_3
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Seq Scan on beta_pos_p2 t2_3
+                     Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Nested Loop
+               Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.b = t2_4.b) AND (t1_4.c = t2_4.c))
+               ->  Seq Scan on alpha_pos_p3 t1_4
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Seq Scan on beta_pos_p3 t2_4
+                     Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+(29 rows)
+
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;
+ a  |  b  |  c   | a  |  b  |  c   
+----+-----+------+----+-----+------
+ -1 | 104 | 0004 | -1 | 104 | 0004
+ -1 | 109 | 0009 | -1 | 109 | 0009
+ -1 | 204 | 0004 | -1 | 204 | 0004
+ -1 | 209 | 0009 | -1 | 209 | 0009
+  1 | 104 | 0004 |  1 | 104 | 0004
+  1 | 109 | 0009 |  1 | 109 | 0009
+  1 | 204 | 0004 |  1 | 204 | 0004
+  1 | 209 | 0009 |  1 | 209 | 0009
+(8 rows)
+
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 575ba7b8d4..df30f851e8 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -445,11 +445,638 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
 
--- partitionwise join can not be applied if only one of joining tables has
--- default partition
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
-ANALYZE prt2;
 
+--
+-- Test advanced partition-matching algorithm for partitioned join
+--
+
+-- Tests for range-partitioned tables
+CREATE TABLE prt1_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_adv_p1 PARTITION OF prt1_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_adv_p2 PARTITION OF prt1_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_adv_p3 PARTITION OF prt1_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_adv_a_idx ON prt1_adv (a);
+INSERT INTO prt1_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_adv;
+
+CREATE TABLE prt2_adv (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_adv_p1 PARTITION OF prt2_adv FOR VALUES FROM (100) TO (150);
+CREATE TABLE prt2_adv_p2 PARTITION OF prt2_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt2_adv_p3 PARTITION OF prt2_adv FOR VALUES FROM (350) TO (500);
+CREATE INDEX prt2_adv_b_idx ON prt2_adv (b);
+INSERT INTO prt2_adv_p1 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 149) i;
+INSERT INTO prt2_adv_p2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(200, 299) i;
+INSERT INTO prt2_adv_p3 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_adv;
+
+-- inner join
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+-- Test cases where one side has an extra partition
+CREATE TABLE prt2_adv_extra PARTITION OF prt2_adv FOR VALUES FROM (500) TO (MAXVALUE);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(500, 599) i;
+ANALYZE prt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt1_adv t2 WHERE t1.b = t2.a) AND t1.a = 0 ORDER BY t1.b;
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+-- 3-way join where not every pair of relations can do partitioned join
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c, t3.a, t3.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) INNER JOIN prt1_adv t3 ON (t1.b = t3.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a, t3.a;
+SELECT t1.b, t1.c, t2.a, t2.c, t3.a, t3.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) INNER JOIN prt1_adv t3 ON (t1.b = t3.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a, t3.a;
+
+DROP TABLE prt2_adv_extra;
+
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in such cases
+ALTER TABLE prt2_adv DETACH PARTITION prt2_adv_p3;
+-- Split prt2_adv_p3 into two partitions so that prt1_adv_p3 matches both
+CREATE TABLE prt2_adv_p3_1 PARTITION OF prt2_adv FOR VALUES FROM (350) TO (375);
+CREATE TABLE prt2_adv_p3_2 PARTITION OF prt2_adv FOR VALUES FROM (375) TO (500);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+DROP TABLE prt2_adv_p3_1;
+DROP TABLE prt2_adv_p3_2;
+ANALYZE prt2_adv;
+
+-- Test default partitions
+ALTER TABLE prt1_adv DETACH PARTITION prt1_adv_p1;
+-- Change prt1_adv_p1 to the default partition
+ALTER TABLE prt1_adv ATTACH PARTITION prt1_adv_p1 DEFAULT;
+ALTER TABLE prt1_adv DETACH PARTITION prt1_adv_p3;
+ANALYZE prt1_adv;
+
+-- We can do partitioned join even if only one of relations has the default
+-- partition
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- Restore prt1_adv_p3
+ALTER TABLE prt1_adv ATTACH PARTITION prt1_adv_p3 FOR VALUES FROM (300) TO (400);
+ANALYZE prt1_adv;
+
+-- Restore prt2_adv_p3
+ALTER TABLE prt2_adv ATTACH PARTITION prt2_adv_p3 FOR VALUES FROM (350) TO (500);
+ANALYZE prt2_adv;
+
+-- Partitioned join can't be applied because the default partition of prt1_adv
+-- matches prt2_adv_p1 and prt2_adv_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+ALTER TABLE prt2_adv DETACH PARTITION prt2_adv_p3;
+-- Change prt2_adv_p3 to the default partition
+ALTER TABLE prt2_adv ATTACH PARTITION prt2_adv_p3 DEFAULT;
+ANALYZE prt2_adv;
+
+-- Partitioned join can't be applied because the default partition of prt1_adv
+-- matches prt2_adv_p1 and prt2_adv_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt1_adv_p3;
+ANALYZE prt1_adv;
+
+DROP TABLE prt2_adv_p3;
+ANALYZE prt2_adv;
+
+CREATE TABLE prt3_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt3_adv_p1 PARTITION OF prt3_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt3_adv_p2 PARTITION OF prt3_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt3_adv_a_idx ON prt3_adv (a);
+INSERT INTO prt3_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(200, 399) i;
+ANALYZE prt3_adv;
+
+-- 3-way join to test the default partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) LEFT JOIN prt3_adv t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) LEFT JOIN prt3_adv t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+
+DROP TABLE prt1_adv;
+DROP TABLE prt2_adv;
+DROP TABLE prt3_adv;
+
+-- Test interaction of partitioned join with partition pruning
+CREATE TABLE prt1_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_adv_p1 PARTITION OF prt1_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_adv_p2 PARTITION OF prt1_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_adv_p3 PARTITION OF prt1_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_adv_a_idx ON prt1_adv (a);
+INSERT INTO prt1_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_adv;
+
+CREATE TABLE prt2_adv (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_adv_p1 PARTITION OF prt2_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt2_adv_p2 PARTITION OF prt2_adv FOR VALUES FROM (200) TO (400);
+CREATE INDEX prt2_adv_b_idx ON prt2_adv (b);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt1_adv_p3;
+CREATE TABLE prt1_adv_default PARTITION OF prt1_adv DEFAULT;
+ANALYZE prt1_adv;
+
+CREATE TABLE prt2_adv_default PARTITION OF prt2_adv DEFAULT;
+ANALYZE prt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt1_adv;
+DROP TABLE prt2_adv;
+
+
+-- Tests for list-partitioned tables
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0001', '0003');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt1_adv_p3 PARTITION OF plt1_adv FOR VALUES IN ('0008', '0009');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+ANALYZE plt1_adv;
+
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt2_adv_p3 PARTITION OF plt2_adv FOR VALUES IN ('0007', '0009');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+-- Test cases where one side has an extra partition
+CREATE TABLE plt2_adv_extra PARTITION OF plt2_adv FOR VALUES IN ('0000');
+INSERT INTO plt2_adv_extra VALUES (0, 0, '0000');
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2_adv t1 LEFT JOIN plt1_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt2_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt1_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt2_adv_extra;
+
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in such cases
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2;
+-- Split plt2_adv_p2 into two partitions so that plt1_adv_p2 matches both
+CREATE TABLE plt2_adv_p2_1 PARTITION OF plt2_adv FOR VALUES IN ('0004');
+CREATE TABLE plt2_adv_p2_2 PARTITION OF plt2_adv FOR VALUES IN ('0006');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6);
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt2_adv_p2_1;
+DROP TABLE plt2_adv_p2_2;
+-- Restore plt2_adv_p2
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2 FOR VALUES IN ('0004', '0006');
+
+-- Test NULL partitions
+ALTER TABLE plt1_adv DETACH PARTITION plt1_adv_p1;
+-- Change plt1_adv_p1 to the NULL partition
+CREATE TABLE plt1_adv_p1_null PARTITION OF plt1_adv FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p3;
+-- Change plt2_adv_p3 to the NULL partition
+CREATE TABLE plt2_adv_p3_null PARTITION OF plt2_adv FOR VALUES IN (NULL, '0007', '0009');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (7, 9);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt1_adv_p1_null;
+-- Restore plt1_adv_p1
+ALTER TABLE plt1_adv ATTACH PARTITION plt1_adv_p1 FOR VALUES IN ('0001', '0003');
+-- Add to plt1_adv the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt1_adv_extra PARTITION OF plt1_adv FOR VALUES IN (NULL);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+
+DROP TABLE plt2_adv_p3_null;
+-- Restore plt2_adv_p3
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p3 FOR VALUES IN ('0007', '0009');
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+-- Add to plt2_adv the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt2_adv_extra PARTITION OF plt2_adv FOR VALUES IN (NULL);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+-- 3-way join to test the NULL partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt1_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt1_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt1_adv_extra;
+DROP TABLE plt2_adv_extra;
+
+-- Test default partitions
+ALTER TABLE plt1_adv DETACH PARTITION plt1_adv_p1;
+-- Change plt1_adv_p1 to the default partition
+ALTER TABLE plt1_adv ATTACH PARTITION plt1_adv_p1 DEFAULT;
+DROP TABLE plt1_adv_p3;
+ANALYZE plt1_adv;
+
+DROP TABLE plt2_adv_p3;
+ANALYZE plt2_adv;
+
+-- We can do partitioned join even if only one of relations has the default
+-- partition
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2;
+-- Change plt2_adv_p2 to contain '0005' in addition to '0004' and '0006' as
+-- the key values
+CREATE TABLE plt2_adv_p2_ext PARTITION OF plt2_adv FOR VALUES IN ('0004', '0005', '0006');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 5, 6);
+ANALYZE plt2_adv;
+
+-- Partitioned join can't be applied because the default partition of plt1_adv
+-- matches plt2_adv_p1 and plt2_adv_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2_ext;
+-- Change plt2_adv_p2_ext to the default partition
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2_ext DEFAULT;
+ANALYZE plt2_adv;
+
+-- Partitioned join can't be applied because the default partition of plt1_adv
+-- matches plt2_adv_p1 and plt2_adv_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt2_adv_p2_ext;
+-- Restore plt2_adv_p2
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_adv;
+
+CREATE TABLE plt3_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_adv_p1 PARTITION OF plt3_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt3_adv_p2 PARTITION OF plt3_adv FOR VALUES IN ('0007', '0009');
+INSERT INTO plt3_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6, 7, 9);
+ANALYZE plt3_adv;
+
+-- 3-way join to test the default partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Test cases where one side has the default partition while the other side
+-- has the NULL partition
+DROP TABLE plt2_adv_p1;
+-- Add the NULL partition to plt2_adv
+CREATE TABLE plt2_adv_p1_null PARTITION OF plt2_adv FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt2_adv_p1_null;
+-- Add the NULL partition that contains only NULL values as the key values
+CREATE TABLE plt2_adv_p1_null PARTITION OF plt2_adv FOR VALUES IN (NULL);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+DROP TABLE plt3_adv;
+
+-- Test interaction of partitioned join with partition pruning
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0001');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0002');
+CREATE TABLE plt1_adv_p3 PARTITION OF plt1_adv FOR VALUES IN ('0003');
+CREATE TABLE plt1_adv_p4 PARTITION OF plt1_adv FOR VALUES IN (NULL, '0004', '0005');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN (NULL);
+CREATE TABLE plt2_adv_p3 PARTITION OF plt2_adv FOR VALUES IN ('0003');
+CREATE TABLE plt2_adv_p4 PARTITION OF plt2_adv FOR VALUES IN ('0004', '0005');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+
+CREATE TABLE plt1_adv_default PARTITION OF plt1_adv DEFAULT;
+ANALYZE plt1_adv;
+
+CREATE TABLE plt2_adv_default PARTITION OF plt2_adv DEFAULT;
+ANALYZE plt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+
+-- Test the process_outer_partition() code path
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1_adv;
+
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0002');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2_adv;
+
+CREATE TABLE plt3_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_adv_p1 PARTITION OF plt3_adv FOR VALUES IN ('0001');
+CREATE TABLE plt3_adv_p2 PARTITION OF plt3_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3_adv;
+
+-- This tests that when merging partitions from plt1_adv and plt2_adv in
+-- merge_list_bounds(), process_outer_partition() returns an already-assigned
+-- merged partition when re-called with plt1_adv_p1 for the second list value
+-- '0001' of that partitin
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.c = t2.c)) FULL JOIN plt3_adv t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.c = t2.c)) FULL JOIN plt3_adv t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+DROP TABLE plt3_adv;
+
+
+-- Tests for multi-level partitioned tables
+CREATE TABLE alpha (a double precision, b int, c text) PARTITION BY RANGE (a);
+CREATE TABLE alpha_neg PARTITION OF alpha FOR VALUES FROM ('-Infinity') TO (0) PARTITION BY RANGE (b);
+CREATE TABLE alpha_pos PARTITION OF alpha FOR VALUES FROM (0) TO (10.0) PARTITION BY LIST (c);
+CREATE TABLE alpha_neg_p1 PARTITION OF alpha_neg FOR VALUES FROM (100) TO (200);
+CREATE TABLE alpha_neg_p2 PARTITION OF alpha_neg FOR VALUES FROM (200) TO (300);
+CREATE TABLE alpha_neg_p3 PARTITION OF alpha_neg FOR VALUES FROM (300) TO (400);
+CREATE TABLE alpha_pos_p1 PARTITION OF alpha_pos FOR VALUES IN ('0001', '0003');
+CREATE TABLE alpha_pos_p2 PARTITION OF alpha_pos FOR VALUES IN ('0004', '0006');
+CREATE TABLE alpha_pos_p3 PARTITION OF alpha_pos FOR VALUES IN ('0008', '0009');
+INSERT INTO alpha_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+INSERT INTO alpha_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+ANALYZE alpha;
+
+CREATE TABLE beta (a double precision, b int, c text) PARTITION BY RANGE (a);
+CREATE TABLE beta_neg PARTITION OF beta FOR VALUES FROM (-10.0) TO (0) PARTITION BY RANGE (b);
+CREATE TABLE beta_pos PARTITION OF beta FOR VALUES FROM (0) TO ('Infinity') PARTITION BY LIST (c);
+CREATE TABLE beta_neg_p1 PARTITION OF beta_neg FOR VALUES FROM (100) TO (150);
+CREATE TABLE beta_neg_p2 PARTITION OF beta_neg FOR VALUES FROM (200) TO (300);
+CREATE TABLE beta_neg_p3 PARTITION OF beta_neg FOR VALUES FROM (350) TO (500);
+CREATE TABLE beta_pos_p1 PARTITION OF beta_pos FOR VALUES IN ('0002', '0003');
+CREATE TABLE beta_pos_p2 PARTITION OF beta_pos FOR VALUES IN ('0004', '0006');
+CREATE TABLE beta_pos_p3 PARTITION OF beta_pos FOR VALUES IN ('0007', '0009');
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+ANALYZE beta;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b;
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b, t2.b;
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b, t2.b;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;
-- 
2.14.3 (Apple Git-98)

0002-Changes.patchapplication/octet-stream; name=0002-Changes.patchDownload
From 11f93f75f05315404d5838ff050b0d66d1633a78 Mon Sep 17 00:00:00 2001
From: Etsuro Fujita <etsuro@EtsuronoMacBook-Air.local>
Date: Sat, 4 Apr 2020 00:05:59 +0900
Subject: [PATCH 2/2] Changes.

---
 doc/src/sgml/config.sgml              |   6 +-
 src/backend/nodes/outfuncs.c          |   2 +-
 src/backend/optimizer/README          |  27 +++++
 src/backend/optimizer/path/joinrels.c | 183 +++++++++++++++++++---------------
 src/backend/optimizer/util/relnode.c  |  10 +-
 src/backend/partitioning/partbounds.c |  86 ++++++++++------
 src/include/nodes/pathnodes.h         |  11 +-
 7 files changed, 198 insertions(+), 127 deletions(-)

diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 2de21903a1..50cffb8694 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4693,9 +4693,9 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
         which allows a join between partitioned tables to be performed by
         joining the matching partitions.  Partitionwise join currently applies
         only when the join conditions include all the partition keys, which
-        must be of the same data type and have exactly matching sets of child
-        partitions.  Because partitionwise join planning can use significantly
-        more CPU time and memory during planning, the default is
+        must be of the same data type and have one-to-one matching sets of
+        child partitions.  Because partitionwise join planning can use
+        significantly more CPU time and memory during planning, the default is
         <literal>off</literal>.
        </para>
       </listitem>
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 55dfa27157..baee36892f 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -2288,7 +2288,7 @@ _outRelOptInfo(StringInfo str, const RelOptInfo *node)
 	WRITE_BOOL_FIELD(has_eclass_joins);
 	WRITE_BOOL_FIELD(consider_partitionwise_join);
 	WRITE_BITMAPSET_FIELD(top_parent_relids);
-	WRITE_BOOL_FIELD(merged);
+	WRITE_BOOL_FIELD(partbounds_merged);
 	WRITE_BITMAPSET_FIELD(all_partrels);
 	WRITE_NODE_FIELD(partitioned_child_rels);
 }
diff --git a/src/backend/optimizer/README b/src/backend/optimizer/README
index 89ce373d5e..13633841f3 100644
--- a/src/backend/optimizer/README
+++ b/src/backend/optimizer/README
@@ -1106,6 +1106,33 @@ into joins between their partitions is called partitionwise join. We will use
 term "partitioned relation" for either a partitioned table or a join between
 compatibly partitioned tables.
 
+The technique is extended to some cases where the joining tables don't have
+exactly the same partition bounds, by an advanced partition-matching
+algorithm: it checks to see if there is a relationship where each partition of
+one joining table matches/overlaps at most one partition of the other, and
+vice versa; in which case the join between the joining tables can be broken
+down into joins between the matching partitions (ie, the join relation is
+considerd partitioned), so the algorithm produces the pairs of the matching
+partitions, plus the partition bounds for the join relation, to allow
+partitionwise join for the join.  The algorithm is implemented in
+partition_bounds_merge().  For an N-way join relation considered partitioned
+by this extension, not every pair of joining relations can use partitionwise
+join.  For example:
+
+	(A leftjoin B on (Pab)) innerjoin C on (Pac)
+
+where A, B, and C are partitioned tables, and A has an extra partition
+compared to B and C.  When considering partitionwise join for the join {A B},
+the extra partition of A doesn't have a matching partition on the nullable
+side, which is the case that the current implementation of partitionwise join
+can't handle.  So {A B} is not considered partitioned, and thus the pair of
+{A B} and C considered for the 3-way join can't use partitionwise join.  On
+the other hand, the pair of {A C} and B can use partitionwise join, because
+{A C} is considered partitioned, eliminating the extra partition (see identity
+1 on outer join reordering).  The partitionwise joinability of the N-way join
+relation is determined based on the first pair of joining relations that are
+both partitioned and can use partitionwise join.
+
 The partitioning properties of a partitioned relation are stored in its
 RelOptInfo.  The information about data types of partition keys are stored in
 PartitionSchemeData structure. The planner maintains a list of canonical
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 530ebed245..2c0ce6d2f3 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -48,6 +48,10 @@ static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 												Relids left_relids, Relids right_relids);
 static int	match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel,
 										 bool strict_op);
+static void compute_partition_bounds(PlannerInfo *root, RelOptInfo *rel1,
+									 RelOptInfo *rel2, RelOptInfo *joinrel,
+									 SpecialJoinInfo *parent_sjinfo,
+									 List **parts1, List **parts2);
 static void get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
 						RelOptInfo *rel1, RelOptInfo *rel2,
 						List **parts1, List **parts2);
@@ -1360,7 +1364,6 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	bool		merged = false;
 	List	   *parts1 = NIL;
 	List	   *parts2 = NIL;
 	ListCell   *lcr1 = NULL;
@@ -1397,89 +1400,12 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	Assert(joinrel->part_scheme == rel1->part_scheme &&
 		   joinrel->part_scheme == rel2->part_scheme);
 
-	/*
-	 * If we don't have the partition bounds for the join rel yet, try to
-	 * compute those along with pairs of partitions to be joined.
-	 */
-	if (joinrel->nparts == -1)
-	{
-		PartitionScheme part_scheme = joinrel->part_scheme;
-		PartitionBoundInfo boundinfo = NULL;
-		int			nparts = 0;
-
-		Assert(joinrel->boundinfo == NULL);
-		Assert(joinrel->part_rels == NULL);
-
-		/*
-		 * See if the partition bounds for inputs are exactly the same, in
-		 * which case we don't need to work hard: the join rel have the same
-		 * partition bounds as inputs, and the partitions with the same
-		 * cardinal positions form the pairs.
-		 *
-		 * Note: even in cases where one or both inputs have merged bounds,
-		 * it would be possible for both the bounds to be exactly the same, but
-		 * it seems unlikely to be worth the cycles to check.
-		 */
-		if (!rel1->merged &&
-			!rel2->merged &&
-			rel1->nparts == rel2->nparts &&
-			partition_bounds_equal(part_scheme->partnatts,
-								   part_scheme->parttyplen,
-								   part_scheme->parttypbyval,
-								   rel1->boundinfo, rel2->boundinfo))
-		{
-			boundinfo = rel1->boundinfo;
-			nparts = rel1->nparts;
-		}
-		else
-		{
-			/* Try merging the partition bounds for inputs. */
-			boundinfo = partition_bounds_merge(part_scheme->partnatts,
-											   part_scheme->partsupfunc,
-											   part_scheme->partcollation,
-											   rel1, rel2,
-											   parent_sjinfo->jointype,
-											   &parts1, &parts2);
-			if (boundinfo == NULL)
-			{
-				joinrel->nparts = 0;
-				return;
-			}
-			nparts = list_length(parts1);
-			merged = true;
-		}
-
-		Assert(nparts > 0);
-		joinrel->boundinfo = boundinfo;
-		joinrel->merged = merged;
-		joinrel->nparts = nparts;
-		joinrel->part_rels =
-			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
-	}
-	else
-	{
-		Assert(joinrel->nparts > 0);
-		Assert(joinrel->boundinfo);
-		Assert(joinrel->part_rels);
+	Assert(!(joinrel->partbounds_merged && (joinrel->nparts <= 0)));
 
-		/*
-		 * If the join rel's merged flag is true, it means inputs are not
-		 * guaranteed to have the same partition bounds, therefore we can't
-		 * assume that the partitions at the same cardinal positions form the
-		 * pairs; let get_matching_part_pairs() generate the pairs.  Otherwise,
-		 * nothing to do since we can assume that.
-		 */
-		if (joinrel->merged)
-		{
-			get_matching_part_pairs(root, joinrel, rel1, rel2,
-									&parts1, &parts2);
-			Assert(list_length(parts1) == joinrel->nparts);
-			Assert(list_length(parts2) == joinrel->nparts);
-			merged = true;
-		}
-	}
+	compute_partition_bounds(root, rel1, rel2, joinrel, parent_sjinfo,
+							 &parts1, &parts2);
 
-	if (merged)
+	if (joinrel->partbounds_merged)
 	{
 		lcr1 = list_head(parts1);
 		lcr2 = list_head(parts2);
@@ -1503,7 +1429,7 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
-		if (merged)
+		if (joinrel->partbounds_merged)
 		{
 			child_rel1 = lfirst_node(RelOptInfo, lcr1);
 			child_rel2 = lfirst_node(RelOptInfo, lcr2);
@@ -1835,6 +1761,97 @@ match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op)
 	return -1;
 }
 
+/*
+ * compute_partition_bounds
+ *		Compute the partition bounds for a join rel from those for inputs
+ */
+static void
+compute_partition_bounds(PlannerInfo *root, RelOptInfo *rel1,
+						 RelOptInfo *rel2, RelOptInfo *joinrel,
+						 SpecialJoinInfo *parent_sjinfo,
+						 List **parts1, List **parts2)
+{
+	/*
+	 * If we don't have the partition bounds for the join rel yet, try to
+	 * compute those along with pairs of partitions to be joined.
+	 */
+	if (joinrel->nparts == -1)
+	{
+		PartitionScheme part_scheme = joinrel->part_scheme;
+		PartitionBoundInfo boundinfo = NULL;
+		int			nparts = 0;
+
+		Assert(joinrel->boundinfo == NULL);
+		Assert(joinrel->part_rels == NULL);
+
+		/*
+		 * See if the partition bounds for inputs are exactly the same, in
+		 * which case we don't need to work hard: the join rel have the same
+		 * partition bounds as inputs, and the partitions with the same
+		 * cardinal positions form the pairs.
+		 *
+		 * Note: even in cases where one or both inputs have merged bounds,
+		 * it would be possible for both the bounds to be exactly the same, but
+		 * it seems unlikely to be worth the cycles to check.
+		 */
+		if (!rel1->partbounds_merged &&
+			!rel2->partbounds_merged &&
+			rel1->nparts == rel2->nparts &&
+			partition_bounds_equal(part_scheme->partnatts,
+								   part_scheme->parttyplen,
+								   part_scheme->parttypbyval,
+								   rel1->boundinfo, rel2->boundinfo))
+		{
+			boundinfo = rel1->boundinfo;
+			nparts = rel1->nparts;
+		}
+		else
+		{
+			/* Try merging the partition bounds for inputs. */
+			boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											   part_scheme->partsupfunc,
+											   part_scheme->partcollation,
+											   rel1, rel2,
+											   parent_sjinfo->jointype,
+											   parts1, parts2);
+			if (boundinfo == NULL)
+			{
+				joinrel->nparts = 0;
+				return;
+			}
+			nparts = list_length(*parts1);
+			joinrel->partbounds_merged = true;
+		}
+
+		Assert(nparts > 0);
+		joinrel->boundinfo = boundinfo;
+		joinrel->nparts = nparts;
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+	}
+	else
+	{
+		Assert(joinrel->nparts > 0);
+		Assert(joinrel->boundinfo);
+		Assert(joinrel->part_rels);
+
+		/*
+		 * If the join rel's partbounds_merged flag is true, it means inputs
+		 * are not guaranteed to have the same partition bounds, therefore we
+		 * can't assume that the partitions at the same cardinal positions form
+		 * the pairs; let get_matching_part_pairs() generate the pairs.
+		 * Otherwise, nothing to do since we can assume that.
+		 */
+		if (joinrel->partbounds_merged)
+		{
+			get_matching_part_pairs(root, joinrel, rel1, rel2,
+									parts1, parts2);
+			Assert(list_length(*parts1) == joinrel->nparts);
+			Assert(list_length(*parts2) == joinrel->nparts);
+		}
+	}
+}
+
 /*
  * get_matching_part_pairs
  *		Generate pairs of partitions to be joined from the two inputs
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 0e4944ac8e..433f031d0f 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -242,7 +242,7 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->part_scheme = NULL;
 	rel->nparts = -1;
 	rel->boundinfo = NULL;
-	rel->merged = false;
+	rel->partbounds_merged = false;
 	rel->partition_qual = NIL;
 	rel->part_rels = NULL;
 	rel->all_partrels = NULL;
@@ -657,7 +657,7 @@ build_join_rel(PlannerInfo *root,
 	joinrel->part_scheme = NULL;
 	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
-	joinrel->merged = false;
+	joinrel->partbounds_merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
 	joinrel->all_partrels = NULL;
@@ -835,7 +835,7 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->part_scheme = NULL;
 	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
-	joinrel->merged = false;
+	joinrel->partbounds_merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
 	joinrel->all_partrels = NULL;
@@ -1668,8 +1668,8 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 		   !joinrel->boundinfo);
 
 	/*
-	 * If the join relation is partitioned, it use the same partitioning scheme
-	 * as the joining relations.
+	 * If the join relation is partitioned, it uses the same partitioning
+	 * scheme as the joining relations.
 	 *
 	 * Note: we calculate the partition bounds, number of partitions, and
 	 * child-join relations of the join relation in try_partitionwise_join().
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 24dbc2c8f3..e4c74d6c03 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -69,7 +69,10 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
-/* Per-partitioned-relation data for merge_list_bounds()/merge_range_bounds() */
+/*
+ * Mapping from partitions of a partitioned relation to partitions of a join
+ * relation supposed to be partitioned (a.k.a merged partitions)
+ */
 typedef struct PartitionMap
 {
 	int			nparts;			/* number of partitions */
@@ -998,16 +1001,18 @@ partition_bounds_copy(PartitionBoundInfo src,
 
 /*
  * partition_bounds_merge
+ *		Check to see if there is a relationship where each partition of
+ *		'outer_rel' matches/overlaps at most one partition of 'inner_rel', and
+ *		vice versa; and if so, build and return the partition bounds for a join
+ *		relation between the rels, generating two lists of matching/overlapping
+ *		partitions, which are returned to *outer_parts and *inner_parts
+ *		respectively.
  *
- * This function builds and returns the partition bounds for a join relation
- * between input relations, creating two lists of partitions, which are
- * returned to *outer_parts and *inner_parts respectively.  The lists contain
- * the same number of partitions, and the partitions at the same positions in
- * the lists indicate join pairs used for partitioned join.
- *
- * This function returns NULL, setting *outer_parts and *inner_parts to NIL,
- * if a partition on one side matches multiple partitions on the other side,
- * in which case we currently don't support partitioned join.
+ * The lists contain the same number of partitions, and the partitions at the
+ * same positions in the lists indicate join pairs used for partitioned join.
+ * If a partition on one side matches/overlaps multiple partitions on the other
+ * side, this function returns NULL, setting *outer_parts and *inner_parts to
+ * NIL.
  */
 PartitionBoundInfo
 partition_bounds_merge(int partnatts,
@@ -1018,24 +1023,20 @@ partition_bounds_merge(int partnatts,
 {
 	PartitionBoundInfo outer_binfo = outer_rel->boundinfo;
 	PartitionBoundInfo inner_binfo = inner_rel->boundinfo;
-	char		strategy;
 
 	/*
 	 * Currently, this function is called only from try_partitionwise_join(),
 	 * so the join type should be INNER, LEFT, FULL, SEMI, or ANTI.
 	 */
-	if (jointype != JOIN_INNER && jointype != JOIN_LEFT && 
-		jointype != JOIN_FULL && jointype != JOIN_SEMI &&
-		jointype != JOIN_ANTI)
-		elog(ERROR, "unrecognized join type: %d", (int) jointype);
+	Assert(jointype == JOIN_INNER || jointype == JOIN_LEFT ||
+		   jointype == JOIN_FULL || jointype == JOIN_SEMI ||
+		   jointype == JOIN_ANTI);
 
-	/* Bail out if the partitioning strategies are different. */
-	if (outer_binfo->strategy != inner_binfo->strategy)
-		return NULL;
+	/* The partitioning strategies should be the same. */
+	Assert(outer_binfo->strategy == inner_binfo->strategy);
 
-	strategy = outer_binfo->strategy;
 	*outer_parts = *inner_parts = NIL;
-	switch (strategy)
+	switch (outer_binfo->strategy)
 	{
 		case PARTITION_STRATEGY_HASH:
 
@@ -1075,7 +1076,8 @@ partition_bounds_merge(int partnatts,
 									  inner_parts);
 
 		default:
-			elog(ERROR, "unexpected partition strategy: %d", (int) strategy);
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) outer_binfo->strategy);
 			return NULL;				/* keep compiler quiet */
 	}
 }
@@ -1084,6 +1086,18 @@ partition_bounds_merge(int partnatts,
  * merge_list_bounds
  *		Create the partition bounds for a join relation between list
  *		partitioned tables, if possible
+ *
+ * In this function we try to find matching partitions from both sides by
+ * comparing list values stored in their partition bounds.  Since the list
+ * values appear in the ascending order, an algorithm similar to merge join is
+ * used for that.  If a partition doesn't have a matching partition on the
+ * other side, the algorithm tries to match it with the default partition on
+ * the other side if any; if not, the algorithm tries to match it with a
+ * dummy partition on the other side if it is on the non-nullable side of an
+ * outer join.  Also, if both sides have the default partitions, the algorithm
+ * tries to match them with each other.  We give up if the algorithm finds a
+ * partition matching multiple partitions on the other side, which is the
+ * scenario the current implementation of partitioned join can't handle.
  */
 static PartitionBoundInfo
 merge_list_bounds(FmgrInfo *partsupfunc, Oid *partcollation,
@@ -1379,6 +1393,18 @@ cleanup:
  * merge_range_bounds
  *		Create the partition bounds for a join relation between range
  *		partitioned tables, if possible
+ *
+ * In this function we try to find overlapping partitions from both sides by
+ * comparing ranges stored in their partition bounds.  Since the ranges
+ * appear in the ascending order, an algorithm similar to merge join is
+ * used for that.  If a partition doesn't have an overlapping partition on the
+ * other side, the algorithm tries to match it with the default partition on
+ * the other side if any; if not, the algorithm tries to match it with a
+ * dummy partition on the other side if it is on the non-nullable side of an
+ * outer join.  Also, if both sides have the default partitions, the algorithm
+ * tries to match them with each other.  We give up if the algorithm finds a
+ * partition overlapping multiple partitions on the other side, which is the
+ * scenario the current implementation of partitioned join can't handle.
  */
 static PartitionBoundInfo
 merge_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
@@ -1851,8 +1877,8 @@ merge_matching_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
  *		index of the merged partition if successful, -1 otherwise
  *
  * If the partition is newly created, *next_index is incremented.  Also, if it
- * is the default partition of the join relation, *default_partition is set to
- * the index if not already done.
+ * is the default partition of the join relation, *default_index is set to the
+ * index if not already done.
  */
 static int
 process_outer_partition(PartitionMap *outer_map,
@@ -1901,7 +1927,7 @@ process_outer_partition(PartitionMap *outer_map,
 		 * has to be scanned all the way anyway, so the resulting partition
 		 * will contain all key values from the default partition, which any
 		 * other partition of the join relation will not contain.  Thus the
-		 * resutling partition will act as the default partition of the join
+		 * resulting partition will act as the default partition of the join
 		 * relation; record the index in *default_index if not already done.
 		 */
 		if (jointype == JOIN_FULL)
@@ -1932,8 +1958,8 @@ process_outer_partition(PartitionMap *outer_map,
  *		index of the merged partition if successful, -1 otherwise
  *
  * If the partition is newly created, *next_index is incremented.  Also, if it
- * is the default partition of the join relation, *default_partition is set to
- * the index if not already done.
+ * is the default partition of the join relation, *default_index is set to the
+ * index if not already done.
  */
 static int
 process_inner_partition(PartitionMap *outer_map,
@@ -1982,7 +2008,7 @@ process_inner_partition(PartitionMap *outer_map,
 		 * has to be scanned all the way anyway, so the resulting partition
 		 * will contain all key values from the default partition, which any
 		 * other partition of the join relation will not contain.  Thus the
-		 * resutling partition will act as the default partition of the join
+		 * resulting partition will act as the default partition of the join
 		 * relation; record the index in *default_index if not already done.
 		 */
 		if (IS_OUTER_JOIN(jointype))
@@ -2044,13 +2070,13 @@ merge_null_partitions(PartitionMap *outer_map,
 	{
 		Assert(outer_null >= 0 && outer_null < outer_map->nparts);
 		if (outer_map->merged_indexes[outer_null] == -1)
-		 	consider_outer_null = true;
+			consider_outer_null = true;
 	}
 	if (inner_has_null)
 	{
 		Assert(inner_null >= 0 && inner_null < inner_map->nparts);
 		if (inner_map->merged_indexes[inner_null] == -1)
-		 	consider_inner_null = true;
+			consider_inner_null = true;
 	}
 
 	/* If both flags are set false, we don't need to do anything. */
@@ -2235,7 +2261,7 @@ merge_default_partitions(PartitionMap *outer_map,
  *
  * Note: The caller assumes that the given partition doesn't have a non-dummy
  * matching partition on the other side, but if the given partition finds the
- * matchig partition later, we will adjust the assignment.
+ * matching partition later, we will adjust the assignment.
  */
 static int
 merge_partition_with_dummy(PartitionMap *map, int index, int *next_index)
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index 7f9c4ab1f1..0b1eb00223 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -579,7 +579,7 @@ typedef struct PartitionSchemeData *PartitionScheme;
  *		part_scheme - Partitioning scheme of the relation
  *		nparts - Number of partitions
  *		boundinfo - Partition bounds
- *		merged - true if partition bounds are merged ones
+ *		partbounds_merged - true if partition bounds are merged ones
  *		partition_qual - Partition constraint if not the root
  *		part_rels - RelOptInfos for each partition
  *		all_partrels - Relids set of all partition relids
@@ -720,11 +720,12 @@ typedef struct RelOptInfo
 
 	/* used for partitioned relations */
 	PartitionScheme part_scheme;	/* Partitioning scheme. */
-	int			nparts;			/* number of partitions; 0 = not partitioned;
-								 * -1 = not yet set */
+	int			nparts;			/* number of partitions; -1 if not yet set;
+								 * in case of a join relation 0 means it's
+								 * considered unpartitioned */
 	struct PartitionBoundInfoData *boundinfo;	/* Partition bounds */
-	bool		merged;			/* true if partition bounds were created by
-								 * partition_bounds_merge() */
+	bool		partbounds_merged;	/* true if partition bounds were created by
+									 * partition_bounds_merge() */
 	List	   *partition_qual; /* partition constraint */
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of partitions,
 									 * stored in the same order of bounds */
-- 
2.14.3 (Apple Git-98)

#131Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Etsuro Fujita (#130)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Sat, Apr 4, 2020 at 12:15 AM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

Attached is the original patch (0001) and one patch (0002) with
changes including those by Tomas and Ashutosh.

I merged the patches into one and rebased it against HEAD. Attached
is a new version, in which I added the commit message as well. Does
that make sense? If there are no objections, I’ll commit the patch.

Best regards,
Etsuro Fujita

Attachments:

v34-0001-Allow-partitionwise-joins-in-more-cases.patchapplication/octet-stream; name=v34-0001-Allow-partitionwise-joins-in-more-cases.patchDownload
From c50349a8b6bfdc9aba764ba6438038761f19f86f Mon Sep 17 00:00:00 2001
From: Etsuro Fujita <etsuro.fujita@gmail.com>
Date: Mon, 6 Apr 2020 17:10:41 +0900
Subject: [PATCH] Allow partitionwise joins in more cases.

Previously, the partitionwise join technique only allowed partitionwise
join when input partitioned tables had exactly the same partition bounds.
This commit extends the applicability of the technique to some cases when
the tables don't have exactly the same partition bounds, by introducing
an advanced partition-matching algorithm: it checks to see if there is a
relationship where each partition of one partitioned table
matches/overlaps at most one partition of the other, and vice versa; in
which case the join between the tables can be broken down into joins
between the matching partitions, so the algorithm produces the pairs of
the matching partitions, plus the partition bounds for the join relation,
to allow partitionwise join of the tables.  Currently, the algorithm
works for list partitioned and range partitioned tables, but not hash
partitioned tables.  (See comments in partition_bounds_merge().)

Ashutosh Bapat and Etsuro Fujita, most of regression tests by Rajkumar
Raghuwanshi, some of the tests by Mark Dilger and Amul Sul, reviewed by
Dmitry Dolgov and Amul Sul, with additional review at various points by
Ashutosh Bapat, Mark Dilger, Robert Haas, Antonin Houska, Amit Langote,
Justin Pryzby, and Tomas Vondra

Discussion:https://postgr.es/m/CAFjFpRdjQvaUEV5DJX3TW6pU5eq54NCkadtxHX2JiJG_GvbrCA@mail.gmail.com
---
 doc/src/sgml/config.sgml                     |    6 +-
 src/backend/nodes/outfuncs.c                 |    2 +
 src/backend/optimizer/README                 |   27 +
 src/backend/optimizer/path/joinrels.c        |  263 ++-
 src/backend/optimizer/util/inherit.c         |    2 +
 src/backend/optimizer/util/relnode.c         |   45 +-
 src/backend/partitioning/partbounds.c        | 1864 ++++++++++++++++++
 src/include/nodes/pathnodes.h                |    9 +-
 src/include/partitioning/partbounds.h        |    9 +
 src/test/regress/expected/partition_join.out | 2608 +++++++++++++++++++++++++-
 src/test/regress/sql/partition_join.sql      |  639 ++++++-
 11 files changed, 5391 insertions(+), 83 deletions(-)

diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 114db38116..ce83ea450b 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4712,9 +4712,9 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
         which allows a join between partitioned tables to be performed by
         joining the matching partitions.  Partitionwise join currently applies
         only when the join conditions include all the partition keys, which
-        must be of the same data type and have exactly matching sets of child
-        partitions.  Because partitionwise join planning can use significantly
-        more CPU time and memory during planning, the default is
+        must be of the same data type and have one-to-one matching sets of
+        child partitions.  Because partitionwise join planning can use
+        significantly more CPU time and memory during planning, the default is
         <literal>off</literal>.
        </para>
       </listitem>
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index f4aecdcbcd..f121781512 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -2288,6 +2288,8 @@ _outRelOptInfo(StringInfo str, const RelOptInfo *node)
 	WRITE_BOOL_FIELD(has_eclass_joins);
 	WRITE_BOOL_FIELD(consider_partitionwise_join);
 	WRITE_BITMAPSET_FIELD(top_parent_relids);
+	WRITE_BOOL_FIELD(partbounds_merged);
+	WRITE_BITMAPSET_FIELD(all_partrels);
 	WRITE_NODE_FIELD(partitioned_child_rels);
 }
 
diff --git a/src/backend/optimizer/README b/src/backend/optimizer/README
index 89ce373d5e..13633841f3 100644
--- a/src/backend/optimizer/README
+++ b/src/backend/optimizer/README
@@ -1106,6 +1106,33 @@ into joins between their partitions is called partitionwise join. We will use
 term "partitioned relation" for either a partitioned table or a join between
 compatibly partitioned tables.
 
+The technique is extended to some cases where the joining tables don't have
+exactly the same partition bounds, by an advanced partition-matching
+algorithm: it checks to see if there is a relationship where each partition of
+one joining table matches/overlaps at most one partition of the other, and
+vice versa; in which case the join between the joining tables can be broken
+down into joins between the matching partitions (ie, the join relation is
+considerd partitioned), so the algorithm produces the pairs of the matching
+partitions, plus the partition bounds for the join relation, to allow
+partitionwise join for the join.  The algorithm is implemented in
+partition_bounds_merge().  For an N-way join relation considered partitioned
+by this extension, not every pair of joining relations can use partitionwise
+join.  For example:
+
+	(A leftjoin B on (Pab)) innerjoin C on (Pac)
+
+where A, B, and C are partitioned tables, and A has an extra partition
+compared to B and C.  When considering partitionwise join for the join {A B},
+the extra partition of A doesn't have a matching partition on the nullable
+side, which is the case that the current implementation of partitionwise join
+can't handle.  So {A B} is not considered partitioned, and thus the pair of
+{A B} and C considered for the 3-way join can't use partitionwise join.  On
+the other hand, the pair of {A C} and B can use partitionwise join, because
+{A C} is considered partitioned, eliminating the extra partition (see identity
+1 on outer join reordering).  The partitionwise joinability of the N-way join
+relation is determined based on the first pair of joining relations that are
+both partitioned and can use partitionwise join.
+
 The partitioning properties of a partitioned relation are stored in its
 RelOptInfo.  The information about data types of partition keys are stored in
 PartitionSchemeData structure. The planner maintains a list of canonical
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 5b8d71dc26..aae198ca42 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -45,6 +45,13 @@ static void try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1,
 static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 												SpecialJoinInfo *parent_sjinfo,
 												Relids left_relids, Relids right_relids);
+static void compute_partition_bounds(PlannerInfo *root, RelOptInfo *rel1,
+									 RelOptInfo *rel2, RelOptInfo *joinrel,
+									 SpecialJoinInfo *parent_sjinfo,
+									 List **parts1, List **parts2);
+static void get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+									RelOptInfo *rel1, RelOptInfo *rel2,
+									List **parts1, List **parts2);
 
 
 /*
@@ -1354,25 +1361,29 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 {
 	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
 	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
-	int			nparts;
+	List	   *parts1 = NIL;
+	List	   *parts2 = NIL;
+	ListCell   *lcr1 = NULL;
+	ListCell   *lcr2 = NULL;
 	int			cnt_parts;
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
 
 	/* Nothing to do, if the join relation is not partitioned. */
-	if (!IS_PARTITIONED_REL(joinrel))
+	if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
 		return;
 
 	/* The join relation should have consider_partitionwise_join set. */
 	Assert(joinrel->consider_partitionwise_join);
 
 	/*
-	 * Since this join relation is partitioned, all the base relations
-	 * participating in this join must be partitioned and so are all the
-	 * intermediate join relations.
+	 * We can not perform partitionwise join if either of the joining relations
+	 * is not partitioned.
 	 */
-	Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2));
+	if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2))
+		return;
+
 	Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2));
 
 	/* The joining relations should have consider_partitionwise_join set. */
@@ -1386,35 +1397,28 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	Assert(joinrel->part_scheme == rel1->part_scheme &&
 		   joinrel->part_scheme == rel2->part_scheme);
 
-	/*
-	 * Since we allow partitionwise join only when the partition bounds of the
-	 * joining relations exactly match, the partition bounds of the join
-	 * should match those of the joining relations.
-	 */
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel1->boundinfo));
-	Assert(partition_bounds_equal(joinrel->part_scheme->partnatts,
-								  joinrel->part_scheme->parttyplen,
-								  joinrel->part_scheme->parttypbyval,
-								  joinrel->boundinfo, rel2->boundinfo));
+	Assert(!(joinrel->partbounds_merged && (joinrel->nparts <= 0)));
 
-	nparts = joinrel->nparts;
+	compute_partition_bounds(root, rel1, rel2, joinrel, parent_sjinfo,
+							 &parts1, &parts2);
+
+	if (joinrel->partbounds_merged)
+	{
+		lcr1 = list_head(parts1);
+		lcr2 = list_head(parts2);
+	}
 
 	/*
 	 * Create child-join relations for this partitioned join, if those don't
 	 * exist. Add paths to child-joins for a pair of child relations
 	 * corresponding to the given pair of parent relations.
 	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
 	{
-		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
-		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
-		bool		rel1_empty = (child_rel1 == NULL ||
-								  IS_DUMMY_REL(child_rel1));
-		bool		rel2_empty = (child_rel2 == NULL ||
-								  IS_DUMMY_REL(child_rel2));
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		bool		rel1_empty;
+		bool		rel2_empty;
 		SpecialJoinInfo *child_sjinfo;
 		List	   *child_restrictlist;
 		RelOptInfo *child_joinrel;
@@ -1422,6 +1426,22 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		AppendRelInfo **appinfos;
 		int			nappinfos;
 
+		if (joinrel->partbounds_merged)
+		{
+			child_rel1 = lfirst_node(RelOptInfo, lcr1);
+			child_rel2 = lfirst_node(RelOptInfo, lcr2);
+			lcr1 = lnext(parts1, lcr1);
+			lcr2 = lnext(parts2, lcr2);
+		}
+		else
+		{
+			child_rel1 = rel1->part_rels[cnt_parts];
+			child_rel2 = rel2->part_rels[cnt_parts];
+		}
+
+		rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1));
+		rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2));
+
 		/*
 		 * Check for cases where we can prove that this segment of the join
 		 * returns no rows, due to one or both inputs being empty (including
@@ -1519,6 +1539,8 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 												 child_sjinfo,
 												 child_sjinfo->jointype);
 			joinrel->part_rels[cnt_parts] = child_joinrel;
+			joinrel->all_partrels = bms_add_members(joinrel->all_partrels,
+													child_joinrel->relids);
 		}
 
 		Assert(bms_equal(child_joinrel->relids, child_joinrelids));
@@ -1570,3 +1592,190 @@ build_child_join_sjinfo(PlannerInfo *root, SpecialJoinInfo *parent_sjinfo,
 
 	return sjinfo;
 }
+
+/*
+ * compute_partition_bounds
+ *		Compute the partition bounds for a join rel from those for inputs
+ */
+static void
+compute_partition_bounds(PlannerInfo *root, RelOptInfo *rel1,
+						 RelOptInfo *rel2, RelOptInfo *joinrel,
+						 SpecialJoinInfo *parent_sjinfo,
+						 List **parts1, List **parts2)
+{
+	/*
+	 * If we don't have the partition bounds for the join rel yet, try to
+	 * compute those along with pairs of partitions to be joined.
+	 */
+	if (joinrel->nparts == -1)
+	{
+		PartitionScheme part_scheme = joinrel->part_scheme;
+		PartitionBoundInfo boundinfo = NULL;
+		int			nparts = 0;
+
+		Assert(joinrel->boundinfo == NULL);
+		Assert(joinrel->part_rels == NULL);
+
+		/*
+		 * See if the partition bounds for inputs are exactly the same, in
+		 * which case we don't need to work hard: the join rel have the same
+		 * partition bounds as inputs, and the partitions with the same
+		 * cardinal positions form the pairs.
+		 *
+		 * Note: even in cases where one or both inputs have merged bounds,
+		 * it would be possible for both the bounds to be exactly the same, but
+		 * it seems unlikely to be worth the cycles to check.
+		 */
+		if (!rel1->partbounds_merged &&
+			!rel2->partbounds_merged &&
+			rel1->nparts == rel2->nparts &&
+			partition_bounds_equal(part_scheme->partnatts,
+								   part_scheme->parttyplen,
+								   part_scheme->parttypbyval,
+								   rel1->boundinfo, rel2->boundinfo))
+		{
+			boundinfo = rel1->boundinfo;
+			nparts = rel1->nparts;
+		}
+		else
+		{
+			/* Try merging the partition bounds for inputs. */
+			boundinfo = partition_bounds_merge(part_scheme->partnatts,
+											   part_scheme->partsupfunc,
+											   part_scheme->partcollation,
+											   rel1, rel2,
+											   parent_sjinfo->jointype,
+											   parts1, parts2);
+			if (boundinfo == NULL)
+			{
+				joinrel->nparts = 0;
+				return;
+			}
+			nparts = list_length(*parts1);
+			joinrel->partbounds_merged = true;
+		}
+
+		Assert(nparts > 0);
+		joinrel->boundinfo = boundinfo;
+		joinrel->nparts = nparts;
+		joinrel->part_rels =
+			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+	}
+	else
+	{
+		Assert(joinrel->nparts > 0);
+		Assert(joinrel->boundinfo);
+		Assert(joinrel->part_rels);
+
+		/*
+		 * If the join rel's partbounds_merged flag is true, it means inputs
+		 * are not guaranteed to have the same partition bounds, therefore we
+		 * can't assume that the partitions at the same cardinal positions form
+		 * the pairs; let get_matching_part_pairs() generate the pairs.
+		 * Otherwise, nothing to do since we can assume that.
+		 */
+		if (joinrel->partbounds_merged)
+		{
+			get_matching_part_pairs(root, joinrel, rel1, rel2,
+									parts1, parts2);
+			Assert(list_length(*parts1) == joinrel->nparts);
+			Assert(list_length(*parts2) == joinrel->nparts);
+		}
+	}
+}
+
+/*
+ * get_matching_part_pairs
+ *		Generate pairs of partitions to be joined from the two inputs
+ */
+static void
+get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel,
+						RelOptInfo *rel1, RelOptInfo *rel2,
+						List **parts1, List **parts2)
+{
+	bool		rel1_is_simple = IS_SIMPLE_REL(rel1);
+	bool		rel2_is_simple = IS_SIMPLE_REL(rel2);
+	int 		cnt_parts;
+
+	*parts1 = NIL;
+	*parts2 = NIL;
+
+	for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++)
+	{
+		RelOptInfo *child_joinrel = joinrel->part_rels[cnt_parts];
+		RelOptInfo *child_rel1;
+		RelOptInfo *child_rel2;
+		Relids		child_relids1;
+		Relids		child_relids2;
+
+		/*
+		 * If this segment of the join is empty, it means that this segment
+		 * was ignored when previously creating child-join paths for it in
+		 * try_partitionwise_join() as it would not contribute to the join
+		 * result, due to one or both inputs being empty; add NULL to each of
+		 * the given lists so that this segment will be ignored again in that
+		 * function.
+		 */
+		if (!child_joinrel)
+		{
+			*parts1 = lappend(*parts1, NULL);
+			*parts2 = lappend(*parts2, NULL);
+			continue;
+		}
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel1 side.
+		 */
+		child_relids1 = bms_intersect(child_joinrel->relids,
+									  rel1->all_partrels);
+		Assert(bms_num_members(child_relids1) == bms_num_members(rel1->relids));
+
+		/*
+		 * Get a child rel for rel1 with the relids.  Note that we should have
+		 * the child rel even if rel1 is a join rel, because in that case the
+		 * partitions specified in the relids would have matching/overlapping
+		 * boundaries, so the specified partitions should be considered as ones
+		 * to be joined when planning partitionwise joins of rel1, meaning that
+		 * the child rel would have been built by the time we get here.
+		 */
+		if (rel1_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids1);
+
+			child_rel1 = find_base_rel(root, varno);
+		}
+		else
+			child_rel1 = find_join_rel(root, child_relids1);
+		Assert(child_rel1);
+
+		/*
+		 * Get a relids set of partition(s) involved in this join segment that
+		 * are from the rel2 side.
+		 */
+		child_relids2 = bms_intersect(child_joinrel->relids,
+									  rel2->all_partrels);
+		Assert(bms_num_members(child_relids2) == bms_num_members(rel2->relids));
+
+		/*
+		 * Get a child rel for rel2 with the relids.  See above comments.
+		 */
+		if (rel2_is_simple)
+		{
+			int			varno = bms_singleton_member(child_relids2);
+
+			child_rel2 = find_base_rel(root, varno);
+		}
+		else
+			child_rel2 = find_join_rel(root, child_relids2);
+		Assert(child_rel2);
+
+		/*
+		 * The join of rel1 and rel2 is legal, so is the join of the child
+		 * rels obtained above; add them to the given lists as a join pair
+		 * producing this join segment.
+		 */
+		*parts1 = lappend(*parts1, child_rel1);
+		*parts2 = lappend(*parts2, child_rel2);
+	}
+}
diff --git a/src/backend/optimizer/util/inherit.c b/src/backend/optimizer/util/inherit.c
index 7db67fdf34..3132fd35a5 100644
--- a/src/backend/optimizer/util/inherit.c
+++ b/src/backend/optimizer/util/inherit.c
@@ -376,6 +376,8 @@ expand_partitioned_rtentry(PlannerInfo *root, RelOptInfo *relinfo,
 		/* Create the otherrel RelOptInfo too. */
 		childrelinfo = build_simple_rel(root, childRTindex, relinfo);
 		relinfo->part_rels[i] = childrelinfo;
+		relinfo->all_partrels = bms_add_members(relinfo->all_partrels,
+												childrelinfo->relids);
 
 		/* If this child is itself partitioned, recurse */
 		if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index af1fb48648..cbb1ee3b33 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -249,10 +249,12 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->has_eclass_joins = false;
 	rel->consider_partitionwise_join = false;	/* might get changed later */
 	rel->part_scheme = NULL;
-	rel->nparts = 0;
+	rel->nparts = -1;
 	rel->boundinfo = NULL;
+	rel->partbounds_merged = false;
 	rel->partition_qual = NIL;
 	rel->part_rels = NULL;
+	rel->all_partrels = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->partitioned_child_rels = NIL;
@@ -662,10 +664,12 @@ build_join_rel(PlannerInfo *root,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->partbounds_merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -838,10 +842,12 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->consider_partitionwise_join = false;	/* might get changed later */
 	joinrel->top_parent_relids = NULL;
 	joinrel->part_scheme = NULL;
-	joinrel->nparts = 0;
+	joinrel->nparts = -1;
 	joinrel->boundinfo = NULL;
+	joinrel->partbounds_merged = false;
 	joinrel->partition_qual = NIL;
 	joinrel->part_rels = NULL;
+	joinrel->all_partrels = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 	joinrel->partitioned_child_rels = NIL;
@@ -1645,7 +1651,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 	 * of the way the query planner deduces implied equalities and reorders
 	 * the joins.  Please see optimizer/README for details.
 	 */
-	if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) ||
+	if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL ||
 		!outer_rel->consider_partitionwise_join ||
 		!inner_rel->consider_partitionwise_join ||
 		outer_rel->part_scheme != inner_rel->part_scheme ||
@@ -1658,24 +1664,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 
 	part_scheme = outer_rel->part_scheme;
 
-	Assert(REL_HAS_ALL_PART_PROPS(outer_rel) &&
-		   REL_HAS_ALL_PART_PROPS(inner_rel));
-
-	/*
-	 * For now, our partition matching algorithm can match partitions only
-	 * when the partition bounds of the joining relations are exactly same.
-	 * So, bail out otherwise.
-	 */
-	if (outer_rel->nparts != inner_rel->nparts ||
-		!partition_bounds_equal(part_scheme->partnatts,
-								part_scheme->parttyplen,
-								part_scheme->parttypbyval,
-								outer_rel->boundinfo, inner_rel->boundinfo))
-	{
-		Assert(!IS_PARTITIONED_REL(joinrel));
-		return;
-	}
-
 	/*
 	 * This function will be called only once for each joinrel, hence it
 	 * should not have partitioning fields filled yet.
@@ -1685,18 +1673,15 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 		   !joinrel->boundinfo);
 
 	/*
-	 * Join relation is partitioned using the same partitioning scheme as the
-	 * joining relations and has same bounds.
+	 * If the join relation is partitioned, it uses the same partitioning
+	 * scheme as the joining relations.
+	 *
+	 * Note: we calculate the partition bounds, number of partitions, and
+	 * child-join relations of the join relation in try_partitionwise_join().
 	 */
 	joinrel->part_scheme = part_scheme;
-	joinrel->boundinfo = outer_rel->boundinfo;
-	joinrel->nparts = outer_rel->nparts;
 	set_joinrel_partition_key_exprs(joinrel, outer_rel, inner_rel, jointype);
 
-	/* part_rels[] will be filled later, but allocate it now */
-	joinrel->part_rels =
-		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
-
 	/*
 	 * Set the consider_partitionwise_join flag.
 	 */
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 4c47f54a57..e4c74d6c03 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -26,6 +26,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
+#include "nodes/pathnodes.h"
 #include "parser/parse_coerce.h"
 #include "partitioning/partbounds.h"
 #include "partitioning/partdesc.h"
@@ -68,6 +69,28 @@ typedef struct PartitionRangeBound
 	bool		lower;			/* this is the lower (vs upper) bound */
 } PartitionRangeBound;
 
+/*
+ * Mapping from partitions of a partitioned relation to partitions of a join
+ * relation supposed to be partitioned (a.k.a merged partitions)
+ */
+typedef struct PartitionMap
+{
+	int			nparts;			/* number of partitions */
+	int		   *merged_indexes;	/* indexes of merged partitions */
+	bool	   *merged;			/* flags to indicate whether partitions are
+								 * merged with non-dummy partitions */
+	bool		did_remapping;	/* did we re-map partitions? */
+	int		   *old_indexes;	/* old indexes of merged partitions if
+								 * did_remapping */
+} PartitionMap;
+
+/* Macro for comparing two range bounds */
+#define compare_range_bounds(partnatts, partsupfunc, partcollations, \
+							 bound1, bound2) \
+	(partition_rbound_cmp(partnatts, partsupfunc, partcollations, \
+						  (bound1)->datums, (bound1)->kind, (bound1)->lower, \
+						  bound2))
+
 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);
@@ -79,6 +102,116 @@ static PartitionBoundInfo create_list_bounds(PartitionBoundSpec **boundspecs,
 											 int nparts, PartitionKey key, int **mapping);
 static PartitionBoundInfo create_range_bounds(PartitionBoundSpec **boundspecs,
 											  int nparts, PartitionKey key, int **mapping);
+static PartitionBoundInfo merge_list_bounds(FmgrInfo *partsupfunc,
+											Oid *collations,
+											RelOptInfo *outer_rel,
+											RelOptInfo *inner_rel,
+											JoinType jointype,
+											List **outer_parts,
+											List **inner_parts);
+static PartitionBoundInfo merge_range_bounds(int partnatts,
+											 FmgrInfo *partsupfuncs,
+											 Oid *partcollations,
+											 RelOptInfo *outer_rel,
+											 RelOptInfo *inner_rel,
+											 JoinType jointype,
+											 List **outer_parts,
+											 List **inner_parts);
+static void init_partition_map(RelOptInfo *rel, PartitionMap *map);
+static void free_partition_map(PartitionMap *map);
+static bool is_dummy_partition(RelOptInfo *rel, int part_index);
+static int merge_matching_partitions(PartitionMap *outer_map,
+									 PartitionMap *inner_map,
+									 int outer_part,
+									 int inner_part,
+									 int *next_index);
+static int process_outer_partition(PartitionMap *outer_map,
+								   PartitionMap *inner_map,
+								   bool outer_has_default,
+								   bool inner_has_default,
+								   int outer_index,
+								   int inner_default,
+								   JoinType jointype,
+								   int *next_index,
+								   int *default_index);
+static int process_inner_partition(PartitionMap *outer_map,
+								   PartitionMap *inner_map,
+								   bool outer_has_default,
+								   bool inner_has_default,
+								   int inner_index,
+								   int outer_default,
+								   JoinType jointype,
+								   int *next_index,
+								   int *default_index);
+static void merge_null_partitions(PartitionMap *outer_map,
+								  PartitionMap *inner_map,
+								  bool outer_has_null,
+								  bool inner_has_null,
+								  int outer_null,
+								  int inner_null,
+								  JoinType jointype,
+								  int *next_index,
+								  int *null_index);
+static void merge_default_partitions(PartitionMap *outer_map,
+									 PartitionMap *inner_map,
+									 bool outer_has_default,
+									 bool inner_has_default,
+									 int outer_default,
+									 int inner_default,
+									 JoinType jointype,
+									 int *next_index,
+									 int *default_index);
+static int merge_partition_with_dummy(PartitionMap *map, int index,
+									  int *next_index);
+static void fix_merged_indexes(PartitionMap *outer_map,
+							   PartitionMap *inner_map,
+							   int nmerged, List *merged_indexes);
+static void generate_matching_part_pairs(RelOptInfo *outer_rel,
+										 RelOptInfo *inner_rel,
+										 PartitionMap *outer_map,
+										 PartitionMap *inner_map,
+										 int nmerged,
+										 List **outer_parts,
+										 List **inner_parts);
+static PartitionBoundInfo build_merged_partition_bounds(char strategy,
+														List *merged_datums,
+														List *merged_kinds,
+														List *merged_indexes,
+														int null_index,
+														int default_index);
+static int get_range_partition(RelOptInfo *rel,
+							   PartitionBoundInfo bi,
+							   int *lb_pos,
+							   PartitionRangeBound *lb,
+							   PartitionRangeBound *ub);
+static int get_range_partition_internal(PartitionBoundInfo bi,
+										int *lb_pos,
+										PartitionRangeBound *lb,
+										PartitionRangeBound *ub);
+static bool compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+									 Oid *partcollations,
+									 PartitionRangeBound *outer_lb,
+									 PartitionRangeBound *outer_ub,
+									 PartitionRangeBound *inner_lb,
+									 PartitionRangeBound *inner_ub,
+									 int *lb_cmpval, int *ub_cmpval);
+static void get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+									Oid *partcollations, JoinType jointype,
+									PartitionRangeBound *outer_lb,
+									PartitionRangeBound *outer_ub,
+									PartitionRangeBound *inner_lb,
+									PartitionRangeBound *inner_ub,
+									int	lb_cmpval, int ub_cmpval,
+									PartitionRangeBound *merged_lb,
+									PartitionRangeBound *merged_ub);
+static void add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+									Oid *partcollations,
+									PartitionRangeBound *merged_lb,
+									PartitionRangeBound *merged_ub,
+									int merged_index,
+									List **merged_datums,
+									List **merged_kinds,
+									List **merged_indexes);
 static PartitionRangeBound *make_one_partition_rbound(PartitionKey key, int index,
 													  List *datums, bool lower);
 static int32 partition_hbound_cmp(int modulus1, int remainder1, int modulus2,
@@ -866,6 +999,1737 @@ partition_bounds_copy(PartitionBoundInfo src,
 	return dest;
 }
 
+/*
+ * partition_bounds_merge
+ *		Check to see if there is a relationship where each partition of
+ *		'outer_rel' matches/overlaps at most one partition of 'inner_rel', and
+ *		vice versa; and if so, build and return the partition bounds for a join
+ *		relation between the rels, generating two lists of matching/overlapping
+ *		partitions, which are returned to *outer_parts and *inner_parts
+ *		respectively.
+ *
+ * The lists contain the same number of partitions, and the partitions at the
+ * same positions in the lists indicate join pairs used for partitioned join.
+ * If a partition on one side matches/overlaps multiple partitions on the other
+ * side, this function returns NULL, setting *outer_parts and *inner_parts to
+ * NIL.
+ */
+PartitionBoundInfo
+partition_bounds_merge(int partnatts,
+					   FmgrInfo *partsupfunc, Oid *partcollation,
+					   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+					   JoinType jointype,
+					   List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo outer_binfo = outer_rel->boundinfo;
+	PartitionBoundInfo inner_binfo = inner_rel->boundinfo;
+
+	/*
+	 * Currently, this function is called only from try_partitionwise_join(),
+	 * so the join type should be INNER, LEFT, FULL, SEMI, or ANTI.
+	 */
+	Assert(jointype == JOIN_INNER || jointype == JOIN_LEFT ||
+		   jointype == JOIN_FULL || jointype == JOIN_SEMI ||
+		   jointype == JOIN_ANTI);
+
+	/* The partitioning strategies should be the same. */
+	Assert(outer_binfo->strategy == inner_binfo->strategy);
+
+	*outer_parts = *inner_parts = NIL;
+	switch (outer_binfo->strategy)
+	{
+		case PARTITION_STRATEGY_HASH:
+
+			/*
+			 * For hash partitioned tables, we currently support partitioned
+			 * join only when the partition bounds for them exactly match.
+			 *
+			 * XXX: it might be possible to relax the restriction to support
+			 * cases where hash partitioned tables have missing partitions
+			 * and/or different moduli, but it's not clear if it would be
+			 * useful to support the former case since it's unusual to have
+			 * missing partitions.  On the other hand, it would be useful to
+			 * support the latter case, but in that case, there is a high
+			 * probability that a partition on one side will match multiple
+			 * partitions on the other side, which is the scenario the current
+			 * implementation of partitioned join can't handle.
+			 */
+			return NULL;
+
+		case PARTITION_STRATEGY_LIST:
+			return merge_list_bounds(partsupfunc,
+									 partcollation,
+									 outer_rel,
+									 inner_rel,
+									 jointype,
+									 outer_parts,
+									 inner_parts);
+
+		case PARTITION_STRATEGY_RANGE:
+			return merge_range_bounds(partnatts,
+									  partsupfunc,
+									  partcollation,
+									  outer_rel,
+									  inner_rel,
+									  jointype,
+									  outer_parts,
+									  inner_parts);
+
+		default:
+			elog(ERROR, "unexpected partition strategy: %d",
+				 (int) outer_binfo->strategy);
+			return NULL;				/* keep compiler quiet */
+	}
+}
+
+/*
+ * merge_list_bounds
+ *		Create the partition bounds for a join relation between list
+ *		partitioned tables, if possible
+ *
+ * In this function we try to find matching partitions from both sides by
+ * comparing list values stored in their partition bounds.  Since the list
+ * values appear in the ascending order, an algorithm similar to merge join is
+ * used for that.  If a partition doesn't have a matching partition on the
+ * other side, the algorithm tries to match it with the default partition on
+ * the other side if any; if not, the algorithm tries to match it with a
+ * dummy partition on the other side if it is on the non-nullable side of an
+ * outer join.  Also, if both sides have the default partitions, the algorithm
+ * tries to match them with each other.  We give up if the algorithm finds a
+ * partition matching multiple partitions on the other side, which is the
+ * scenario the current implementation of partitioned join can't handle.
+ */
+static PartitionBoundInfo
+merge_list_bounds(FmgrInfo *partsupfunc, Oid *partcollation,
+				  RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+				  JoinType jointype,
+				  List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	bool		outer_has_null = partition_bound_accepts_nulls(outer_bi);
+	bool		inner_has_null = partition_bound_accepts_nulls(inner_bi);
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			outer_pos;
+	int			inner_pos;
+	int			next_index = 0;
+	int			null_index = -1;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_indexes = NIL;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_LIST);
+	/* List partitioning doesn't require kinds. */
+	Assert(!outer_bi->kind && !inner_bi->kind);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * If the default partitions (if any) have been proven empty, deem them
+	 * non-existent.
+	 */
+	if (outer_has_default && is_dummy_partition(outer_rel, outer_default))
+		outer_has_default = false;
+	if (inner_has_default && is_dummy_partition(inner_rel, inner_default))
+		inner_has_default = false;
+
+	/*
+	 * Merge partitions from both sides.  In each iteration we compare a pair
+	 * of list values, one from each side, and decide whether the corresponding
+	 * partitions match or not.  If the two values match exactly, move to the
+	 * next pair of list values, otherwise move to the next list value on the
+	 * side with a smaller list value.
+	 */
+	outer_pos = inner_pos = 0;
+	while (outer_pos < outer_bi->ndatums || inner_pos < inner_bi->ndatums)
+	{
+		int			outer_index = -1;
+		int			inner_index = -1;
+		Datum	   *outer_datums;
+		Datum	   *inner_datums;
+		int			cmpval;
+		Datum	   *merged_datum = NULL;
+		int			merged_index = -1;
+
+		if (outer_pos < outer_bi->ndatums)
+		{
+			/*
+			 * If the partition on the outer side has been proven empty, ignore
+			 * it and move to the next datum on the outer side.
+			 */
+			outer_index = outer_bi->indexes[outer_pos];
+			if (is_dummy_partition(outer_rel, outer_index))
+			{
+				outer_pos++;
+				continue;
+			}
+		}
+		if (inner_pos < inner_bi->ndatums)
+		{
+			/*
+			 * If the partition on the inner side has been proven empty, ignore
+			 * it and move to the next datum on the inner side.
+			 */
+			inner_index = inner_bi->indexes[inner_pos];
+			if (is_dummy_partition(inner_rel, inner_index))
+			{
+				inner_pos++;
+				continue;
+			}
+		}
+
+		/* Get the list values. */
+		outer_datums = outer_pos < outer_bi->ndatums ?
+			outer_bi->datums[outer_pos] : NULL;
+		inner_datums = inner_pos < inner_bi->ndatums ?
+			inner_bi->datums[inner_pos] : NULL;
+
+		/*
+		 * We run this loop till both sides finish.  This allows us to avoid
+		 * duplicating code to handle the remaining values on the side which
+		 * finishes later.  For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra value higher than any other value on the unfinished side.
+		 * That way we advance the values on the unfinished side till all of
+		 * its values are exhausted.
+		 */
+		if (outer_pos >= outer_bi->ndatums)
+			cmpval = 1;
+		else if (inner_pos >= inner_bi->ndatums)
+			cmpval = -1;
+		else
+		{
+			Assert(outer_datums != NULL && inner_datums != NULL);
+			cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0],
+													 partcollation[0],
+													 outer_datums[0],
+													 inner_datums[0]));
+		}
+
+		if (cmpval == 0)
+		{
+			/* Two list values match exactly. */
+			Assert(outer_pos < outer_bi->ndatums);
+			Assert(inner_pos < inner_bi->ndatums);
+			Assert(outer_index >= 0);
+			Assert(inner_index >= 0);
+
+			/*
+			 * Try merging both paritions.  If successful, add the list value
+			 * and index of the merged partition below.
+			 */
+			merged_index = merge_matching_partitions(&outer_map, &inner_map,
+													 outer_index, inner_index,
+													 &next_index);
+			if (merged_index == -1)
+				goto cleanup;
+
+			merged_datum = outer_datums;
+
+			/* Move to the next pair of list values. */
+			outer_pos++;
+			inner_pos++;
+		}
+		else if (cmpval < 0)
+		{
+			/* A list value missing from the inner side. */
+			Assert(outer_pos < outer_bi->ndatums);
+
+			/*
+			 * If the inner side has the default partition, or this is an outer
+			 * join, try to assign a merged partition to the outer partition
+			 * (see process_outer_partition()).  Otherwise, the outer partition
+			 * will not contribute to the result.
+			 */
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				/* Get the outer partition. */
+				outer_index = outer_bi->indexes[outer_pos];
+				Assert(outer_index >= 0);
+				merged_index = process_outer_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   outer_index,
+													   inner_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+				merged_datum = outer_datums;
+			}
+
+			/* Move to the next list value on the outer side. */
+			outer_pos++;
+		}
+		else
+		{
+			/* A list value missing from the outer side. */
+			Assert(cmpval > 0);
+			Assert(inner_pos < inner_bi->ndatums);
+
+			/*
+			 * If the outer side has the default partition, or this is a FULL
+			 * join, try to assign a merged partition to the inner partition
+			 * (see process_inner_partition()).  Otherwise, the inner partition
+			 * will not contribute to the result.
+			 */
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				/* Get the inner partition. */
+				inner_index = inner_bi->indexes[inner_pos];
+				Assert(inner_index >= 0);
+				merged_index = process_inner_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   inner_index,
+													   outer_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+				merged_datum = inner_datums;
+			}
+
+			/* Move to the next list value on the inner side. */
+			inner_pos++;
+		}
+
+		/*
+		 * If we assigned a merged partition, add the list value and index of
+		 * the merged partition if appropriate.
+		 */
+		if (merged_index >= 0 && merged_index != default_index)
+		{
+			merged_datums = lappend(merged_datums, merged_datum);
+			merged_indexes = lappend_int(merged_indexes, merged_index);
+		}
+	}
+
+	/*
+	 * If the NULL partitions (if any) have been proven empty, deem them
+	 * non-existent.
+	 */
+	if (outer_has_null &&
+		is_dummy_partition(outer_rel, outer_bi->null_index))
+		outer_has_null = false;
+	if (inner_has_null &&
+		is_dummy_partition(inner_rel, inner_bi->null_index))
+		inner_has_null = false;
+
+	/* Merge the NULL partitions if any. */
+	if (outer_has_null || inner_has_null)
+		merge_null_partitions(&outer_map, &inner_map,
+							  outer_has_null, inner_has_null,
+							  outer_bi->null_index, inner_bi->null_index,
+							  jointype, &next_index, &null_index);
+	else
+		Assert(null_index == -1);
+
+	/* Merge the default partitions if any. */
+	if (outer_has_default || inner_has_default)
+		merge_default_partitions(&outer_map, &inner_map,
+								 outer_has_default, inner_has_default,
+								 outer_default, inner_default,
+								 jointype, &next_index, &default_index);
+	else
+		Assert(default_index == -1);
+
+	/* If we have merged partitions, create the partition bounds. */
+	if (next_index > 0)
+	{
+		/* Fix the merged_indexes list if necessary. */
+		if (outer_map.did_remapping || inner_map.did_remapping)
+		{
+			Assert(jointype == JOIN_FULL);
+			fix_merged_indexes(&outer_map, &inner_map,
+							   next_index, merged_indexes);
+		}
+
+		/* Use maps to match partitions from inputs. */
+		generate_matching_part_pairs(outer_rel, inner_rel,
+									 &outer_map, &inner_map,
+									 next_index,
+									 outer_parts, inner_parts);
+		Assert(*outer_parts != NIL);
+		Assert(*inner_parts != NIL);
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) <= next_index);
+
+		/* Make a PartitionBoundInfo struct to return. */
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  NIL,
+													  merged_indexes,
+													  null_index,
+													  default_index);
+		Assert(merged_bounds);
+	}
+
+cleanup:
+	/* Free local memory before returning. */
+	list_free(merged_datums);
+	list_free(merged_indexes);
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+
+	return merged_bounds;
+}
+
+/*
+ * merge_range_bounds
+ *		Create the partition bounds for a join relation between range
+ *		partitioned tables, if possible
+ *
+ * In this function we try to find overlapping partitions from both sides by
+ * comparing ranges stored in their partition bounds.  Since the ranges
+ * appear in the ascending order, an algorithm similar to merge join is
+ * used for that.  If a partition doesn't have an overlapping partition on the
+ * other side, the algorithm tries to match it with the default partition on
+ * the other side if any; if not, the algorithm tries to match it with a
+ * dummy partition on the other side if it is on the non-nullable side of an
+ * outer join.  Also, if both sides have the default partitions, the algorithm
+ * tries to match them with each other.  We give up if the algorithm finds a
+ * partition overlapping multiple partitions on the other side, which is the
+ * scenario the current implementation of partitioned join can't handle.
+ */
+static PartitionBoundInfo
+merge_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+				   Oid *partcollations,
+				   RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+				   JoinType jointype,
+				   List **outer_parts, List **inner_parts)
+{
+	PartitionBoundInfo merged_bounds = NULL;
+	PartitionBoundInfo outer_bi = outer_rel->boundinfo;
+	PartitionBoundInfo inner_bi = inner_rel->boundinfo;
+	bool		outer_has_default = partition_bound_has_default(outer_bi);
+	bool		inner_has_default = partition_bound_has_default(inner_bi);
+	int			outer_default = outer_bi->default_index;
+	int			inner_default = inner_bi->default_index;
+	PartitionMap outer_map;
+	PartitionMap inner_map;
+	int			outer_index;
+	int			inner_index;
+	int			outer_lb_pos;
+	int			inner_lb_pos;
+	PartitionRangeBound outer_lb;
+	PartitionRangeBound outer_ub;
+	PartitionRangeBound inner_lb;
+	PartitionRangeBound inner_ub;
+	int			next_index = 0;
+	int			default_index = -1;
+	List	   *merged_datums = NIL;
+	List	   *merged_kinds = NIL;
+	List	   *merged_indexes = NIL;
+
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+	Assert(outer_bi->strategy == inner_bi->strategy &&
+		   outer_bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	init_partition_map(outer_rel, &outer_map);
+	init_partition_map(inner_rel, &inner_map);
+
+	/*
+	 * If the default partitions (if any) have been proven empty, deem them
+	 * non-existent.
+	 */
+	if (outer_has_default && is_dummy_partition(outer_rel, outer_default))
+		outer_has_default = false;
+	if (inner_has_default && is_dummy_partition(inner_rel, inner_default))
+		inner_has_default = false;
+
+	/*
+	 * Merge partitions from both sides.  In each iteration we compare a pair
+	 * of ranges, one from each side, and decide whether the corresponding
+	 * partitions match or not.  If the two ranges overlap, move to the next
+	 * pair of ranges, otherwise move to the next range on the side with a
+	 * lower range.  outer_lb_pos/inner_lb_pos keep track of the positions of
+	 * lower bounds in the datums arrays in the outer/inner PartitionBoundInfos
+	 * respectively.
+	 */
+	outer_lb_pos = inner_lb_pos = 0;
+	outer_index = get_range_partition(outer_rel, outer_bi, &outer_lb_pos,
+									  &outer_lb, &outer_ub);
+	inner_index = get_range_partition(inner_rel, inner_bi, &inner_lb_pos,
+									  &inner_lb, &inner_ub);
+	while (outer_index >= 0 || inner_index >= 0)
+	{
+		bool		overlap;
+		int			ub_cmpval;
+		int			lb_cmpval;
+		PartitionRangeBound merged_lb = {-1, NULL, NULL, true};
+		PartitionRangeBound merged_ub = {-1, NULL, NULL, false};
+		int			merged_index = -1;
+
+		/*
+		 * We run this loop till both sides finish.  This allows us to avoid
+		 * duplicating code to handle the remaining ranges on the side which
+		 * finishes later.  For that we set the comparison parameter cmpval in
+		 * such a way that it appears as if the side which finishes earlier has
+		 * an extra range higher than any other range on the unfinished side.
+		 * That way we advance the ranges on the unfinished side till all of
+		 * its ranges are exhausted.
+		 */
+		if (outer_index == -1)
+		{
+			overlap = false;
+			lb_cmpval = 1;
+			ub_cmpval = 1;
+		}
+		else if (inner_index == -1)
+		{
+			overlap = false;
+			lb_cmpval = -1;
+			ub_cmpval = -1;
+		}
+		else
+			overlap = compare_range_partitions(partnatts, partsupfuncs,
+											   partcollations,
+											   &outer_lb, &outer_ub,
+											   &inner_lb, &inner_ub,
+											   &lb_cmpval, &ub_cmpval);
+
+		if (overlap)
+		{
+			/* Two ranges overlap; form a join pair. */
+
+			PartitionRangeBound save_outer_ub;
+			PartitionRangeBound save_inner_ub;
+
+			/* Both partitions should not have been merged yet. */
+			Assert(outer_index >= 0);
+			Assert(outer_map.merged_indexes[outer_index] == -1 &&
+				   outer_map.merged[outer_index] == false);
+			Assert(inner_index >= 0);
+			Assert(inner_map.merged_indexes[inner_index] == -1 &&
+				   inner_map.merged[inner_index] == false);
+
+			/*
+			 * Get the index of the merged partition.  Both partitions aren't
+			 * merged yet, so the partitions should be merged successfully.
+			 */
+			merged_index = merge_matching_partitions(&outer_map, &inner_map,
+													 outer_index, inner_index,
+													 &next_index);
+			Assert(merged_index >= 0);
+
+			/* Get the range of the merged partition. */
+			get_merged_range_bounds(partnatts, partsupfuncs,
+									partcollations, jointype,
+									&outer_lb, &outer_ub,
+									&inner_lb, &inner_ub,
+									lb_cmpval, ub_cmpval,
+									&merged_lb, &merged_ub);
+
+			/* Save the upper bounds of both partitions for use below. */
+			save_outer_ub = outer_ub;
+			save_inner_ub = inner_ub;
+
+			/* Move to the next pair of ranges. */
+			outer_index = get_range_partition(outer_rel, outer_bi, &outer_lb_pos,
+											  &outer_lb, &outer_ub);
+			inner_index = get_range_partition(inner_rel, inner_bi, &inner_lb_pos,
+											  &inner_lb, &inner_ub);
+
+			/*
+			 * If the range of a partition on one side overlaps the range of
+			 * the next partition on the other side, that will cause the
+			 * partition on one side to match at least two partitions on the
+			 * other side, which is the case that we currently don't support
+			 * partitioned join for; give up.
+			 */
+			if (ub_cmpval > 0 && inner_index >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &save_outer_ub, &inner_lb) > 0)
+				goto cleanup;
+			if (ub_cmpval < 0 && outer_index >= 0 &&
+				compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									 &outer_lb, &save_inner_ub) < 0)
+				goto cleanup;
+
+			/*
+			 * A row from a non-overlapping portion (if any) of a partition
+			 * on one side might find its join partner in the default
+			 * partition (if any) on the other side, causing the same
+			 * situation as above; give up in that case.
+			 */
+			if ((outer_has_default && (lb_cmpval > 0 || ub_cmpval < 0)) ||
+				(inner_has_default && (lb_cmpval < 0 || ub_cmpval > 0)))
+				goto cleanup;
+		}
+		else if (ub_cmpval < 0)
+		{
+			/* A non-overlapping outer range. */
+
+			/* The outer partition should not have been merged yet. */
+			Assert(outer_index >= 0);
+			Assert(outer_map.merged_indexes[outer_index] == -1 &&
+				   outer_map.merged[outer_index] == false);
+
+			/*
+			 * If the inner side has the default partition, or this is an outer
+			 * join, try to assign a merged partition to the outer partition
+			 * (see process_outer_partition()).  Otherwise, the outer partition
+			 * will not contribute to the result.
+			 */
+			if (inner_has_default || IS_OUTER_JOIN(jointype))
+			{
+				merged_index = process_outer_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   outer_index,
+													   inner_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+				merged_lb = outer_lb;
+				merged_ub = outer_ub;
+			}
+
+			/* Move to the next range on the outer side. */
+			outer_index = get_range_partition(outer_rel, outer_bi, &outer_lb_pos,
+											  &outer_lb, &outer_ub);
+		}
+		else
+		{
+			/* A non-overlapping inner range. */
+			Assert(ub_cmpval > 0);
+
+			/* The inner partition should not have been merged yet. */
+			Assert(inner_index >= 0);
+			Assert(inner_map.merged_indexes[inner_index] == -1 &&
+				   inner_map.merged[inner_index] == false);
+
+			/*
+			 * If the outer side has the default partition, or this is a FULL
+			 * join, try to assign a merged partition to the inner partition
+			 * (see process_inner_partition()).  Otherwise, the inner partition
+			 * will not contribute to the result.
+			 */
+			if (outer_has_default || jointype == JOIN_FULL)
+			{
+				merged_index = process_inner_partition(&outer_map,
+													   &inner_map,
+													   outer_has_default,
+													   inner_has_default,
+													   inner_index,
+													   outer_default,
+													   jointype,
+													   &next_index,
+													   &default_index);
+				if (merged_index == -1)
+					goto cleanup;
+				merged_lb = inner_lb;
+				merged_ub = inner_ub;
+			}
+
+			/* Move to the next range on the inner side. */
+			inner_index = get_range_partition(inner_rel, inner_bi, &inner_lb_pos,
+											  &inner_lb, &inner_ub);
+		}
+
+		/*
+		 * If we assigned a merged partition, add the range bounds and index of
+		 * the merged partition if appropriate.
+		 */
+		if (merged_index >= 0 && merged_index != default_index)
+			add_merged_range_bounds(partnatts, partsupfuncs, partcollations,
+									&merged_lb, &merged_ub, merged_index,
+									&merged_datums, &merged_kinds,
+									&merged_indexes);
+	}
+
+	/* Merge the default partitions if any. */
+	if (outer_has_default || inner_has_default)
+		merge_default_partitions(&outer_map, &inner_map,
+								 outer_has_default, inner_has_default,
+								 outer_default, inner_default,
+								 jointype, &next_index, &default_index);
+	else
+		Assert(default_index == -1);
+
+	/* If we have merged partitions, create the partition bounds. */
+	if (next_index > 0)
+	{
+		/*
+		 * Unlike the case of list partitioning, we wouldn't have re-merged
+		 * partitions, so did_remapping should be left alone.
+		 */
+		Assert(!outer_map.did_remapping);
+		Assert(!inner_map.did_remapping);
+
+		/* Use maps to match partitions from inputs. */
+		generate_matching_part_pairs(outer_rel, inner_rel,
+									 &outer_map, &inner_map,
+									 next_index,
+									 outer_parts, inner_parts);
+		Assert(*outer_parts != NIL);
+		Assert(*inner_parts != NIL);
+		Assert(list_length(*outer_parts) == list_length(*inner_parts));
+		Assert(list_length(*outer_parts) == next_index);
+
+		/* Make a PartitionBoundInfo struct to return. */
+		merged_bounds = build_merged_partition_bounds(outer_bi->strategy,
+													  merged_datums,
+													  merged_kinds,
+													  merged_indexes,
+													  -1,
+													  default_index);
+		Assert(merged_bounds);
+	}
+
+cleanup:
+	/* Free local memory before returning. */
+	list_free(merged_datums);
+	list_free(merged_kinds);
+	list_free(merged_indexes);
+	free_partition_map(&outer_map);
+	free_partition_map(&inner_map);
+
+	return merged_bounds;
+}
+
+/*
+ * init_partition_map
+ *		Initialize a PartitionMap struct for given relation
+ */
+static void
+init_partition_map(RelOptInfo *rel, PartitionMap *map)
+{
+	int			nparts = rel->nparts;
+	int			i;
+
+	map->nparts = nparts;
+	map->merged_indexes = (int *) palloc(sizeof(int) * nparts);
+	map->merged = (bool *) palloc(sizeof(bool) * nparts);
+	map->did_remapping = false;
+	map->old_indexes = (int *) palloc(sizeof(int) * nparts);
+	for (i = 0; i < nparts; i++)
+	{
+		map->merged_indexes[i] = map->old_indexes[i] = -1;
+		map->merged[i] = false;
+	}
+}
+
+/*
+ * free_partition_map
+ */
+static void
+free_partition_map(PartitionMap *map)
+{
+	pfree(map->merged_indexes);
+	pfree(map->merged);
+	pfree(map->old_indexes);
+}
+
+/*
+ * is_dummy_partition --- has partition been proven empty?
+ */
+static bool
+is_dummy_partition(RelOptInfo *rel, int part_index)
+{
+	RelOptInfo *part_rel;
+
+	Assert(part_index >= 0);
+	part_rel = rel->part_rels[part_index];
+	if (part_rel == NULL || IS_DUMMY_REL(part_rel))
+		return true;
+	return false;
+}
+
+/*
+ * merge_matching_partitions
+ *		Try to merge given outer/inner partitions, and return the index of a
+ *		merged partition produced from them if successful, -1 otherwise
+ *
+ * If the merged partition is newly created, *next_index is incremented.
+ */
+static int
+merge_matching_partitions(PartitionMap *outer_map, PartitionMap *inner_map,
+						  int outer_index, int inner_index, int *next_index)
+{
+	int 		outer_merged_index;
+	int 		inner_merged_index;
+	bool 		outer_merged;
+	bool 		inner_merged;
+
+	Assert(outer_index >= 0 && outer_index < outer_map->nparts);
+	outer_merged_index = outer_map->merged_indexes[outer_index];
+	outer_merged = outer_map->merged[outer_index];
+	Assert(inner_index >= 0 && inner_index < inner_map->nparts);
+	inner_merged_index = inner_map->merged_indexes[inner_index];
+	inner_merged = inner_map->merged[inner_index];
+
+	/*
+	 * Handle cases where we have already assigned a merged partition to each
+	 * of the given partitions.
+	 */
+	if (outer_merged_index >= 0 && inner_merged_index >= 0)
+	{
+		/*
+		 * If the mereged partitions are the same, no need to do anything;
+		 * return the index of the merged partitions.  Otherwise, if each of
+		 * the given partitions has been merged with a dummy partition on the
+		 * other side, re-map them to either of the two merged partitions.
+		 * Otherwise, they can't be merged, so return -1.
+		 */
+		if (outer_merged_index == inner_merged_index)
+		{
+			Assert(outer_merged);
+			Assert(inner_merged);
+			return outer_merged_index;
+		}
+		if (!outer_merged && !inner_merged)
+		{
+			/*
+			 * This can only happen for a list-partitioning case.  We re-map
+			 * them to the merged partition with the smaller of the two merged
+			 * indexes to preserve the property that the canonical order of
+			 * list partitions is determined by the indexes assigned to the
+			 * smallest list value of each partition.
+			 */
+			if (outer_merged_index < inner_merged_index)
+			{
+				outer_map->merged[outer_index] = true;
+				inner_map->merged_indexes[inner_index] = outer_merged_index;
+				inner_map->merged[inner_index] = true;
+				inner_map->did_remapping = true;
+				inner_map->old_indexes[inner_index] = inner_merged_index;
+				return outer_merged_index;
+			}
+			else
+			{
+				inner_map->merged[inner_index] = true;
+				outer_map->merged_indexes[outer_index] = inner_merged_index;
+				outer_map->merged[outer_index] = true;
+				outer_map->did_remapping = true;
+				outer_map->old_indexes[outer_index] = outer_merged_index;
+				return inner_merged_index;
+			}
+		}
+		return -1;
+	}
+
+	/* At least one of the given partitions should not have yet been merged. */
+	Assert(outer_merged_index == -1 || inner_merged_index == -1);
+
+	/*
+	 * If neither of them has been merged, merge them.  Otherwise, if one has
+	 * been merged with a dummy relation on the other side (and the other
+	 * hasn't yet been merged with anything), re-merge them.  Otherwise, they
+	 * can't be merged, so return -1.
+	 */
+	if (outer_merged_index == -1 && inner_merged_index == -1)
+	{
+		int 		merged_index = *next_index;
+
+		Assert(!outer_merged);
+		Assert(!inner_merged);
+		outer_map->merged_indexes[outer_index] = merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged_indexes[inner_index] = merged_index;
+		inner_map->merged[inner_index] = true;
+		*next_index = *next_index + 1;
+		return merged_index;
+	}
+	if (outer_merged_index >= 0 && !outer_map->merged[outer_index])
+	{
+		Assert(inner_merged_index == -1);
+		Assert(!inner_merged);
+		inner_map->merged_indexes[inner_index] = outer_merged_index;
+		inner_map->merged[inner_index] = true;
+		outer_map->merged[outer_index] = true;
+		return outer_merged_index;
+	}
+	if (inner_merged_index >= 0 && !inner_map->merged[inner_index])
+	{
+		Assert(outer_merged_index == -1);
+		Assert(!outer_merged);
+		outer_map->merged_indexes[outer_index] = inner_merged_index;
+		outer_map->merged[outer_index] = true;
+		inner_map->merged[inner_index] = true;
+		return inner_merged_index;
+	}
+	return -1;
+}
+
+/*
+ * process_outer_partition
+ *		Try to assign given outer partition a merged partition, and return the
+ *		index of the merged partition if successful, -1 otherwise
+ *
+ * If the partition is newly created, *next_index is incremented.  Also, if it
+ * is the default partition of the join relation, *default_index is set to the
+ * index if not already done.
+ */
+static int
+process_outer_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int outer_index,
+						int inner_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index)
+{
+	int 		merged_index = -1;
+
+	Assert(outer_index >= 0);
+
+	/*
+	 * If the inner side has the default partition, a row from the outer
+	 * partition might find its join partner in the default partition; try
+	 * merging the outer partition with the default partition.  Otherwise, this
+	 * should be an outer join, in which case the outer partition has to be
+	 * scanned all the way anyway; merge the outer partition with a dummy
+	 * partition on the other side.
+	 */
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0);
+
+		/*
+		 * If the outer side has the default partition as well, the default
+		 * partition on the inner side will have two matching partitions on the
+		 * other side: the outer partition and the default partition on the
+		 * outer side.  Partitionwise join doesn't handle this scenario yet.
+		 */
+		if (outer_has_default)
+			return -1;
+
+		merged_index = merge_matching_partitions(outer_map, inner_map,
+												 outer_index, inner_default,
+												 next_index);
+		if (merged_index == -1)
+			return -1;
+
+		/*
+		 * If this is a FULL join, the default partition on the inner side
+		 * has to be scanned all the way anyway, so the resulting partition
+		 * will contain all key values from the default partition, which any
+		 * other partition of the join relation will not contain.  Thus the
+		 * resulting partition will act as the default partition of the join
+		 * relation; record the index in *default_index if not already done.
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (*default_index == -1)
+				*default_index = merged_index;
+			else
+				Assert(*default_index == merged_index);
+		}
+	}
+	else
+	{
+		Assert(IS_OUTER_JOIN(jointype));
+		Assert(jointype != JOIN_RIGHT);
+
+		/* If we have already assigned a partition, no need to do anything. */
+		merged_index = outer_map->merged_indexes[outer_index];
+		if (merged_index == -1)
+			merged_index = merge_partition_with_dummy(outer_map, outer_index,
+													  next_index);
+	}
+	return merged_index;
+}
+
+/*
+ * process_inner_partition
+ *		Try to assign given inner partition a merged partition, and return the
+ *		index of the merged partition if successful, -1 otherwise
+ *
+ * If the partition is newly created, *next_index is incremented.  Also, if it
+ * is the default partition of the join relation, *default_index is set to the
+ * index if not already done.
+ */
+static int
+process_inner_partition(PartitionMap *outer_map,
+						PartitionMap *inner_map,
+						bool outer_has_default,
+						bool inner_has_default,
+						int inner_index,
+						int outer_default,
+						JoinType jointype,
+						int *next_index,
+						int *default_index)
+{
+	int 		merged_index = -1;
+
+	Assert(inner_index >= 0);
+
+	/*
+	 * If the outer side has the default partition, a row from the inner
+	 * partition might find its join partner in the default partition; try
+	 * merging the inner partition with the default partition.  Otherwise, this
+	 * should be a FULL join, in which case the inner partition has to be
+	 * scanned all the way anyway; merge the inner partition with a dummy
+	 * partition on the other side.
+	 */
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0);
+
+		/*
+		 * If the inner side has the default partition as well, the default
+		 * partition on the outer side will have two matching partitions on the
+		 * other side: the inner partition and the default partition on the
+		 * inner side.  Partitionwise join doesn't handle this scenario yet.
+		 */
+		if (inner_has_default)
+			return -1;
+
+		merged_index = merge_matching_partitions(outer_map, inner_map,
+												 outer_default, inner_index,
+												 next_index);
+		if (merged_index == -1)
+			return -1;
+
+		/*
+		 * If this is an outer join, the default partition on the outer side
+		 * has to be scanned all the way anyway, so the resulting partition
+		 * will contain all key values from the default partition, which any
+		 * other partition of the join relation will not contain.  Thus the
+		 * resulting partition will act as the default partition of the join
+		 * relation; record the index in *default_index if not already done.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			if (*default_index == -1)
+				*default_index = merged_index;
+			else
+				Assert(*default_index == merged_index);
+		}
+	}
+	else
+	{
+		Assert(jointype == JOIN_FULL);
+
+		/* If we have already assigned a partition, no need to do anything. */
+		merged_index = inner_map->merged_indexes[inner_index];
+		if (merged_index == -1)
+			merged_index = merge_partition_with_dummy(inner_map, inner_index,
+													  next_index);
+	}
+	return merged_index;
+}
+
+/*
+ * merge_null_partitions
+ *		Merge the NULL partitions from a join's outer and inner sides.
+ *
+ * If the merged partition produced from them is the NULL partition of the join
+ * relation, *null_index is set to the index of the merged partition.
+ *
+ * Note: We assume here that the join clause for a partitioned join is strict
+ * because have_partkey_equi_join() requires that the corresponding operator
+ * be mergejoinable, and we currently assume that mergejoinable operators are
+ * strict (see MJEvalOuterValues()/MJEvalInnerValues()).
+ */
+static void
+merge_null_partitions(PartitionMap *outer_map,
+					  PartitionMap *inner_map,
+					  bool outer_has_null,
+					  bool inner_has_null,
+					  int outer_null,
+					  int inner_null,
+					  JoinType jointype,
+					  int *next_index,
+					  int *null_index)
+{
+	bool 		consider_outer_null = false;
+	bool 		consider_inner_null = false;
+
+	Assert(outer_has_null || inner_has_null);
+	Assert(*null_index == -1);
+
+	/*
+	 * Check whether the NULL partitions have already been merged and if so,
+	 * set the consider_outer_null/consider_inner_null flags.
+	 */
+	if (outer_has_null)
+	{
+		Assert(outer_null >= 0 && outer_null < outer_map->nparts);
+		if (outer_map->merged_indexes[outer_null] == -1)
+			consider_outer_null = true;
+	}
+	if (inner_has_null)
+	{
+		Assert(inner_null >= 0 && inner_null < inner_map->nparts);
+		if (inner_map->merged_indexes[inner_null] == -1)
+			consider_inner_null = true;
+	}
+
+	/* If both flags are set false, we don't need to do anything. */
+	if (!consider_outer_null && !consider_inner_null)
+		return;
+
+	if (consider_outer_null && !consider_inner_null)
+	{
+		Assert(outer_has_null);
+
+		/*
+		 * If this is an outer join, the NULL partition on the outer side has
+		 * to be scanned all the way anyway; merge the NULL partition with a
+		 * dummy partition on the other side.  In that case consider_outer_null
+		 * means that the NULL partition only contains NULL values as the key
+		 * values, so the merged partition will do so; treat it as the NULL
+		 * partition of the join relation.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			*null_index = merge_partition_with_dummy(outer_map, outer_null,
+													 next_index);
+		}
+	}
+	else if (!consider_outer_null && consider_inner_null)
+	{
+		Assert(inner_has_null);
+
+		/*
+		 * If this is a FULL join, the NULL partition on the inner side has
+		 * to be scanned all the way anyway; merge the NULL partition with a
+		 * dummy partition on the other side.  In that case consider_inner_null
+		 * means that the NULL partition only contains NULL values as the key
+		 * values, so the merged partition will do so; treat it as the NULL
+		 * partition of the join relation.
+		 */
+		if (jointype == JOIN_FULL)
+			*null_index = merge_partition_with_dummy(inner_map, inner_null,
+													 next_index);
+	}
+	else
+	{
+		Assert(consider_outer_null && consider_inner_null);
+		Assert(outer_has_null);
+		Assert(inner_has_null);
+
+		/*
+		 * If this is an outer join, the NULL partition on the outer side (and
+		 * that on the inner side if this is a FULL join) have to be scanned
+		 * all the way anyway, so merge them.  Note that each of the NULL
+		 * partitions isn't merged yet, so they should be merged successfully.
+		 * Like the above, each of the NULL partitions only contains NULL
+		 * values as the key values, so the merged partition will do so; treat
+		 * it as the NULL partition of the join relation.
+		 *
+		 * Note: if this an INNER/SEMI join, the join clause will never be
+		 * satisfied by two NULL values (see comments above), so both the NULL
+		 * partitions can be eliminated.
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			*null_index = merge_matching_partitions(outer_map, inner_map,
+													outer_null, inner_null,
+													next_index);
+			Assert(*null_index >= 0);
+		}
+	}
+}
+
+/*
+ * merge_default_partitions
+ *		Merge the default partitions from a join's outer and inner sides.
+ *
+ * If the merged partition produced from them is the default partition of the
+ * join relation, *default_index is set to the index of the merged partition.
+ */
+static void
+merge_default_partitions(PartitionMap *outer_map,
+						 PartitionMap *inner_map,
+						 bool outer_has_default,
+						 bool inner_has_default,
+						 int outer_default,
+						 int inner_default,
+						 JoinType jointype,
+						 int *next_index,
+						 int *default_index)
+{
+	int 		outer_merged_index = -1;
+	int 		inner_merged_index = -1;
+
+	Assert(outer_has_default || inner_has_default);
+
+	/* Get the merged partition indexes for the default partitions. */
+	if (outer_has_default)
+	{
+		Assert(outer_default >= 0 && outer_default < outer_map->nparts);
+		outer_merged_index = outer_map->merged_indexes[outer_default];
+	}
+	if (inner_has_default)
+	{
+		Assert(inner_default >= 0 && inner_default < inner_map->nparts);
+		inner_merged_index = inner_map->merged_indexes[inner_default];
+	}
+
+	if (outer_has_default && !inner_has_default)
+	{
+		/*
+		 * If this is an outer join, the default partition on the outer side
+		 * has to be scanned all the way anyway; if we have not yet assigned a
+		 * partition, merge the default partition with a dummy partition on the
+		 * other side.  The merged partition will act as the default partition
+		 * of the join relation (see comments in process_inner_partition()).
+		 */
+		if (IS_OUTER_JOIN(jointype))
+		{
+			Assert(jointype != JOIN_RIGHT);
+			if (outer_merged_index == -1)
+			{
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(outer_map,
+															outer_default,
+															next_index);
+			}
+			else
+				Assert(*default_index == outer_merged_index);
+		}
+		else
+			Assert(*default_index == -1);
+	}
+	else if (!outer_has_default && inner_has_default)
+	{
+		/*
+		 * If this is a FULL join, the default partition on the inner side
+		 * has to be scanned all the way anyway; if we have not yet assigned a
+		 * partition, merge the default partition with a dummy partition on the
+		 * other side.  The merged partition will act as the default partition
+		 * of the join relation (see comments in process_outer_partition()).
+		 */
+		if (jointype == JOIN_FULL)
+		{
+			if (inner_merged_index == -1)
+			{
+				Assert(*default_index == -1);
+				*default_index = merge_partition_with_dummy(inner_map,
+															inner_default,
+															next_index);
+			}
+			else
+				Assert(*default_index == inner_merged_index);
+		}
+		else
+			Assert(*default_index == -1);
+	}
+	else
+	{
+		Assert(outer_has_default && inner_has_default);
+
+		/*
+		 * The default partitions have to be joined with each other, so merge
+		 * them.  Note that each of the default partitions isn't merged yet
+		 * (see, process_outer_partition()/process_innerer_partition()), so
+		 * they should be merged successfully.  The merged partition will act
+		 * as the default partition of the join relation.
+		 */
+		Assert(outer_merged_index == -1);
+		Assert(inner_merged_index == -1);
+		Assert(*default_index == -1);
+		*default_index = merge_matching_partitions(outer_map,
+												   inner_map,
+												   outer_default,
+												   inner_default,
+												   next_index);
+		Assert(*default_index >= 0);
+	}
+}
+
+/*
+ * merge_partition_with_dummy
+ *		Assign given partition a new partition of a join relation
+ *
+ * Note: The caller assumes that the given partition doesn't have a non-dummy
+ * matching partition on the other side, but if the given partition finds the
+ * matching partition later, we will adjust the assignment.
+ */
+static int
+merge_partition_with_dummy(PartitionMap *map, int index, int *next_index)
+{
+	int 		merged_index = *next_index;
+
+	Assert(index >= 0 && index < map->nparts);
+	Assert(map->merged_indexes[index] == -1);
+	Assert(!map->merged[index]);
+	map->merged_indexes[index] = merged_index;
+	/* Leave the merged flag alone! */
+	*next_index = *next_index + 1;
+	return merged_index;
+}
+
+/*
+ * fix_merged_indexes
+ *		Adjust merged indexes of re-merged partitions
+ */
+static void
+fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map,
+				   int nmerged, List *merged_indexes)
+{
+	int		   *new_indexes;
+	int			merged_index;
+	int			i;
+	ListCell   *lc;
+
+	Assert(nmerged > 0);
+
+	new_indexes = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		new_indexes[i] = -1;
+
+	/* Build the mapping of old merged indexes to new merged indexes. */
+	if (outer_map->did_remapping)
+	{
+		for (i = 0; i < outer_map->nparts; i++)
+		{
+			merged_index = outer_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = outer_map->merged_indexes[i];
+		}
+	}
+	if (inner_map->did_remapping)
+	{
+		for (i = 0; i < inner_map->nparts; i++)
+		{
+			merged_index = inner_map->old_indexes[i];
+			if (merged_index >= 0)
+				new_indexes[merged_index] = inner_map->merged_indexes[i];
+		}
+	}
+
+	/* Fix the merged_indexes list using the mapping. */
+	foreach(lc, merged_indexes)
+	{
+		merged_index = lfirst_int(lc);
+		Assert(merged_index >= 0);
+		if (new_indexes[merged_index] >= 0)
+			lfirst_int(lc) = new_indexes[merged_index];
+	}
+
+	pfree(new_indexes);
+}
+
+/*
+ * generate_matching_part_pairs
+ *		Generate a pair of lists of partitions that produce merged partitions
+ *
+ * The lists of partitions are built in the order of merged partition indexes,
+ * and returned in *outer_parts and *inner_parts.
+ */
+static void
+generate_matching_part_pairs(RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+							 PartitionMap *outer_map, PartitionMap *inner_map,
+							 int nmerged,
+							 List **outer_parts, List **inner_parts)
+{
+	int			outer_nparts = outer_map->nparts;
+	int			inner_nparts = inner_map->nparts;
+	int		   *outer_indexes;
+	int		   *inner_indexes;
+	int			max_nparts;
+	int 		i;
+
+	Assert(nmerged > 0);
+	Assert(*outer_parts == NIL);
+	Assert(*inner_parts == NIL);
+
+	outer_indexes = (int *) palloc(sizeof(int) * nmerged);
+	inner_indexes = (int *) palloc(sizeof(int) * nmerged);
+	for (i = 0; i < nmerged; i++)
+		outer_indexes[i] = inner_indexes[i] = -1;
+
+	/* Set pairs of matching partitions. */
+	Assert(outer_nparts == outer_rel->nparts);
+	Assert(inner_nparts == inner_rel->nparts);
+	max_nparts = Max(outer_nparts, inner_nparts);
+	for (i = 0; i < max_nparts; i++)
+	{
+		if (i < outer_nparts)
+		{
+			int 		merged_index = outer_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				outer_indexes[merged_index] = i;
+			}
+		}
+		if (i < inner_nparts)
+		{
+			int 		merged_index = inner_map->merged_indexes[i];
+
+			if (merged_index >= 0)
+			{
+				Assert(merged_index < nmerged);
+				inner_indexes[merged_index] = i;
+			}
+		}
+	}
+
+	/* Build the list pairs. */
+	for (i = 0; i < nmerged; i++)
+	{
+		int			outer_index = outer_indexes[i];
+		int			inner_index = inner_indexes[i];
+
+		/*
+		 * If both partitions are dummy, it means the merged partition that had
+		 * been assigned to the outer/inner partition was removed when
+		 * re-merging the outer/inner partition in merge_matching_partitions();
+		 * ignore the merged partition.
+		 */
+		if (outer_index == -1 && inner_index == -1)
+			continue;
+
+		*outer_parts = lappend(*outer_parts, outer_index >= 0 ?
+							   outer_rel->part_rels[outer_index] : NULL);
+		*inner_parts = lappend(*inner_parts, inner_index >= 0 ?
+							   inner_rel->part_rels[inner_index] : NULL);
+	}
+
+	pfree(outer_indexes);
+	pfree(inner_indexes);
+}
+
+/*
+ * build_merged_partition_bounds
+ *		Create a PartitionBoundInfo struct from merged partition bounds
+ */
+static PartitionBoundInfo
+build_merged_partition_bounds(char strategy, List *merged_datums,
+							  List *merged_kinds, List *merged_indexes,
+							  int null_index, int default_index)
+{
+	PartitionBoundInfo merged_bounds;
+	int			ndatums = list_length(merged_datums);
+	int			pos;
+	ListCell   *lc;
+
+	merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+	merged_bounds->strategy = strategy;
+	merged_bounds->ndatums = ndatums;
+
+	merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) * ndatums);
+	pos = 0;
+	foreach(lc, merged_datums)
+		merged_bounds->datums[pos++] = (Datum *) lfirst(lc);
+
+	if (strategy == PARTITION_STRATEGY_RANGE)
+	{
+		Assert(list_length(merged_kinds) == ndatums);
+		merged_bounds->kind = (PartitionRangeDatumKind **)
+			palloc(sizeof(PartitionRangeDatumKind *) * ndatums);
+		pos = 0;
+		foreach(lc, merged_kinds)
+			merged_bounds->kind[pos++] = (PartitionRangeDatumKind *) lfirst(lc);
+
+		/* There are ndatums+1 indexes in the case of range partitioning. */
+		merged_indexes = lappend_int(merged_indexes, -1);
+		ndatums++;
+	}
+	else
+	{
+		Assert(strategy == PARTITION_STRATEGY_LIST);
+		Assert(merged_kinds == NIL);
+		merged_bounds->kind = NULL;
+	}
+
+	Assert(list_length(merged_indexes) == ndatums);
+	merged_bounds->indexes = (int *) palloc(sizeof(int) * ndatums);
+	pos = 0;
+	foreach(lc, merged_indexes)
+		merged_bounds->indexes[pos++] = lfirst_int(lc);
+
+	merged_bounds->null_index = null_index;
+	merged_bounds->default_index = default_index;
+
+	return merged_bounds;
+}
+
+/*
+ * get_range_partition
+ *		Get the next non-dummy partition of a range-partitioned relation,
+ *		returning the index of that partition
+ *
+ * *lb and *ub are set to the lower and upper bounds of that partition
+ * respectively, and *lb_pos is advanced to the next lower bound, if any.
+ */
+static int
+get_range_partition(RelOptInfo *rel,
+					PartitionBoundInfo bi,
+					int *lb_pos,
+					PartitionRangeBound *lb,
+					PartitionRangeBound *ub)
+{
+	int			part_index;
+
+	Assert(bi->strategy == PARTITION_STRATEGY_RANGE);
+
+	do {
+		part_index = get_range_partition_internal(bi, lb_pos, lb, ub);
+		if (part_index == -1)
+			return -1;
+	} while (is_dummy_partition(rel, part_index));
+
+	return part_index;
+}
+
+static int
+get_range_partition_internal(PartitionBoundInfo bi,
+							 int *lb_pos,
+							 PartitionRangeBound *lb,
+							 PartitionRangeBound *ub)
+{
+	/* Return the index as -1 if we've exhausted all lower bounds. */
+	if (*lb_pos >= bi->ndatums)
+		return -1;
+
+	/* A lower bound should have at least one more bound after it. */
+	Assert(*lb_pos + 1 < bi->ndatums);
+
+	/* Set the lower bound. */
+	lb->index = bi->indexes[*lb_pos];
+	lb->datums = bi->datums[*lb_pos];
+	lb->kind = bi->kind[*lb_pos];
+	lb->lower = true;
+	/* Set the upper bound. */
+	ub->index = bi->indexes[*lb_pos + 1];
+	ub->datums = bi->datums[*lb_pos + 1];
+	ub->kind = bi->kind[*lb_pos + 1];
+	ub->lower = false;
+
+	/* The index assigned to an upper bound should be valid. */
+	Assert(ub->index >= 0);
+
+	/*
+	 * Advance the position to the next lower bound.  If there are no bounds
+	 * left beyond the upper bound, we have reached the last lower bound.
+	 */
+	if (*lb_pos + 2 >= bi->ndatums)
+		*lb_pos = bi->ndatums;
+	else
+	{
+		/*
+		 * If the index assigned to the bound next to the upper bound isn't
+		 * valid, that is the next lower bound; else, the upper bound is also
+		 * the lower bound of the next range partition.
+		 */
+		if (bi->indexes[*lb_pos + 2] < 0)
+			*lb_pos = *lb_pos + 2;
+		else
+			*lb_pos = *lb_pos + 1;
+	}
+
+	return ub->index;
+}
+
+/*
+ * compare_range_partitions
+ *		Compare the bounds of two range partitions, and return true if the
+ *		two partitions overlap, false otherwise
+ *
+ * *lb_cmpval is set to -1, 0, or 1 if the outer partition's lower bound is
+ * lower than, equal to, or higher than the inner partition's lower bound
+ * respectively.  Likewise, *ub_cmpval is set to -1, 0, or 1 if the outer
+ * partition's upper bound is lower than, equal to, or higher than the inner
+ * partition's upper bound respectively.
+ */
+static bool
+compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs,
+						 Oid *partcollations,
+						 PartitionRangeBound *outer_lb,
+						 PartitionRangeBound *outer_ub,
+						 PartitionRangeBound *inner_lb,
+						 PartitionRangeBound *inner_ub,
+						 int *lb_cmpval, int *ub_cmpval)
+{
+	/*
+	 * Check if the outer partition's upper bound is lower than the inner
+	 * partition's lower bound; if so the partitions aren't overlapping.
+	 */
+	if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+							 outer_ub, inner_lb) < 0)
+	{
+		*lb_cmpval = -1;
+		*ub_cmpval = -1;
+		return false;
+	}
+
+	/*
+	 * Check if the outer partition's lower bound is higher than the inner
+	 * partition's upper bound; if so the partitions aren't overlapping.
+	 */
+	if (compare_range_bounds(partnatts, partsupfuncs, partcollations,
+							 outer_lb, inner_ub) > 0)
+	{
+		*lb_cmpval = 1;
+		*ub_cmpval = 1;
+		return false;
+	}
+
+	/* All other cases indicate overlapping partitions. */
+	*lb_cmpval = compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									  outer_lb, inner_lb);
+	*ub_cmpval = compare_range_bounds(partnatts, partsupfuncs, partcollations,
+									  outer_ub, inner_ub);
+	return true;
+}
+
+/*
+ * get_merged_range_bounds
+ *		Given the bounds of range partitions to be joined, determine the bounds
+ *		of a merged partition produced from the range partitions
+ *
+ * *merged_lb and *merged_ub are set to the lower and upper bounds of the
+ * merged partition.
+ */
+static void
+get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations, JoinType jointype,
+						PartitionRangeBound *outer_lb,
+						PartitionRangeBound *outer_ub,
+						PartitionRangeBound *inner_lb,
+						PartitionRangeBound *inner_ub,
+						int	lb_cmpval, int ub_cmpval,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub)
+{
+	Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								outer_lb, inner_lb) == lb_cmpval);
+	Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations,
+								outer_ub, inner_ub) == ub_cmpval);
+
+	switch (jointype)
+	{
+		case JOIN_INNER:
+		case JOIN_SEMI:
+
+			/*
+			 * An INNER/SEMI join will have the rows that fit both sides, so
+			 * the lower bound of the merged partition will be the higher of
+			 * the two lower bounds, and the upper bound of the merged
+			 * partition will be the lower of the two upper bounds.
+			 */
+			*merged_lb = (lb_cmpval > 0) ? *outer_lb : *inner_lb;
+			*merged_ub = (ub_cmpval < 0) ? *outer_ub : *inner_ub;
+			break;
+
+		case JOIN_LEFT:
+		case JOIN_ANTI:
+
+			/*
+			 * A LEFT/ANTI join will have all the rows from the outer side, so
+			 * the bounds of the merged partition will be the same as the outer
+			 * bounds.
+			 */
+			*merged_lb = *outer_lb;
+			*merged_ub = *outer_ub;
+			break;
+
+		case JOIN_FULL:
+
+			/*
+			 * A FULL join will have all the rows from both sides, so the lower
+			 * bound of the merged partition will be the lower of the two lower
+			 * bounds, and the upper bound of the merged partition will be the
+			 * higher of the two upper bounds.
+			 */
+			*merged_lb = (lb_cmpval < 0) ? *outer_lb : *inner_lb;
+			*merged_ub = (ub_cmpval > 0) ? *outer_ub : *inner_ub;
+			break;
+
+		default:
+			elog(ERROR, "unrecognized join type: %d", (int) jointype);
+	}
+}
+
+/*
+ * add_merged_range_bounds
+ *		Add the bounds of a merged partition to the lists of range bounds
+ */
+static void
+add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs,
+						Oid *partcollations,
+						PartitionRangeBound *merged_lb,
+						PartitionRangeBound *merged_ub,
+						int merged_index,
+						List **merged_datums,
+						List **merged_kinds,
+						List **merged_indexes)
+{
+	int			cmpval;
+
+	if (!*merged_datums)
+	{
+		/* First merged partition */
+		Assert(!*merged_kinds);
+		Assert(!*merged_indexes);
+		cmpval = 1;
+	}
+	else
+	{
+		PartitionRangeBound	prev_ub;
+
+		Assert(*merged_datums);
+		Assert(*merged_kinds);
+		Assert(*merged_indexes);
+
+		/* Get the last upper bound. */
+		prev_ub.index = llast_int(*merged_indexes);
+		prev_ub.datums = (Datum *) llast(*merged_datums);
+		prev_ub.kind = (PartitionRangeDatumKind *) llast(*merged_kinds);
+		prev_ub.lower = false;
+
+		/*
+		 * We pass to partition_rbound_cmp() lower1 as false to prevent it
+		 * from considering the last upper bound to be smaller than the lower
+		 * bound of the merged partition when the values of the two range
+		 * bounds compare equal.
+		 */
+		cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations,
+									  merged_lb->datums, merged_lb->kind,
+									  false, &prev_ub);
+		Assert(cmpval >= 0);
+	}
+
+	/*
+	 * If the lower bound is higher than the last upper bound, add the lower
+	 * bound with the index as -1 indicating that that is a lower bound; else,
+	 * the last upper bound will be reused as the lower bound of the merged
+	 * partition, so skip this.
+	 */
+	if (cmpval > 0)
+	{
+		*merged_datums = lappend(*merged_datums, merged_lb->datums);
+		*merged_kinds = lappend(*merged_kinds, merged_lb->kind);
+		*merged_indexes = lappend_int(*merged_indexes, -1);
+	}
+
+	/* Add the upper bound and index of the merged partition. */
+	*merged_datums = lappend(*merged_datums, merged_ub->datums);
+	*merged_kinds = lappend(*merged_kinds, merged_ub->kind);
+	*merged_indexes = lappend_int(*merged_indexes, merged_index);
+}
+
 /*
  * partitions_are_ordered
  *		Determine whether the partitions described by 'boundinfo' are ordered,
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index 469c686e3f..7027c1901f 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -597,8 +597,10 @@ typedef struct PartitionSchemeData *PartitionScheme;
  *		part_scheme - Partitioning scheme of the relation
  *		nparts - Number of partitions
  *		boundinfo - Partition bounds
+ *		partbounds_merged - true if partition bounds are merged ones
  *		partition_qual - Partition constraint if not the root
  *		part_rels - RelOptInfos for each partition
+ *		all_partrels - Relids set of all partition relids
  *		partexprs, nullable_partexprs - Partition key expressions
  *		partitioned_child_rels - RT indexes of unpruned partitions of
  *								 this relation that are partitioned tables
@@ -735,11 +737,16 @@ typedef struct RelOptInfo
 
 	/* used for partitioned relations: */
 	PartitionScheme part_scheme;	/* Partitioning scheme */
-	int			nparts;			/* Number of partitions */
+	int			nparts;			/* Number of partitions; -1 if not yet set;
+								 * in case of a join relation 0 means it's
+								 * considered unpartitioned */
 	struct PartitionBoundInfoData *boundinfo;	/* Partition bounds */
+	bool		partbounds_merged;	/* True if partition bounds were created
+									 * by partition_bounds_merge() */
 	List	   *partition_qual; /* Partition constraint, if not the root */
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of partitions,
 									 * stored in the same order as bounds */
+	Relids		all_partrels;	/* Relids set of all partition relids */
 	List	  **partexprs;		/* Non-nullable partition key expressions */
 	List	  **nullable_partexprs; /* Nullable partition key expressions */
 	List	   *partitioned_child_rels; /* List of RT indexes */
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
index be940b5192..dfc720720b 100644
--- a/src/include/partitioning/partbounds.h
+++ b/src/include/partitioning/partbounds.h
@@ -16,6 +16,7 @@
 #include "nodes/pg_list.h"
 #include "partitioning/partdefs.h"
 #include "utils/relcache.h"
+struct RelOptInfo;				/* avoid including pathnodes.h here */
 
 
 /*
@@ -87,6 +88,14 @@ extern bool partition_bounds_equal(int partnatts, int16 *parttyplen,
 								   PartitionBoundInfo b2);
 extern PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src,
 												PartitionKey key);
+extern PartitionBoundInfo partition_bounds_merge(int partnatts,
+												 FmgrInfo *partsupfunc,
+												 Oid *partcollation,
+												 struct RelOptInfo *outer_rel,
+												 struct RelOptInfo *inner_rel,
+												 JoinType jointype,
+												 List **outer_parts,
+												 List **inner_parts);
 extern bool partitions_are_ordered(PartitionBoundInfo boundinfo, int nparts);
 extern void check_new_partition_bound(char *relname, Relation parent,
 									  PartitionBoundSpec *spec);
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index b3fbe47bde..27588c883e 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -2055,30 +2055,2606 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
                ->  Seq Scan on prt1_n_p2 t1_2
 (10 rows)
 
--- partitionwise join can not be applied if only one of joining tables has
--- default partition
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
-ANALYZE prt2;
+--
+-- Test advanced partition-matching algorithm for partitioned join
+--
+-- Tests for range-partitioned tables
+CREATE TABLE prt1_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_adv_p1 PARTITION OF prt1_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_adv_p2 PARTITION OF prt1_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_adv_p3 PARTITION OF prt1_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_adv_a_idx ON prt1_adv (a);
+INSERT INTO prt1_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_adv;
+CREATE TABLE prt2_adv (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_adv_p1 PARTITION OF prt2_adv FOR VALUES FROM (100) TO (150);
+CREATE TABLE prt2_adv_p2 PARTITION OF prt2_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt2_adv_p3 PARTITION OF prt2_adv FOR VALUES FROM (350) TO (500);
+CREATE INDEX prt2_adv_b_idx ON prt2_adv (b);
+INSERT INTO prt2_adv_p1 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 149) i;
+INSERT INTO prt2_adv_p2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(200, 299) i;
+INSERT INTO prt2_adv_p3 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_adv;
+-- inner join
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
-                    QUERY PLAN                    
---------------------------------------------------
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
  Sort
    Sort Key: t1.a
-   ->  Hash Join
-         Hash Cond: (t2.b = t1.a)
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 100 | 0 | 0100
+ 125 | 0 | 0125
+ 200 | 0 | 0200
+ 225 | 0 | 0225
+ 250 | 0 | 0250
+ 275 | 0 | 0275
+ 350 | 0 | 0350
+ 375 | 0 | 0375
+(8 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 |     | 
+ 175 | 0175 |     | 
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 300 | 0300 |     | 
+ 325 | 0325 |     | 
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(12 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 150 | 0 | 0150
+ 175 | 0 | 0175
+ 300 | 0 | 0300
+ 325 | 0 | 0325
+(4 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                                QUERY PLAN                                
+--------------------------------------------------------------------------
+ Sort
+   Sort Key: prt1_adv.a, prt2_adv.b
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (prt1_adv_1.a = prt2_adv_1.b)
+               Filter: (((175) = prt1_adv_1.a) OR ((425) = prt2_adv_1.b))
+               ->  Seq Scan on prt1_adv_p1 prt1_adv_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 prt2_adv_1
+                           Filter: (a = 0)
+         ->  Hash Full Join
+               Hash Cond: (prt1_adv_2.a = prt2_adv_2.b)
+               Filter: (((175) = prt1_adv_2.a) OR ((425) = prt2_adv_2.b))
+               ->  Seq Scan on prt1_adv_p2 prt1_adv_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 prt2_adv_2
+                           Filter: (a = 0)
+         ->  Hash Full Join
+               Hash Cond: (prt2_adv_3.b = prt1_adv_3.a)
+               Filter: (((175) = prt1_adv_3.a) OR ((425) = prt2_adv_3.b))
+               ->  Seq Scan on prt2_adv_p3 prt2_adv_3
+                     Filter: (a = 0)
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 prt1_adv_3
+                           Filter: (b = 0)
+(27 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 175 | 0175 |     | 
+     |      | 425 | 0425
+(2 rows)
+
+-- Test cases where one side has an extra partition
+CREATE TABLE prt2_adv_extra PARTITION OF prt2_adv FOR VALUES FROM (500) TO (MAXVALUE);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(500, 599) i;
+ANALYZE prt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(8 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+         ->  Hash Semi Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+         ->  Hash Semi Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 100 | 0 | 0100
+ 125 | 0 | 0125
+ 200 | 0 | 0200
+ 225 | 0 | 0225
+ 250 | 0 | 0250
+ 275 | 0 | 0275
+ 350 | 0 | 0350
+ 375 | 0 | 0375
+(8 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_3.b = t1_3.a)
+               ->  Seq Scan on prt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 |     | 
+ 175 | 0175 |     | 
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+ 300 | 0300 |     | 
+ 325 | 0325 |     | 
+ 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375
+(12 rows)
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t2.a
+   ->  Hash Right Join
+         Hash Cond: (t2.a = t1.b)
          ->  Append
-               ->  Seq Scan on prt2_p1 t2_1
-               ->  Seq Scan on prt2_p2 t2_2
-               ->  Seq Scan on prt2_p3 t2_3
+               ->  Seq Scan on prt1_adv_p1 t2_1
+               ->  Seq Scan on prt1_adv_p2 t2_2
+               ->  Seq Scan on prt1_adv_p3 t2_3
          ->  Hash
                ->  Append
-                     ->  Seq Scan on prt1_p1 t1_1
+                     ->  Seq Scan on prt2_adv_p1 t1_1
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_adv_p2 t1_2
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_adv_p3 t1_3
+                           Filter: (a = 0)
+                     ->  Seq Scan on prt2_adv_extra t1_4
+                           Filter: (a = 0)
+(18 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: (t1_1.a = t2_1.b)
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+         ->  Hash Anti Join
+               Hash Cond: (t1_2.a = t2_2.b)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+         ->  Hash Anti Join
+               Hash Cond: (t1_3.a = t2_3.b)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+               ->  Hash
+                     ->  Seq Scan on prt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+ 150 | 0 | 0150
+ 175 | 0 | 0175
+ 300 | 0 | 0300
+ 325 | 0 | 0325
+(4 rows)
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt1_adv t2 WHERE t1.b = t2.a) AND t1.a = 0 ORDER BY t1.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.b
+   ->  Hash Anti Join
+         Hash Cond: (t1.b = t2.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t1_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p2 t1_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p3 t1_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_extra t1_4
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 t2_1
+                     ->  Seq Scan on prt1_adv_p2 t2_2
+                     ->  Seq Scan on prt1_adv_p3 t2_3
+(18 rows)
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                           QUERY PLAN                           
+----------------------------------------------------------------
+ Sort
+   Sort Key: prt1_adv.a, prt2_adv.b
+   ->  Hash Full Join
+         Hash Cond: (prt2_adv.b = prt1_adv.a)
+         Filter: (((175) = prt1_adv.a) OR ((425) = prt2_adv.b))
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 prt2_adv_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p2 prt2_adv_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p3 prt2_adv_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_extra prt2_adv_4
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 prt1_adv_1
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p2 t1_2
+                     ->  Seq Scan on prt1_adv_p2 prt1_adv_2
                            Filter: (b = 0)
-                     ->  Seq Scan on prt1_p3 t1_3
+                     ->  Seq Scan on prt1_adv_p3 prt1_adv_3
                            Filter: (b = 0)
+(22 rows)
+
+-- 3-way join where not every pair of relations can do partitioned join
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c, t3.a, t3.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) INNER JOIN prt1_adv t3 ON (t1.b = t3.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a, t3.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.b, t2.a
+   ->  Append
+         ->  Nested Loop Left Join
+               ->  Nested Loop
+                     ->  Seq Scan on prt2_adv_p1 t1_1
+                           Filter: (a = 0)
+                     ->  Index Scan using prt1_adv_p1_a_idx on prt1_adv_p1 t3_1
+                           Index Cond: (a = t1_1.b)
+               ->  Index Scan using prt1_adv_p1_a_idx on prt1_adv_p1 t2_1
+                     Index Cond: (a = t1_1.b)
+         ->  Hash Right Join
+               Hash Cond: (t2_2.a = t1_2.b)
+               ->  Seq Scan on prt1_adv_p2 t2_2
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_2.a = t1_2.b)
+                           ->  Seq Scan on prt1_adv_p2 t3_2
+                           ->  Hash
+                                 ->  Seq Scan on prt2_adv_p2 t1_2
+                                       Filter: (a = 0)
+         ->  Hash Right Join
+               Hash Cond: (t2_3.a = t1_3.b)
+               ->  Seq Scan on prt1_adv_p3 t2_3
+               ->  Hash
+                     ->  Hash Join
+                           Hash Cond: (t3_3.a = t1_3.b)
+                           ->  Seq Scan on prt1_adv_p3 t3_3
+                           ->  Hash
+                                 ->  Seq Scan on prt2_adv_p3 t1_3
+                                       Filter: (a = 0)
+(31 rows)
+
+SELECT t1.b, t1.c, t2.a, t2.c, t3.a, t3.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) INNER JOIN prt1_adv t3 ON (t1.b = t3.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a, t3.a;
+  b  |  c   |  a  |  c   |  a  |  c   
+-----+------+-----+------+-----+------
+ 100 | 0100 | 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275 | 275 | 0275
+ 350 | 0350 | 350 | 0350 | 350 | 0350
+ 375 | 0375 | 375 | 0375 | 375 | 0375
+(8 rows)
+
+DROP TABLE prt2_adv_extra;
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in such cases
+ALTER TABLE prt2_adv DETACH PARTITION prt2_adv_p3;
+-- Split prt2_adv_p3 into two partitions so that prt1_adv_p3 matches both
+CREATE TABLE prt2_adv_p3_1 PARTITION OF prt2_adv FOR VALUES FROM (350) TO (375);
+CREATE TABLE prt2_adv_p3_2 PARTITION OF prt2_adv FOR VALUES FROM (375) TO (500);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Seq Scan on prt2_adv_p3_1 t2_3
+               ->  Seq Scan on prt2_adv_p3_2 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Semi Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+                     ->  Seq Scan on prt2_adv_p3_1 t2_3
+                     ->  Seq Scan on prt2_adv_p3_2 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Hash Right Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Seq Scan on prt2_adv_p3_1 t2_3
+               ->  Seq Scan on prt2_adv_p3_2 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 t1_3
+                           Filter: (b = 0)
+(17 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Anti Join
+         Hash Cond: (t1.a = t2.b)
+         ->  Append
+               ->  Seq Scan on prt1_adv_p1 t1_1
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_adv_p2 t1_2
+                     Filter: (b = 0)
+               ->  Seq Scan on prt1_adv_p3 t1_3
+                     Filter: (b = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt2_adv_p1 t2_1
+                     ->  Seq Scan on prt2_adv_p2 t2_2
+                     ->  Seq Scan on prt2_adv_p3_1 t2_3
+                     ->  Seq Scan on prt2_adv_p3_2 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                           QUERY PLAN                           
+----------------------------------------------------------------
+ Sort
+   Sort Key: prt1_adv.a, prt2_adv.b
+   ->  Hash Full Join
+         Hash Cond: (prt2_adv.b = prt1_adv.a)
+         Filter: (((175) = prt1_adv.a) OR ((425) = prt2_adv.b))
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 prt2_adv_1
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p2 prt2_adv_2
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p3_1 prt2_adv_3
+                     Filter: (a = 0)
+               ->  Seq Scan on prt2_adv_p3_2 prt2_adv_4
+                     Filter: (a = 0)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p1 prt1_adv_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p2 prt1_adv_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 prt1_adv_3
+                           Filter: (b = 0)
+(22 rows)
+
+DROP TABLE prt2_adv_p3_1;
+DROP TABLE prt2_adv_p3_2;
+ANALYZE prt2_adv;
+-- Test default partitions
+ALTER TABLE prt1_adv DETACH PARTITION prt1_adv_p1;
+-- Change prt1_adv_p1 to the default partition
+ALTER TABLE prt1_adv ATTACH PARTITION prt1_adv_p1 DEFAULT;
+ALTER TABLE prt1_adv DETACH PARTITION prt1_adv_p3;
+ANALYZE prt1_adv;
+-- We can do partitioned join even if only one of relations has the default
+-- partition
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_2
+                           Filter: (b = 0)
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_1
+                           Filter: (b = 0)
+(15 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+(6 rows)
+
+-- Restore prt1_adv_p3
+ALTER TABLE prt1_adv ATTACH PARTITION prt1_adv_p3 FOR VALUES FROM (300) TO (400);
+ANALYZE prt1_adv;
+-- Restore prt2_adv_p3
+ALTER TABLE prt2_adv ATTACH PARTITION prt2_adv_p3 FOR VALUES FROM (350) TO (500);
+ANALYZE prt2_adv;
+-- Partitioned join can't be applied because the default partition of prt1_adv
+-- matches prt2_adv_p1 and prt2_adv_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Seq Scan on prt2_adv_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p2 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p1 t1_3
+                           Filter: (b = 0)
+(16 rows)
+
+ALTER TABLE prt2_adv DETACH PARTITION prt2_adv_p3;
+-- Change prt2_adv_p3 to the default partition
+ALTER TABLE prt2_adv ATTACH PARTITION prt2_adv_p3 DEFAULT;
+ANALYZE prt2_adv;
+-- Partitioned join can't be applied because the default partition of prt1_adv
+-- matches prt2_adv_p1 and prt2_adv_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: (t2.b = t1.a)
+         ->  Append
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Seq Scan on prt2_adv_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_adv_p2 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p3 t1_2
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_adv_p1 t1_3
+                           Filter: (b = 0)
+(16 rows)
+
+DROP TABLE prt1_adv_p3;
+ANALYZE prt1_adv;
+DROP TABLE prt2_adv_p3;
+ANALYZE prt2_adv;
+CREATE TABLE prt3_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt3_adv_p1 PARTITION OF prt3_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt3_adv_p2 PARTITION OF prt3_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt3_adv_a_idx ON prt3_adv (a);
+INSERT INTO prt3_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(200, 399) i;
+ANALYZE prt3_adv;
+-- 3-way join to test the default partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) LEFT JOIN prt3_adv t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b, t3.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: (t3_1.a = t1_1.a)
+               ->  Seq Scan on prt3_adv_p1 t3_1
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_2.b = t1_1.a)
+                           ->  Seq Scan on prt2_adv_p2 t2_2
+                           ->  Hash
+                                 ->  Seq Scan on prt1_adv_p2 t1_1
+                                       Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: (t3_2.a = t1_2.a)
+               ->  Seq Scan on prt3_adv_p2 t3_2
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: (t2_1.b = t1_2.a)
+                           ->  Seq Scan on prt2_adv_p1 t2_1
+                           ->  Hash
+                                 ->  Seq Scan on prt1_adv_p1 t1_2
+                                       Filter: (b = 0)
+(23 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) LEFT JOIN prt3_adv t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+  a  |  c   |  b  |  c   |  a  |  c   
+-----+------+-----+------+-----+------
+ 100 | 0100 | 100 | 0100 |     | 
+ 125 | 0125 | 125 | 0125 |     | 
+ 150 | 0150 |     |      |     | 
+ 175 | 0175 |     |      |     | 
+ 200 | 0200 | 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275 | 275 | 0275
+(8 rows)
+
+DROP TABLE prt1_adv;
+DROP TABLE prt2_adv;
+DROP TABLE prt3_adv;
+-- Test interaction of partitioned join with partition pruning
+CREATE TABLE prt1_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_adv_p1 PARTITION OF prt1_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_adv_p2 PARTITION OF prt1_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_adv_p3 PARTITION OF prt1_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_adv_a_idx ON prt1_adv (a);
+INSERT INTO prt1_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_adv;
+CREATE TABLE prt2_adv (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_adv_p1 PARTITION OF prt2_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt2_adv_p2 PARTITION OF prt2_adv FOR VALUES FROM (200) TO (400);
+CREATE INDEX prt2_adv_b_idx ON prt2_adv (b);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+                        QUERY PLAN                         
+-----------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: ((a < 300) AND (b = 0))
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: ((a < 300) AND (b = 0))
+(15 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 | 150 | 0150
+ 175 | 0175 | 175 | 0175
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+(8 rows)
+
+DROP TABLE prt1_adv_p3;
+CREATE TABLE prt1_adv_default PARTITION OF prt1_adv DEFAULT;
+ANALYZE prt1_adv;
+CREATE TABLE prt2_adv_default PARTITION OF prt2_adv DEFAULT;
+ANALYZE prt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+                                QUERY PLAN                                
+--------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on prt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p1 t1_1
+                           Filter: ((a >= 100) AND (a < 300) AND (b = 0))
+         ->  Hash Join
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on prt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on prt1_adv_p2 t1_2
+                           Filter: ((a >= 100) AND (a < 300) AND (b = 0))
+(15 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 100 | 0100 | 100 | 0100
+ 125 | 0125 | 125 | 0125
+ 150 | 0150 | 150 | 0150
+ 175 | 0175 | 175 | 0175
+ 200 | 0200 | 200 | 0200
+ 225 | 0225 | 225 | 0225
+ 250 | 0250 | 250 | 0250
+ 275 | 0275 | 275 | 0275
+(8 rows)
+
+DROP TABLE prt1_adv;
+DROP TABLE prt2_adv;
+-- Tests for list-partitioned tables
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0001', '0003');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt1_adv_p3 PARTITION OF plt1_adv FOR VALUES IN ('0008', '0009');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+ANALYZE plt1_adv;
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt2_adv_p3 PARTITION OF plt2_adv FOR VALUES IN ('0007', '0009');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+(6 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 1 | 1 | 0001
+ 8 | 8 | 0008
+(2 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p3 t2_3
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+   |      | 2 | 0002
+   |      | 7 | 0007
+(8 rows)
+
+-- Test cases where one side has an extra partition
+CREATE TABLE plt2_adv_extra PARTITION OF plt2_adv FOR VALUES IN ('0000');
+INSERT INTO plt2_adv_extra VALUES (0, 0, '0000');
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 |   | 
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 8 | 0008 |   | 
+ 9 | 0009 | 9 | 0009
+(6 rows)
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2_adv t1 LEFT JOIN plt1_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt1_adv_p1 t2_1
+               ->  Seq Scan on plt1_adv_p2 t2_2
+               ->  Seq Scan on plt1_adv_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_adv_extra t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_adv_p1 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_adv_p2 t1_3
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_adv_p3 t1_4
+                           Filter: (b < 10)
+(18 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t2_3
+(18 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 1 | 1 | 0001
+ 8 | 8 | 0008
+(2 rows)
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt2_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt1_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Anti Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_extra t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p1 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3 t1_4
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t2_1
+                     ->  Seq Scan on plt1_adv_p2 t2_2
+                     ->  Seq Scan on plt1_adv_p3 t2_3
+(18 rows)
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt2_adv_extra t2_1
+               ->  Seq Scan on plt2_adv_p1 t2_2
+               ->  Seq Scan on plt2_adv_p2 t2_3
+               ->  Seq Scan on plt2_adv_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+(15 rows)
+
+DROP TABLE plt2_adv_extra;
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in such cases
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2;
+-- Split plt2_adv_p2 into two partitions so that plt1_adv_p2 matches both
+CREATE TABLE plt2_adv_p2_1 PARTITION OF plt2_adv FOR VALUES IN ('0004');
+CREATE TABLE plt2_adv_p2_2 PARTITION OF plt2_adv FOR VALUES IN ('0006');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6);
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_1 t2_2
+               ->  Seq Scan on plt2_adv_p2_2 t2_3
+               ->  Seq Scan on plt2_adv_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(17 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Semi Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+                     ->  Seq Scan on plt2_adv_p2_1 t2_2
+                     ->  Seq Scan on plt2_adv_p2_2 t2_3
+                     ->  Seq Scan on plt2_adv_p3 t2_4
+(17 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_1 t2_2
+               ->  Seq Scan on plt2_adv_p2_2 t2_3
+               ->  Seq Scan on plt2_adv_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(17 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Anti Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         ->  Append
+               ->  Seq Scan on plt1_adv_p1 t1_1
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+                     ->  Seq Scan on plt2_adv_p2_1 t2_2
+                     ->  Seq Scan on plt2_adv_p2_2 t2_3
+                     ->  Seq Scan on plt2_adv_p3 t2_4
+(17 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_1 t2_2
+               ->  Seq Scan on plt2_adv_p2_2 t2_3
+               ->  Seq Scan on plt2_adv_p3 t2_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+(15 rows)
+
+DROP TABLE plt2_adv_p2_1;
+DROP TABLE plt2_adv_p2_2;
+-- Restore plt2_adv_p2
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2 FOR VALUES IN ('0004', '0006');
+-- Test NULL partitions
+ALTER TABLE plt1_adv DETACH PARTITION plt1_adv_p1;
+-- Change plt1_adv_p1 to the NULL partition
+CREATE TABLE plt1_adv_p1_null PARTITION OF plt1_adv FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p3;
+-- Change plt2_adv_p3 to the NULL partition
+CREATE TABLE plt2_adv_p3_null PARTITION OF plt2_adv FOR VALUES IN (NULL, '0007', '0009');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (7, 9);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1_null t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Semi Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1_null t1_1
+                     Filter: (b < 10)
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Semi Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+(19 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a | b |  c   
+---+---+------
+ 3 | 3 | 0003
+ 4 | 4 | 0004
+ 6 | 6 | 0006
+ 9 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1_null t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a  |  c   | a |  c   
+----+------+---+------
+ -1 |      |   | 
+  1 | 0001 |   | 
+  3 | 0003 | 3 | 0003
+  4 | 0004 | 4 | 0004
+  6 | 0006 | 6 | 0006
+  8 | 0008 |   | 
+  9 | 0009 | 9 | 0009
+(7 rows)
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Anti Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on plt1_adv_p1_null t1_1
+                     Filter: (b < 10)
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Nested Loop Anti Join
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+(19 rows)
+
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+ a  | b  |  c   
+----+----+------
+ -1 | -1 | 
+  1 |  1 | 0001
+  8 |  8 | 0008
+(3 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p1_null t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt2_adv_p3_null t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+ -1 |      |    | 
+  1 | 0001 |    | 
+  3 | 0003 |  3 | 0003
+  4 | 0004 |  4 | 0004
+  6 | 0006 |  6 | 0006
+  8 | 0008 |    | 
+  9 | 0009 |  9 | 0009
+    |      | -1 | 
+    |      |  2 | 0002
+    |      |  7 | 0007
+(10 rows)
+
+DROP TABLE plt1_adv_p1_null;
+-- Restore plt1_adv_p1
+ALTER TABLE plt1_adv ATTACH PARTITION plt1_adv_p1 FOR VALUES IN ('0001', '0003');
+-- Add to plt1_adv the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt1_adv_extra PARTITION OF plt1_adv FOR VALUES IN (NULL);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+DROP TABLE plt2_adv_p3_null;
+-- Restore plt2_adv_p3
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p3 FOR VALUES IN ('0007', '0009');
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Seq Scan on plt2_adv_p3 t2_3
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_extra t1_4
+                           Filter: (b < 10)
+(18 rows)
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Hash Full Join
+         Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c))
+         Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10))
+         ->  Append
+               ->  Seq Scan on plt1_adv_p1 t1_1
+               ->  Seq Scan on plt1_adv_p2 t1_2
+               ->  Seq Scan on plt1_adv_p3 t1_3
+               ->  Seq Scan on plt1_adv_extra t1_4
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+                     ->  Seq Scan on plt2_adv_p2 t2_2
+                     ->  Seq Scan on plt2_adv_p3 t2_3
+(15 rows)
+
+-- Add to plt2_adv the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt2_adv_extra PARTITION OF plt2_adv FOR VALUES IN (NULL);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+(21 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+ 9 | 0009 | 9 | 0009
+(4 rows)
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                           Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+               ->  Seq Scan on plt2_adv_p3 t2_3
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_3
+                           Filter: (b < 10)
+         ->  Nested Loop Left Join
+               Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+               ->  Seq Scan on plt1_adv_extra t1_4
+                     Filter: (b < 10)
+               ->  Seq Scan on plt2_adv_extra t2_4
+(26 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a  |  c   | a |  c   
+----+------+---+------
+ -1 |      |   | 
+  1 | 0001 |   | 
+  3 | 0003 | 3 | 0003
+  4 | 0004 | 4 | 0004
+  6 | 0006 | 6 | 0006
+  8 | 0008 |   | 
+  9 | 0009 | 9 | 0009
+(7 rows)
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+                                    QUERY PLAN                                     
+-----------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c))
+               Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p1 t1_1
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p1 t2_1
+         ->  Hash Full Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p2 t1_2
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p2 t2_2
+         ->  Hash Full Join
+               Hash Cond: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c))
+               Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_p3 t1_3
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_p3 t2_3
+         ->  Hash Full Join
+               Hash Cond: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+               Filter: ((COALESCE(t1_4.b, 0) < 10) AND (COALESCE(t2_4.b, 0) < 10))
+               ->  Seq Scan on plt1_adv_extra t1_4
+               ->  Hash
+                     ->  Seq Scan on plt2_adv_extra t2_4
+(27 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+ a  |  c   | a  |  c   
+----+------+----+------
+ -1 |      |    | 
+  1 | 0001 |    | 
+  3 | 0003 |  3 | 0003
+  4 | 0004 |  4 | 0004
+  6 | 0006 |  6 | 0006
+  8 | 0008 |    | 
+  9 | 0009 |  9 | 0009
+    |      | -1 | 
+    |      |  2 | 0002
+    |      |  7 | 0007
+(10 rows)
+
+-- 3-way join to test the NULL partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt1_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t3_1.a = t1_1.a) AND (t3_1.c = t1_1.c))
+               ->  Seq Scan on plt1_adv_p1 t3_1
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+                           ->  Seq Scan on plt2_adv_p1 t2_1
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p1 t1_1
+                                       Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t3_2.a = t1_2.a) AND (t3_2.c = t1_2.c))
+               ->  Seq Scan on plt1_adv_p2 t3_2
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+                           ->  Seq Scan on plt2_adv_p2 t2_2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p2 t1_2
+                                       Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t3_3.a = t1_3.a) AND (t3_3.c = t1_3.c))
+               ->  Seq Scan on plt1_adv_p3 t3_3
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c))
+                           ->  Seq Scan on plt2_adv_p3 t2_3
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p3 t1_3
+                                       Filter: (b < 10)
+         ->  Nested Loop Left Join
+               Join Filter: ((t1_4.a = t3_4.a) AND (t1_4.c = t3_4.c))
+               ->  Nested Loop Left Join
+                     Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+                     ->  Seq Scan on plt1_adv_extra t1_4
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt2_adv_extra t2_4
+               ->  Seq Scan on plt1_adv_extra t3_4
+(41 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt1_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a  |  c   | a |  c   | a |  c   
+----+------+---+------+---+------
+ -1 |      |   |      |   | 
+  1 | 0001 |   |      | 1 | 0001
+  3 | 0003 | 3 | 0003 | 3 | 0003
+  4 | 0004 | 4 | 0004 | 4 | 0004
+  6 | 0006 | 6 | 0006 | 6 | 0006
+  8 | 0008 |   |      | 8 | 0008
+  9 | 0009 | 9 | 0009 | 9 | 0009
+(7 rows)
+
+DROP TABLE plt1_adv_extra;
+DROP TABLE plt2_adv_extra;
+-- Test default partitions
+ALTER TABLE plt1_adv DETACH PARTITION plt1_adv_p1;
+-- Change plt1_adv_p1 to the default partition
+ALTER TABLE plt1_adv ATTACH PARTITION plt1_adv_p1 DEFAULT;
+DROP TABLE plt1_adv_p3;
+ANALYZE plt1_adv;
+DROP TABLE plt2_adv_p3;
+ANALYZE plt2_adv;
+-- We can do partitioned join even if only one of relations has the default
+-- partition
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_1
+                           Filter: (b < 10)
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(3 rows)
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2;
+-- Change plt2_adv_p2 to contain '0005' in addition to '0004' and '0006' as
+-- the key values
+CREATE TABLE plt2_adv_p2_ext PARTITION OF plt2_adv FOR VALUES IN ('0004', '0005', '0006');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 5, 6);
+ANALYZE plt2_adv;
+-- Partitioned join can't be applied because the default partition of plt1_adv
+-- matches plt2_adv_p1 and plt2_adv_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_ext t2_2
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p2 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p1 t1_2
+                           Filter: (b < 10)
+(13 rows)
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2_ext;
+-- Change plt2_adv_p2_ext to the default partition
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2_ext DEFAULT;
+ANALYZE plt2_adv;
+-- Partitioned join can't be applied because the default partition of plt1_adv
+-- matches plt2_adv_p1 and plt2_adv_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Append
+               ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Seq Scan on plt2_adv_p2_ext t2_2
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on plt1_adv_p2 t1_1
+                           Filter: (b < 10)
+                     ->  Seq Scan on plt1_adv_p1 t1_2
+                           Filter: (b < 10)
+(13 rows)
+
+DROP TABLE plt2_adv_p2_ext;
+-- Restore plt2_adv_p2
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_adv;
+CREATE TABLE plt3_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_adv_p1 PARTITION OF plt3_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt3_adv_p2 PARTITION OF plt3_adv FOR VALUES IN ('0007', '0009');
+INSERT INTO plt3_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6, 7, 9);
+ANALYZE plt3_adv;
+-- 3-way join to test the default partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t3_1.a = t1_1.a) AND (t3_1.c = t1_1.c))
+               ->  Seq Scan on plt3_adv_p1 t3_1
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+                           ->  Seq Scan on plt2_adv_p2 t2_2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p2 t1_1
+                                       Filter: (b < 10)
+         ->  Hash Right Join
+               Hash Cond: ((t3_2.a = t1_2.a) AND (t3_2.c = t1_2.c))
+               ->  Seq Scan on plt3_adv_p2 t3_2
+               ->  Hash
+                     ->  Hash Right Join
+                           Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+                           ->  Seq Scan on plt2_adv_p1 t2_1
+                           ->  Hash
+                                 ->  Seq Scan on plt1_adv_p1 t1_2
+                                       Filter: (b < 10)
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   | a |  c   
+---+------+---+------+---+------
+ 1 | 0001 |   |      |   | 
+ 3 | 0003 | 3 | 0003 |   | 
+ 4 | 0004 | 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006 | 6 | 0006
+(4 rows)
+
+-- Test cases where one side has the default partition while the other side
+-- has the NULL partition
+DROP TABLE plt2_adv_p1;
+-- Add the NULL partition to plt2_adv
+CREATE TABLE plt2_adv_p1_null PARTITION OF plt2_adv FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p1_null t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p1 t1_2
+                           Filter: (b < 10)
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p2 t1_1
+                           Filter: (b < 10)
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 1 | 0001 | 1 | 0001
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(4 rows)
+
+DROP TABLE plt2_adv_p1_null;
+-- Add the NULL partition that contains only NULL values as the key values
+CREATE TABLE plt2_adv_p1_null PARTITION OF plt2_adv FOR VALUES IN (NULL);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+                      QUERY PLAN                      
+------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Seq Scan on plt2_adv_p2 t2
+         ->  Hash
+               ->  Seq Scan on plt1_adv_p2 t1
+                     Filter: (b < 10)
+(8 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 4 | 0004 | 4 | 0004
+ 6 | 0006 | 6 | 0006
+(2 rows)
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+DROP TABLE plt3_adv;
+-- Test interaction of partitioned join with partition pruning
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0001');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0002');
+CREATE TABLE plt1_adv_p3 PARTITION OF plt1_adv FOR VALUES IN ('0003');
+CREATE TABLE plt1_adv_p4 PARTITION OF plt1_adv FOR VALUES IN (NULL, '0004', '0005');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN (NULL);
+CREATE TABLE plt2_adv_p3 PARTITION OF plt2_adv FOR VALUES IN ('0003');
+CREATE TABLE plt2_adv_p4 PARTITION OF plt2_adv FOR VALUES IN ('0004', '0005');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+                                       QUERY PLAN                                        
+-----------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p3 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_1
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p4 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p4 t1_2
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 5 | 0005 | 5 | 0005
+(3 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Seq Scan on plt2_adv_p4 t2
+         ->  Hash
+               ->  Seq Scan on plt1_adv_p4 t1
+                     Filter: ((c IS NULL) AND (b < 10))
+(8 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+ a  | c | a | c 
+----+---+---+---
+ -1 |   |   | 
+(1 row)
+
+CREATE TABLE plt1_adv_default PARTITION OF plt1_adv DEFAULT;
+ANALYZE plt1_adv;
+CREATE TABLE plt2_adv_default PARTITION OF plt2_adv DEFAULT;
+ANALYZE plt2_adv;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+                                       QUERY PLAN                                        
+-----------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on plt2_adv_p3 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p3 t1_1
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on plt2_adv_p4 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt1_adv_p4 t1_2
+                           Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[])))
+(15 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+ a |  c   | a |  c   
+---+------+---+------
+ 3 | 0003 | 3 | 0003
+ 4 | 0004 | 4 | 0004
+ 5 | 0005 | 5 | 0005
+(3 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Sort
+   Sort Key: t1.a
+   ->  Hash Right Join
+         Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+         ->  Seq Scan on plt2_adv_p4 t2
+         ->  Hash
+               ->  Seq Scan on plt1_adv_p4 t1
+                     Filter: ((c IS NULL) AND (b < 10))
+(8 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+ a  | c | a | c 
+----+---+---+---
+ -1 |   |   | 
+(1 row)
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+-- Test the process_outer_partition() code path
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1_adv;
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0002');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2_adv;
+CREATE TABLE plt3_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_adv_p1 PARTITION OF plt3_adv FOR VALUES IN ('0001');
+CREATE TABLE plt3_adv_p2 PARTITION OF plt3_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3_adv;
+-- This tests that when merging partitions from plt1_adv and plt2_adv in
+-- merge_list_bounds(), process_outer_partition() returns an already-assigned
+-- merged partition when re-called with plt1_adv_p1 for the second list value
+-- '0001' of that partitin
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.c = t2.c)) FULL JOIN plt3_adv t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t1.a, t2.a, t3.a
+   ->  Append
+         ->  Hash Full Join
+               Hash Cond: (t1_1.c = t3_1.c)
+               Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_1.c = t2_1.c)
+                     ->  Seq Scan on plt1_adv_p1 t1_1
+                     ->  Hash
+                           ->  Seq Scan on plt2_adv_p1 t2_1
+               ->  Hash
+                     ->  Seq Scan on plt3_adv_p1 t3_1
+         ->  Hash Full Join
+               Hash Cond: (t1_2.c = t3_2.c)
+               Filter: (((COALESCE(t1_2.a, 0) % 5) <> 3) AND ((COALESCE(t1_2.a, 0) % 5) <> 4))
+               ->  Hash Left Join
+                     Hash Cond: (t1_2.c = t2_2.c)
+                     ->  Seq Scan on plt1_adv_p2 t1_2
+                     ->  Hash
+                           ->  Seq Scan on plt2_adv_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on plt3_adv_p2 t3_2
+(23 rows)
+
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.c = t2.c)) FULL JOIN plt3_adv t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+ a  |  c   | a  |  c   | a  |  c   
+----+------+----+------+----+------
+  0 | 0000 |    |      |    | 
+  5 | 0000 |    |      |    | 
+ 10 | 0000 |    |      |    | 
+ 15 | 0000 |    |      |    | 
+ 20 | 0000 |    |      |    | 
+  1 | 0001 |    |      |  1 | 0001
+  1 | 0001 |    |      |  6 | 0001
+  1 | 0001 |    |      | 11 | 0001
+  1 | 0001 |    |      | 16 | 0001
+  1 | 0001 |    |      | 21 | 0001
+  6 | 0001 |    |      |  1 | 0001
+  6 | 0001 |    |      |  6 | 0001
+  6 | 0001 |    |      | 11 | 0001
+  6 | 0001 |    |      | 16 | 0001
+  6 | 0001 |    |      | 21 | 0001
+ 11 | 0001 |    |      |  1 | 0001
+ 11 | 0001 |    |      |  6 | 0001
+ 11 | 0001 |    |      | 11 | 0001
+ 11 | 0001 |    |      | 16 | 0001
+ 11 | 0001 |    |      | 21 | 0001
+ 16 | 0001 |    |      |  1 | 0001
+ 16 | 0001 |    |      |  6 | 0001
+ 16 | 0001 |    |      | 11 | 0001
+ 16 | 0001 |    |      | 16 | 0001
+ 16 | 0001 |    |      | 21 | 0001
+ 21 | 0001 |    |      |  1 | 0001
+ 21 | 0001 |    |      |  6 | 0001
+ 21 | 0001 |    |      | 11 | 0001
+ 21 | 0001 |    |      | 16 | 0001
+ 21 | 0001 |    |      | 21 | 0001
+  2 | 0002 |  2 | 0002 |    | 
+  2 | 0002 |  7 | 0002 |    | 
+  2 | 0002 | 12 | 0002 |    | 
+  2 | 0002 | 17 | 0002 |    | 
+  2 | 0002 | 22 | 0002 |    | 
+  7 | 0002 |  2 | 0002 |    | 
+  7 | 0002 |  7 | 0002 |    | 
+  7 | 0002 | 12 | 0002 |    | 
+  7 | 0002 | 17 | 0002 |    | 
+  7 | 0002 | 22 | 0002 |    | 
+ 12 | 0002 |  2 | 0002 |    | 
+ 12 | 0002 |  7 | 0002 |    | 
+ 12 | 0002 | 12 | 0002 |    | 
+ 12 | 0002 | 17 | 0002 |    | 
+ 12 | 0002 | 22 | 0002 |    | 
+ 17 | 0002 |  2 | 0002 |    | 
+ 17 | 0002 |  7 | 0002 |    | 
+ 17 | 0002 | 12 | 0002 |    | 
+ 17 | 0002 | 17 | 0002 |    | 
+ 17 | 0002 | 22 | 0002 |    | 
+ 22 | 0002 |  2 | 0002 |    | 
+ 22 | 0002 |  7 | 0002 |    | 
+ 22 | 0002 | 12 | 0002 |    | 
+ 22 | 0002 | 17 | 0002 |    | 
+ 22 | 0002 | 22 | 0002 |    | 
+(55 rows)
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+DROP TABLE plt3_adv;
+-- Tests for multi-level partitioned tables
+CREATE TABLE alpha (a double precision, b int, c text) PARTITION BY RANGE (a);
+CREATE TABLE alpha_neg PARTITION OF alpha FOR VALUES FROM ('-Infinity') TO (0) PARTITION BY RANGE (b);
+CREATE TABLE alpha_pos PARTITION OF alpha FOR VALUES FROM (0) TO (10.0) PARTITION BY LIST (c);
+CREATE TABLE alpha_neg_p1 PARTITION OF alpha_neg FOR VALUES FROM (100) TO (200);
+CREATE TABLE alpha_neg_p2 PARTITION OF alpha_neg FOR VALUES FROM (200) TO (300);
+CREATE TABLE alpha_neg_p3 PARTITION OF alpha_neg FOR VALUES FROM (300) TO (400);
+CREATE TABLE alpha_pos_p1 PARTITION OF alpha_pos FOR VALUES IN ('0001', '0003');
+CREATE TABLE alpha_pos_p2 PARTITION OF alpha_pos FOR VALUES IN ('0004', '0006');
+CREATE TABLE alpha_pos_p3 PARTITION OF alpha_pos FOR VALUES IN ('0008', '0009');
+INSERT INTO alpha_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+INSERT INTO alpha_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+ANALYZE alpha;
+CREATE TABLE beta (a double precision, b int, c text) PARTITION BY RANGE (a);
+CREATE TABLE beta_neg PARTITION OF beta FOR VALUES FROM (-10.0) TO (0) PARTITION BY RANGE (b);
+CREATE TABLE beta_pos PARTITION OF beta FOR VALUES FROM (0) TO ('Infinity') PARTITION BY LIST (c);
+CREATE TABLE beta_neg_p1 PARTITION OF beta_neg FOR VALUES FROM (100) TO (150);
+CREATE TABLE beta_neg_p2 PARTITION OF beta_neg FOR VALUES FROM (200) TO (300);
+CREATE TABLE beta_neg_p3 PARTITION OF beta_neg FOR VALUES FROM (350) TO (500);
+CREATE TABLE beta_pos_p1 PARTITION OF beta_pos FOR VALUES IN ('0002', '0003');
+CREATE TABLE beta_pos_p2 PARTITION OF beta_pos FOR VALUES IN ('0004', '0006');
+CREATE TABLE beta_pos_p3 PARTITION OF beta_pos FOR VALUES IN ('0007', '0009');
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+ANALYZE beta;
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.b
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.b = t2_1.b))
+               ->  Seq Scan on alpha_neg_p1 t1_1
+                     Filter: ((b >= 125) AND (b < 225))
+               ->  Hash
+                     ->  Seq Scan on beta_neg_p1 t2_1
+         ->  Hash Join
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.b = t1_2.b))
+               ->  Seq Scan on beta_neg_p2 t2_2
+               ->  Hash
+                     ->  Seq Scan on alpha_neg_p2 t1_2
+                           Filter: ((b >= 125) AND (b < 225))
+         ->  Hash Join
+               Hash Cond: ((t2_4.a = t1_4.a) AND (t2_4.b = t1_4.b))
+               ->  Append
+                     ->  Seq Scan on beta_pos_p1 t2_4
+                     ->  Seq Scan on beta_pos_p2 t2_5
+                     ->  Seq Scan on beta_pos_p3 t2_6
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on alpha_pos_p1 t1_4
+                                 Filter: ((b >= 125) AND (b < 225))
+                           ->  Seq Scan on alpha_pos_p2 t1_5
+                                 Filter: ((b >= 125) AND (b < 225))
+                           ->  Seq Scan on alpha_pos_p3 t1_6
+                                 Filter: ((b >= 125) AND (b < 225))
+(29 rows)
+
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b;
+ a  |  b  |  c   | a  |  b  |  c   
+----+-----+------+----+-----+------
+ -1 | 126 | 0006 | -1 | 126 | 0006
+ -1 | 129 | 0009 | -1 | 129 | 0009
+ -1 | 133 | 0003 | -1 | 133 | 0003
+ -1 | 134 | 0004 | -1 | 134 | 0004
+ -1 | 136 | 0006 | -1 | 136 | 0006
+ -1 | 139 | 0009 | -1 | 139 | 0009
+ -1 | 143 | 0003 | -1 | 143 | 0003
+ -1 | 144 | 0004 | -1 | 144 | 0004
+ -1 | 146 | 0006 | -1 | 146 | 0006
+ -1 | 149 | 0009 | -1 | 149 | 0009
+ -1 | 203 | 0003 | -1 | 203 | 0003
+ -1 | 204 | 0004 | -1 | 204 | 0004
+ -1 | 206 | 0006 | -1 | 206 | 0006
+ -1 | 209 | 0009 | -1 | 209 | 0009
+ -1 | 213 | 0003 | -1 | 213 | 0003
+ -1 | 214 | 0004 | -1 | 214 | 0004
+ -1 | 216 | 0006 | -1 | 216 | 0006
+ -1 | 219 | 0009 | -1 | 219 | 0009
+ -1 | 223 | 0003 | -1 | 223 | 0003
+ -1 | 224 | 0004 | -1 | 224 | 0004
+  1 | 126 | 0006 |  1 | 126 | 0006
+  1 | 129 | 0009 |  1 | 129 | 0009
+  1 | 133 | 0003 |  1 | 133 | 0003
+  1 | 134 | 0004 |  1 | 134 | 0004
+  1 | 136 | 0006 |  1 | 136 | 0006
+  1 | 139 | 0009 |  1 | 139 | 0009
+  1 | 143 | 0003 |  1 | 143 | 0003
+  1 | 144 | 0004 |  1 | 144 | 0004
+  1 | 146 | 0006 |  1 | 146 | 0006
+  1 | 149 | 0009 |  1 | 149 | 0009
+  1 | 203 | 0003 |  1 | 203 | 0003
+  1 | 204 | 0004 |  1 | 204 | 0004
+  1 | 206 | 0006 |  1 | 206 | 0006
+  1 | 209 | 0009 |  1 | 209 | 0009
+  1 | 213 | 0003 |  1 | 213 | 0003
+  1 | 214 | 0004 |  1 | 214 | 0004
+  1 | 216 | 0006 |  1 | 216 | 0006
+  1 | 219 | 0009 |  1 | 219 | 0009
+  1 | 223 | 0003 |  1 | 223 | 0003
+  1 | 224 | 0004 |  1 | 224 | 0004
+(40 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b, t2.b;
+                                                              QUERY PLAN                                                              
+--------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.b, t2.b
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c))
+               ->  Append
+                     ->  Seq Scan on alpha_neg_p1 t1_2
+                           Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+                     ->  Seq Scan on alpha_neg_p2 t1_3
+                           Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on beta_neg_p1 t2_2
+                                 Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+                           ->  Seq Scan on beta_neg_p2 t2_3
+                                 Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Nested Loop
+               Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c))
+               ->  Seq Scan on alpha_pos_p2 t1_4
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Seq Scan on beta_pos_p2 t2_4
+                     Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Nested Loop
+               Join Filter: ((t1_5.a = t2_5.a) AND (t1_5.c = t2_5.c))
+               ->  Seq Scan on alpha_pos_p3 t1_5
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Seq Scan on beta_pos_p3 t2_5
+                     Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+(28 rows)
+
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b, t2.b;
+ a  |  b  |  c   | a  |  b  |  c   
+----+-----+------+----+-----+------
+ -1 | 104 | 0004 | -1 | 104 | 0004
+ -1 | 104 | 0004 | -1 | 204 | 0004
+ -1 | 109 | 0009 | -1 | 109 | 0009
+ -1 | 109 | 0009 | -1 | 209 | 0009
+ -1 | 204 | 0004 | -1 | 104 | 0004
+ -1 | 204 | 0004 | -1 | 204 | 0004
+ -1 | 209 | 0009 | -1 | 109 | 0009
+ -1 | 209 | 0009 | -1 | 209 | 0009
+  1 | 104 | 0004 |  1 | 104 | 0004
+  1 | 104 | 0004 |  1 | 204 | 0004
+  1 | 109 | 0009 |  1 | 109 | 0009
+  1 | 109 | 0009 |  1 | 209 | 0009
+  1 | 204 | 0004 |  1 | 104 | 0004
+  1 | 204 | 0004 |  1 | 204 | 0004
+  1 | 209 | 0009 |  1 | 109 | 0009
+  1 | 209 | 0009 |  1 | 209 | 0009
 (16 rows)
 
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;
+                                                           QUERY PLAN                                                           
+--------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.b
+   ->  Append
+         ->  Hash Join
+               Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c))
+               ->  Seq Scan on alpha_neg_p1 t1_1
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Hash
+                     ->  Seq Scan on beta_neg_p1 t2_1
+                           Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Hash Join
+               Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c))
+               ->  Seq Scan on alpha_neg_p2 t1_2
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Hash
+                     ->  Seq Scan on beta_neg_p2 t2_2
+                           Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Nested Loop
+               Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.b = t2_3.b) AND (t1_3.c = t2_3.c))
+               ->  Seq Scan on alpha_pos_p2 t1_3
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Seq Scan on beta_pos_p2 t2_3
+                     Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+         ->  Nested Loop
+               Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.b = t2_4.b) AND (t1_4.c = t2_4.c))
+               ->  Seq Scan on alpha_pos_p3 t1_4
+                     Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))))
+               ->  Seq Scan on beta_pos_p3 t2_4
+                     Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))
+(29 rows)
+
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;
+ a  |  b  |  c   | a  |  b  |  c   
+----+-----+------+----+-----+------
+ -1 | 104 | 0004 | -1 | 104 | 0004
+ -1 | 109 | 0009 | -1 | 109 | 0009
+ -1 | 204 | 0004 | -1 | 204 | 0004
+ -1 | 209 | 0009 | -1 | 209 | 0009
+  1 | 104 | 0004 |  1 | 104 | 0004
+  1 | 109 | 0009 |  1 | 109 | 0009
+  1 | 204 | 0004 |  1 | 204 | 0004
+  1 | 209 | 0009 |  1 | 209 | 0009
+(8 rows)
+
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 575ba7b8d4..df30f851e8 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -445,11 +445,638 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c);
 
--- partitionwise join can not be applied if only one of joining tables has
--- default partition
-ALTER TABLE prt2 DETACH PARTITION prt2_p3;
-ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
-ANALYZE prt2;
 
+--
+-- Test advanced partition-matching algorithm for partitioned join
+--
+
+-- Tests for range-partitioned tables
+CREATE TABLE prt1_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_adv_p1 PARTITION OF prt1_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_adv_p2 PARTITION OF prt1_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_adv_p3 PARTITION OF prt1_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_adv_a_idx ON prt1_adv (a);
+INSERT INTO prt1_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_adv;
+
+CREATE TABLE prt2_adv (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_adv_p1 PARTITION OF prt2_adv FOR VALUES FROM (100) TO (150);
+CREATE TABLE prt2_adv_p2 PARTITION OF prt2_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt2_adv_p3 PARTITION OF prt2_adv FOR VALUES FROM (350) TO (500);
+CREATE INDEX prt2_adv_b_idx ON prt2_adv (b);
+INSERT INTO prt2_adv_p1 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 149) i;
+INSERT INTO prt2_adv_p2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(200, 299) i;
+INSERT INTO prt2_adv_p3 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_adv;
+
+-- inner join
 EXPLAIN (COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+-- Test cases where one side has an extra partition
+CREATE TABLE prt2_adv_extra PARTITION OF prt2_adv FOR VALUES FROM (500) TO (MAXVALUE);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(500, 599) i;
+ANALYZE prt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt2_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt1_adv t2 WHERE t1.b = t2.a) AND t1.a = 0 ORDER BY t1.b;
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+-- 3-way join where not every pair of relations can do partitioned join
+EXPLAIN (COSTS OFF)
+SELECT t1.b, t1.c, t2.a, t2.c, t3.a, t3.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) INNER JOIN prt1_adv t3 ON (t1.b = t3.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a, t3.a;
+SELECT t1.b, t1.c, t2.a, t2.c, t3.a, t3.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) INNER JOIN prt1_adv t3 ON (t1.b = t3.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a, t3.a;
+
+DROP TABLE prt2_adv_extra;
+
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in such cases
+ALTER TABLE prt2_adv DETACH PARTITION prt2_adv_p3;
+-- Split prt2_adv_p3 into two partitions so that prt1_adv_p3 matches both
+CREATE TABLE prt2_adv_p3_1 PARTITION OF prt2_adv FOR VALUES FROM (350) TO (375);
+CREATE TABLE prt2_adv_p3_2 PARTITION OF prt2_adv FOR VALUES FROM (375) TO (500);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i;
+ANALYZE prt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+DROP TABLE prt2_adv_p3_1;
+DROP TABLE prt2_adv_p3_2;
+ANALYZE prt2_adv;
+
+-- Test default partitions
+ALTER TABLE prt1_adv DETACH PARTITION prt1_adv_p1;
+-- Change prt1_adv_p1 to the default partition
+ALTER TABLE prt1_adv ATTACH PARTITION prt1_adv_p1 DEFAULT;
+ALTER TABLE prt1_adv DETACH PARTITION prt1_adv_p3;
+ANALYZE prt1_adv;
+
+-- We can do partitioned join even if only one of relations has the default
+-- partition
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- Restore prt1_adv_p3
+ALTER TABLE prt1_adv ATTACH PARTITION prt1_adv_p3 FOR VALUES FROM (300) TO (400);
+ANALYZE prt1_adv;
+
+-- Restore prt2_adv_p3
+ALTER TABLE prt2_adv ATTACH PARTITION prt2_adv_p3 FOR VALUES FROM (350) TO (500);
+ANALYZE prt2_adv;
+
+-- Partitioned join can't be applied because the default partition of prt1_adv
+-- matches prt2_adv_p1 and prt2_adv_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+ALTER TABLE prt2_adv DETACH PARTITION prt2_adv_p3;
+-- Change prt2_adv_p3 to the default partition
+ALTER TABLE prt2_adv ATTACH PARTITION prt2_adv_p3 DEFAULT;
+ANALYZE prt2_adv;
+
+-- Partitioned join can't be applied because the default partition of prt1_adv
+-- matches prt2_adv_p1 and prt2_adv_p3
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt1_adv_p3;
+ANALYZE prt1_adv;
+
+DROP TABLE prt2_adv_p3;
+ANALYZE prt2_adv;
+
+CREATE TABLE prt3_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt3_adv_p1 PARTITION OF prt3_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt3_adv_p2 PARTITION OF prt3_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt3_adv_a_idx ON prt3_adv (a);
+INSERT INTO prt3_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(200, 399) i;
+ANALYZE prt3_adv;
+
+-- 3-way join to test the default partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) LEFT JOIN prt3_adv t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) LEFT JOIN prt3_adv t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a;
+
+DROP TABLE prt1_adv;
+DROP TABLE prt2_adv;
+DROP TABLE prt3_adv;
+
+-- Test interaction of partitioned join with partition pruning
+CREATE TABLE prt1_adv (a int, b int, c varchar) PARTITION BY RANGE (a);
+CREATE TABLE prt1_adv_p1 PARTITION OF prt1_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt1_adv_p2 PARTITION OF prt1_adv FOR VALUES FROM (200) TO (300);
+CREATE TABLE prt1_adv_p3 PARTITION OF prt1_adv FOR VALUES FROM (300) TO (400);
+CREATE INDEX prt1_adv_a_idx ON prt1_adv (a);
+INSERT INTO prt1_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt1_adv;
+
+CREATE TABLE prt2_adv (a int, b int, c varchar) PARTITION BY RANGE (b);
+CREATE TABLE prt2_adv_p1 PARTITION OF prt2_adv FOR VALUES FROM (100) TO (200);
+CREATE TABLE prt2_adv_p2 PARTITION OF prt2_adv FOR VALUES FROM (200) TO (400);
+CREATE INDEX prt2_adv_b_idx ON prt2_adv (b);
+INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 399) i;
+ANALYZE prt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt1_adv_p3;
+CREATE TABLE prt1_adv_default PARTITION OF prt1_adv DEFAULT;
+ANALYZE prt1_adv;
+
+CREATE TABLE prt2_adv_default PARTITION OF prt2_adv DEFAULT;
+ANALYZE prt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+DROP TABLE prt1_adv;
+DROP TABLE prt2_adv;
+
+
+-- Tests for list-partitioned tables
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0001', '0003');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt1_adv_p3 PARTITION OF plt1_adv FOR VALUES IN ('0008', '0009');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+ANALYZE plt1_adv;
+
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0002', '0003');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt2_adv_p3 PARTITION OF plt2_adv FOR VALUES IN ('0007', '0009');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+-- Test cases where one side has an extra partition
+CREATE TABLE plt2_adv_extra PARTITION OF plt2_adv FOR VALUES IN ('0000');
+INSERT INTO plt2_adv_extra VALUES (0, 0, '0000');
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt2_adv t1 LEFT JOIN plt1_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- anti join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt2_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt1_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt2_adv_extra;
+
+-- Test cases where a partition on one side matches multiple partitions on
+-- the other side; we currently can't do partitioned join in such cases
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2;
+-- Split plt2_adv_p2 into two partitions so that plt1_adv_p2 matches both
+CREATE TABLE plt2_adv_p2_1 PARTITION OF plt2_adv FOR VALUES IN ('0004');
+CREATE TABLE plt2_adv_p2_2 PARTITION OF plt2_adv FOR VALUES IN ('0006');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6);
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt2_adv_p2_1;
+DROP TABLE plt2_adv_p2_2;
+-- Restore plt2_adv_p2
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2 FOR VALUES IN ('0004', '0006');
+
+-- Test NULL partitions
+ALTER TABLE plt1_adv DETACH PARTITION plt1_adv_p1;
+-- Change plt1_adv_p1 to the NULL partition
+CREATE TABLE plt1_adv_p1_null PARTITION OF plt1_adv FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p3;
+-- Change plt2_adv_p3 to the NULL partition
+CREATE TABLE plt2_adv_p3_null PARTITION OF plt2_adv FOR VALUES IN (NULL, '0007', '0009');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (7, 9);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- semi join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- anti join
+EXPLAIN (COSTS OFF)
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+DROP TABLE plt1_adv_p1_null;
+-- Restore plt1_adv_p1
+ALTER TABLE plt1_adv ATTACH PARTITION plt1_adv_p1 FOR VALUES IN ('0001', '0003');
+-- Add to plt1_adv the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt1_adv_extra PARTITION OF plt1_adv FOR VALUES IN (NULL);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+
+DROP TABLE plt2_adv_p3_null;
+-- Restore plt2_adv_p3
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p3 FOR VALUES IN ('0007', '0009');
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- left join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- full join; currently we can't do partitioned join if there are no matched
+-- partitions on the nullable side
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+-- Add to plt2_adv the extra NULL partition containing only NULL values as the
+-- key values
+CREATE TABLE plt2_adv_extra PARTITION OF plt2_adv FOR VALUES IN (NULL);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+-- inner join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- left join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- full join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a;
+
+-- 3-way join to test the NULL partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt1_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt1_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt1_adv_extra;
+DROP TABLE plt2_adv_extra;
+
+-- Test default partitions
+ALTER TABLE plt1_adv DETACH PARTITION plt1_adv_p1;
+-- Change plt1_adv_p1 to the default partition
+ALTER TABLE plt1_adv ATTACH PARTITION plt1_adv_p1 DEFAULT;
+DROP TABLE plt1_adv_p3;
+ANALYZE plt1_adv;
+
+DROP TABLE plt2_adv_p3;
+ANALYZE plt2_adv;
+
+-- We can do partitioned join even if only one of relations has the default
+-- partition
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2;
+-- Change plt2_adv_p2 to contain '0005' in addition to '0004' and '0006' as
+-- the key values
+CREATE TABLE plt2_adv_p2_ext PARTITION OF plt2_adv FOR VALUES IN ('0004', '0005', '0006');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 5, 6);
+ANALYZE plt2_adv;
+
+-- Partitioned join can't be applied because the default partition of plt1_adv
+-- matches plt2_adv_p1 and plt2_adv_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2_ext;
+-- Change plt2_adv_p2_ext to the default partition
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2_ext DEFAULT;
+ANALYZE plt2_adv;
+
+-- Partitioned join can't be applied because the default partition of plt1_adv
+-- matches plt2_adv_p1 and plt2_adv_p2_ext
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt2_adv_p2_ext;
+-- Restore plt2_adv_p2
+ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2 FOR VALUES IN ('0004', '0006');
+ANALYZE plt2_adv;
+
+CREATE TABLE plt3_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_adv_p1 PARTITION OF plt3_adv FOR VALUES IN ('0004', '0006');
+CREATE TABLE plt3_adv_p2 PARTITION OF plt3_adv FOR VALUES IN ('0007', '0009');
+INSERT INTO plt3_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6, 7, 9);
+ANALYZE plt3_adv;
+
+-- 3-way join to test the default partition of a join relation
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+-- Test cases where one side has the default partition while the other side
+-- has the NULL partition
+DROP TABLE plt2_adv_p1;
+-- Add the NULL partition to plt2_adv
+CREATE TABLE plt2_adv_p1_null PARTITION OF plt2_adv FOR VALUES IN (NULL, '0001', '0003');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt2_adv_p1_null;
+-- Add the NULL partition that contains only NULL values as the key values
+CREATE TABLE plt2_adv_p1_null PARTITION OF plt2_adv FOR VALUES IN (NULL);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+DROP TABLE plt3_adv;
+
+-- Test interaction of partitioned join with partition pruning
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0001');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0002');
+CREATE TABLE plt1_adv_p3 PARTITION OF plt1_adv FOR VALUES IN ('0003');
+CREATE TABLE plt1_adv_p4 PARTITION OF plt1_adv FOR VALUES IN (NULL, '0004', '0005');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt1_adv VALUES (-1, -1, NULL);
+ANALYZE plt1_adv;
+
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0001', '0002');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN (NULL);
+CREATE TABLE plt2_adv_p3 PARTITION OF plt2_adv FOR VALUES IN ('0003');
+CREATE TABLE plt2_adv_p4 PARTITION OF plt2_adv FOR VALUES IN ('0004', '0005');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5);
+INSERT INTO plt2_adv VALUES (-1, -1, NULL);
+ANALYZE plt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+
+CREATE TABLE plt1_adv_default PARTITION OF plt1_adv DEFAULT;
+ANALYZE plt1_adv;
+
+CREATE TABLE plt2_adv_default PARTITION OF plt2_adv DEFAULT;
+ANALYZE plt2_adv;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a;
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+
+-- Test the process_outer_partition() code path
+CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0000', '0001', '0002');
+CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt1_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i;
+ANALYZE plt1_adv;
+
+CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0002');
+CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt2_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4);
+ANALYZE plt2_adv;
+
+CREATE TABLE plt3_adv (a int, b int, c text) PARTITION BY LIST (c);
+CREATE TABLE plt3_adv_p1 PARTITION OF plt3_adv FOR VALUES IN ('0001');
+CREATE TABLE plt3_adv_p2 PARTITION OF plt3_adv FOR VALUES IN ('0003', '0004');
+INSERT INTO plt3_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4);
+ANALYZE plt3_adv;
+
+-- This tests that when merging partitions from plt1_adv and plt2_adv in
+-- merge_list_bounds(), process_outer_partition() returns an already-assigned
+-- merged partition when re-called with plt1_adv_p1 for the second list value
+-- '0001' of that partitin
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.c = t2.c)) FULL JOIN plt3_adv t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.c = t2.c)) FULL JOIN plt3_adv t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a;
+
+DROP TABLE plt1_adv;
+DROP TABLE plt2_adv;
+DROP TABLE plt3_adv;
+
+
+-- Tests for multi-level partitioned tables
+CREATE TABLE alpha (a double precision, b int, c text) PARTITION BY RANGE (a);
+CREATE TABLE alpha_neg PARTITION OF alpha FOR VALUES FROM ('-Infinity') TO (0) PARTITION BY RANGE (b);
+CREATE TABLE alpha_pos PARTITION OF alpha FOR VALUES FROM (0) TO (10.0) PARTITION BY LIST (c);
+CREATE TABLE alpha_neg_p1 PARTITION OF alpha_neg FOR VALUES FROM (100) TO (200);
+CREATE TABLE alpha_neg_p2 PARTITION OF alpha_neg FOR VALUES FROM (200) TO (300);
+CREATE TABLE alpha_neg_p3 PARTITION OF alpha_neg FOR VALUES FROM (300) TO (400);
+CREATE TABLE alpha_pos_p1 PARTITION OF alpha_pos FOR VALUES IN ('0001', '0003');
+CREATE TABLE alpha_pos_p2 PARTITION OF alpha_pos FOR VALUES IN ('0004', '0006');
+CREATE TABLE alpha_pos_p3 PARTITION OF alpha_pos FOR VALUES IN ('0008', '0009');
+INSERT INTO alpha_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+INSERT INTO alpha_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9);
+ANALYZE alpha;
+
+CREATE TABLE beta (a double precision, b int, c text) PARTITION BY RANGE (a);
+CREATE TABLE beta_neg PARTITION OF beta FOR VALUES FROM (-10.0) TO (0) PARTITION BY RANGE (b);
+CREATE TABLE beta_pos PARTITION OF beta FOR VALUES FROM (0) TO ('Infinity') PARTITION BY LIST (c);
+CREATE TABLE beta_neg_p1 PARTITION OF beta_neg FOR VALUES FROM (100) TO (150);
+CREATE TABLE beta_neg_p2 PARTITION OF beta_neg FOR VALUES FROM (200) TO (300);
+CREATE TABLE beta_neg_p3 PARTITION OF beta_neg FOR VALUES FROM (350) TO (500);
+CREATE TABLE beta_pos_p1 PARTITION OF beta_pos FOR VALUES IN ('0002', '0003');
+CREATE TABLE beta_pos_p2 PARTITION OF beta_pos FOR VALUES IN ('0004', '0006');
+CREATE TABLE beta_pos_p3 PARTITION OF beta_pos FOR VALUES IN ('0007', '0009');
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+INSERT INTO beta_pos SELECT  1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9);
+ANALYZE beta;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b;
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b, t2.b;
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b, t2.b;
+
+EXPLAIN (COSTS OFF)
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;
+SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;
-- 
2.14.3 (Apple Git-98)

#132Ashutosh Bapat
ashutosh.bapat@2ndquadrant.com
In reply to: Etsuro Fujita (#130)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Fri, 3 Apr 2020 at 20:45, Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

Hi,

On Thu, Apr 2, 2020 at 2:12 AM Ashutosh Bapat
<ashutosh.bapat@2ndquadrant.com> wrote:

On Thu, 26 Mar 2020 at 00:35, Tomas Vondra <tomas.vondra@2ndquadrant.com>

wrote:

I've started reviewing the patch a couple days ago. I haven't done any
extensive testing, but I do have a bunch of initial comments that I can
share now.

1) I wonder if this needs to update src/backend/optimizer/README, which
does have a section about partitionwise joins. It seems formulated in a
way that that probably covers even this more advanced algorithm, but
maybe it should mention handling of default partitions etc.?

Done. Please check the wording. It might need some word smithy.

You heavily changed the existing documentation about PWJ, but I don't
think we really need to do so. Also, IMO I think the description
about default partitions you added is needed in README. I think it
would be better to put such a description in source files. How about
something like the attached, instead? I wrote part of this based on
the commit message in the original versions of the patch you posted.

I corrected some grammar, typos. Broke longer sentences into smaller ones
so that its easy to read and understand. As is the concept is hard to
understand with all its limitations. Thanks for the example. Retained it.

You seem to have removed few comments that explained the algorithm in
detail from build_joinrel_partition_info(). It would have been good to have
those there. But I am ok not having them either.

But it will be good to have following addition I suggested in my patches to
make sure nparts is set to 0 for an unpartitioned relation as per the
comment on nparts in RelOptInfo.
@@ -1653,6 +1663,8 @@ build_joinrel_partition_info(RelOptInfo *joinrel,
RelOptInfo *outer_rel,
jointype, restrictlist))
{
Assert(!IS_PARTITIONED_REL(joinrel));
+ /* Join is not partitioned. */
+ joinrel->nparts = 0;
return;
}

There certainly needs to be some description of the algorithm somewhere,
either in a README or before a suitable function. It doesn't have to be
particularly detailed, a rough outline of the matching would be enough,
so that readers don't have to rebuild the knowledge from pieces
scattered around various comments.

The algorithm for list and range partitioned tables is slightly

different. So, I have added separate prologue to each list_merge_bounds()
and range_merge_bounds(). Please check if that serves the purpose.

Too detailed to me. In this:

+ * If there are multiple partitions from one side matching a given
partition on
+ * the other side, the algorithm bails out since we do not have machinary
for
+ * joining one partition with mulitple partitions. It might happen that
any of
+ * the list items of a partition from the outer relation do not appear in
the
+ * inner relation and there is no default partition in the inner
relation. Such
+ * a partition from the outer side will have no matching partition on the
inner
+ * side. The algorithm will bail out in such a case since we do not have a
+ * mechanism to perform a join with a non-existing relation.

I don't think the last comment is correct; that would apply to the old
versions of this function IIRC, but not to the latest version. How
about something much simpler like the attached, instead?

I know that algorithm pretty well by now, so it suffices for me to say we
use something similar to merge join, but may be for someone without that
background a detailed explanation is useful. But this looks fine at the
moment.

Done. Actually this wasn't updated when partition pruning was

introduced, which could cause a partitionwise join to be not used even when
those conditions were met. Similarly when a query involved whole row
reference. It's hard to explain all the conditions under which
partitionwise join technique will be used. But I have tried to keep it easy
to understand.

IMO I think your words "there is exactly one pair of matching
partitions." is a bit misleading, because that sounds like that PWJ
doesn't allow multiply-segmented join. How about s/exact
matching/one-to-one matching/ in the existing documentation, instead?

Good catch. That was really misleading. Looks good to me.

3) I think the for nparts comment is somewhat misleading:

int nparts; /* number of partitions; 0 = not partitioned;
* -1 = not yet set */

which says that nparts=0 means not partitioned. But then there are
conditions like this:

/* Nothing to do, if the join relation is not partitioned. */
if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
return;

which (ignoring the obsolete comment) seems to say we can have nparts==0
even for partitioned tables, no?

Yeah, I think I was a bit hasty. I fixed this.

For a non-join relation, nparts = 0 and nparts = -1 both have the same
meaning. Although we never set nparts = 0 for a non-join relation?
Otherwise, the comment looks good now.

--
Best Wishes,
Ashutosh

#133Ashutosh Bapat
ashutosh.bapat@2ndquadrant.com
In reply to: Etsuro Fujita (#131)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Here are some changes suggested on top of v34 as per my previous mail.
These are mostly comment changes.

On Mon, 6 Apr 2020 at 13:59, Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

On Sat, Apr 4, 2020 at 12:15 AM Etsuro Fujita <etsuro.fujita@gmail.com>
wrote:

Attached is the original patch (0001) and one patch (0002) with
changes including those by Tomas and Ashutosh.

I merged the patches into one and rebased it against HEAD. Attached
is a new version, in which I added the commit message as well. Does
that make sense? If there are no objections, I’ll commit the patch.

I have no objections.

--
Best Wishes,
Ashutosh

Attachments:

changes_over_v34.patchtext/x-patch; charset=US-ASCII; name=changes_over_v34.patchDownload
diff --git a/src/backend/optimizer/README b/src/backend/optimizer/README
index 13633841f3..d285e63b6b 100644
--- a/src/backend/optimizer/README
+++ b/src/backend/optimizer/README
@@ -1106,32 +1106,33 @@ into joins between their partitions is called partitionwise join. We will use
 term "partitioned relation" for either a partitioned table or a join between
 compatibly partitioned tables.
 
-The technique is extended to some cases where the joining tables don't have
-exactly the same partition bounds, by an advanced partition-matching
-algorithm: it checks to see if there is a relationship where each partition of
-one joining table matches/overlaps at most one partition of the other, and
-vice versa; in which case the join between the joining tables can be broken
-down into joins between the matching partitions (ie, the join relation is
-considerd partitioned), so the algorithm produces the pairs of the matching
-partitions, plus the partition bounds for the join relation, to allow
-partitionwise join for the join.  The algorithm is implemented in
-partition_bounds_merge().  For an N-way join relation considered partitioned
-by this extension, not every pair of joining relations can use partitionwise
+Even if the joining relations do not have exactly same partition bounds,
+partitionwise join can be still applied by by using an advanced
+partition-matching algorithm. For both the joining relations, the algorithm
+checks whether for every given partition of given joining relation there
+existsa matching/overlapping partition in the other joining relation. In such a
+case the join between the joining relations can be broken down into joins
+between their matching/overlapping partitions. The join relation can then be
+considered partitioned. The algorithm produces the pairs of the
+matching/overlapping partitions, plus the partition bounds for the join
+relation, to allow partitionwise join for computing join.  The algorithm is
+implemented in partition_bounds_merge().  For an N-way join relation considered
+partitioned this way, not every pair of joining relations can use partitionwise
 join.  For example:
 
 	(A leftjoin B on (Pab)) innerjoin C on (Pac)
 
-where A, B, and C are partitioned tables, and A has an extra partition
-compared to B and C.  When considering partitionwise join for the join {A B},
-the extra partition of A doesn't have a matching partition on the nullable
-side, which is the case that the current implementation of partitionwise join
-can't handle.  So {A B} is not considered partitioned, and thus the pair of
-{A B} and C considered for the 3-way join can't use partitionwise join.  On
-the other hand, the pair of {A C} and B can use partitionwise join, because
-{A C} is considered partitioned, eliminating the extra partition (see identity
-1 on outer join reordering).  The partitionwise joinability of the N-way join
-relation is determined based on the first pair of joining relations that are
-both partitioned and can use partitionwise join.
+where A, B, and C are partitioned tables. A has an extra partition compared to
+B and C.  When considering partitionwise join for the join {A B}, the extra
+partition of A doesn't have a matching partition on the nullable side, which is
+the case that the current implementation of partitionwise join can't handle.
+So {A B} is not considered partitioned and the pair of {A B} and C considered
+for the 3-way join can not use partitionwise join.  On the other hand, the pair
+of {A C} and B can use partitionwise join, because {A C} is considered
+partitioned by eliminating the extra partition (see identity 1 on outer join
+reordering).  Whether an N-way join can use partitionwise join is determined
+based on the first pair of joining relations that are both partitioned and can
+use partitionwise join.
 
 The partitioning properties of a partitioned relation are stored in its
 RelOptInfo.  The information about data types of partition keys are stored in
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index e4c74d6c03..8ff798fd17 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -70,8 +70,8 @@ typedef struct PartitionRangeBound
 } PartitionRangeBound;
 
 /*
- * Mapping from partitions of a partitioned relation to partitions of a join
- * relation supposed to be partitioned (a.k.a merged partitions)
+ * Mapping from partitions of a partitioned joining relation to partitions of a
+ * join relation being computed (a.k.a merged partitions)
  */
 typedef struct PartitionMap
 {
#134Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: Etsuro Fujita (#131)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Mon, Apr 06, 2020 at 05:28:52PM +0900, Etsuro Fujita wrote:

On Sat, Apr 4, 2020 at 12:15 AM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

Attached is the original patch (0001) and one patch (0002) with
changes including those by Tomas and Ashutosh.

I merged the patches into one and rebased it against HEAD. Attached
is a new version, in which I added the commit message as well. Does
that make sense? If there are no objections, I’ll commit the patch.

+1

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#135Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Tomas Vondra (#134)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Hi Tomas,

On Wed, Apr 8, 2020 at 12:15 AM Tomas Vondra
<tomas.vondra@2ndquadrant.com> wrote:

On Mon, Apr 06, 2020 at 05:28:52PM +0900, Etsuro Fujita wrote:

On Sat, Apr 4, 2020 at 12:15 AM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

Attached is the original patch (0001) and one patch (0002) with
changes including those by Tomas and Ashutosh.

I merged the patches into one and rebased it against HEAD. Attached
is a new version, in which I added the commit message as well. Does
that make sense? If there are no objections, I’ll commit the patch.

+1

Great! It's midnight in Japan now, so I'll push the patch early morning.

Best regards,
Etsuro Fujita

#136Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Etsuro Fujita (#135)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Wed, Apr 8, 2020 at 2:24 AM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

On Wed, Apr 8, 2020 at 12:15 AM Tomas Vondra
<tomas.vondra@2ndquadrant.com> wrote:

On Mon, Apr 06, 2020 at 05:28:52PM +0900, Etsuro Fujita wrote:

On Sat, Apr 4, 2020 at 12:15 AM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

Attached is the original patch (0001) and one patch (0002) with
changes including those by Tomas and Ashutosh.

I merged the patches into one and rebased it against HEAD. Attached
is a new version, in which I added the commit message as well. Does
that make sense? If there are no objections, I’ll commit the patch.

+1

Great! It's midnight in Japan now, so I'll push the patch early morning.

Pushed after modifying some comments further, based on the suggestions
of Ashutosh.

Best regards,
Etsuro Fujita

#137Ashutosh Bapat
ashutosh.bapat@2ndquadrant.com
In reply to: Etsuro Fujita (#136)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Thanks a lot Fujita-san. Thanks Dmitry, Rajkumar, Amul, Mark, Robert,
Antonin, Amit, Justin,Thomas and Tomas for all your help and review.

On Wed, 8 Apr 2020 at 07:07, Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

On Wed, Apr 8, 2020 at 2:24 AM Etsuro Fujita <etsuro.fujita@gmail.com>
wrote:

On Wed, Apr 8, 2020 at 12:15 AM Tomas Vondra
<tomas.vondra@2ndquadrant.com> wrote:

On Mon, Apr 06, 2020 at 05:28:52PM +0900, Etsuro Fujita wrote:

On Sat, Apr 4, 2020 at 12:15 AM Etsuro Fujita <

etsuro.fujita@gmail.com> wrote:

Attached is the original patch (0001) and one patch (0002) with
changes including those by Tomas and Ashutosh.

I merged the patches into one and rebased it against HEAD. Attached
is a new version, in which I added the commit message as well. Does
that make sense? If there are no objections, I’ll commit the patch.

+1

Great! It's midnight in Japan now, so I'll push the patch early morning.

Pushed after modifying some comments further, based on the suggestions
of Ashutosh.

Best regards,
Etsuro Fujita

--
Best Wishes,
Ashutosh

#138Kuntal Ghosh
kuntalghosh.2007@gmail.com
In reply to: Etsuro Fujita (#136)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Hi,

On Wed, Apr 8, 2020 at 7:07 AM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

Pushed after modifying some comments further, based on the suggestions
of Ashutosh.

I'm getting the following warning during compilation.

partbounds.c: In function ‘partition_bounds_merge’:
partbounds.c:1024:21: warning: unused variable ‘inner_binfo’ [-Wunused-variable]
PartitionBoundInfo inner_binfo = inner_rel->boundinfo;
^
For fixing the same, we can declare inner_binfo as
PG_USED_FOR_ASSERTS_ONLY as it is not used for any other purpose.

--
Thanks & Regards,
Kuntal Ghosh
EnterpriseDB: http://www.enterprisedb.com

#139Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Kuntal Ghosh (#138)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Hi Kuntal,

On Wed, Apr 8, 2020 at 4:30 PM Kuntal Ghosh <kuntalghosh.2007@gmail.com> wrote:

I'm getting the following warning during compilation.

partbounds.c: In function ‘partition_bounds_merge’:
partbounds.c:1024:21: warning: unused variable ‘inner_binfo’ [-Wunused-variable]
PartitionBoundInfo inner_binfo = inner_rel->boundinfo;
^
For fixing the same, we can declare inner_binfo as
PG_USED_FOR_ASSERTS_ONLY as it is not used for any other purpose.

I'd propose to remove an assertion causing this (and the
outer_binfo/inner_binfo variables) from partition_bounds_merge(),
rather than doing so, because the assertion is redundant, as we have
the same assertion in merge_list_bounds() and merge_range_bounds().
Please find attached a patch.

Best regards,
Etsuro Fujita

Attachments:

fix-compiler-warning.patchapplication/octet-stream; name=fix-compiler-warning.patchDownload
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 7607501fe7..8d781c75a8 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -1020,9 +1020,6 @@ partition_bounds_merge(int partnatts,
 					   JoinType jointype,
 					   List **outer_parts, List **inner_parts)
 {
-	PartitionBoundInfo outer_binfo = outer_rel->boundinfo;
-	PartitionBoundInfo inner_binfo = inner_rel->boundinfo;
-
 	/*
 	 * Currently, this function is called only from try_partitionwise_join(),
 	 * so the join type should be INNER, LEFT, FULL, SEMI, or ANTI.
@@ -1031,11 +1028,8 @@ partition_bounds_merge(int partnatts,
 		   jointype == JOIN_FULL || jointype == JOIN_SEMI ||
 		   jointype == JOIN_ANTI);
 
-	/* The partitioning strategies should be the same. */
-	Assert(outer_binfo->strategy == inner_binfo->strategy);
-
 	*outer_parts = *inner_parts = NIL;
-	switch (outer_binfo->strategy)
+	switch (outer_rel->boundinfo->strategy)
 	{
 		case PARTITION_STRATEGY_HASH:
 
@@ -1076,7 +1070,7 @@ partition_bounds_merge(int partnatts,
 
 		default:
 			elog(ERROR, "unexpected partition strategy: %d",
-				 (int) outer_binfo->strategy);
+				 (int) outer_rel->boundinfo->strategy);
 			return NULL;				/* keep compiler quiet */
 	}
 }
#140Ashutosh Bapat
ashutosh.bapat@2ndquadrant.com
In reply to: Kuntal Ghosh (#138)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Thanks Kuntal for the report. Let me know if this patch works for you.

On Wed, 8 Apr 2020 at 13:00, Kuntal Ghosh <kuntalghosh.2007@gmail.com>
wrote:

Hi,

On Wed, Apr 8, 2020 at 7:07 AM Etsuro Fujita <etsuro.fujita@gmail.com>
wrote:

Pushed after modifying some comments further, based on the suggestions
of Ashutosh.

I'm getting the following warning during compilation.

partbounds.c: In function ‘partition_bounds_merge’:
partbounds.c:1024:21: warning: unused variable ‘inner_binfo’
[-Wunused-variable]
PartitionBoundInfo inner_binfo = inner_rel->boundinfo;
^
For fixing the same, we can declare inner_binfo as
PG_USED_FOR_ASSERTS_ONLY as it is not used for any other purpose.

--
Thanks & Regards,
Kuntal Ghosh
EnterpriseDB: http://www.enterprisedb.com

--
Best Wishes,
Ashutosh

Attachments:

fix_warning_in_adv_pwj.patchtext/x-patch; charset=US-ASCII; name=fix_warning_in_adv_pwj.patchDownload
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 7607501fe7..4681441dcc 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -1021,7 +1021,6 @@ partition_bounds_merge(int partnatts,
 					   List **outer_parts, List **inner_parts)
 {
 	PartitionBoundInfo outer_binfo = outer_rel->boundinfo;
-	PartitionBoundInfo inner_binfo = inner_rel->boundinfo;
 
 	/*
 	 * Currently, this function is called only from try_partitionwise_join(),
@@ -1032,7 +1031,7 @@ partition_bounds_merge(int partnatts,
 		   jointype == JOIN_ANTI);
 
 	/* The partitioning strategies should be the same. */
-	Assert(outer_binfo->strategy == inner_binfo->strategy);
+	Assert(outer_binfo->strategy == inner_rel->boundinfo->strategy);
 
 	*outer_parts = *inner_parts = NIL;
 	switch (outer_binfo->strategy)
#141Ashutosh Bapat
ashutosh.bapat@2ndquadrant.com
In reply to: Etsuro Fujita (#139)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Wed, 8 Apr 2020 at 15:42, Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

Hi Kuntal,

On Wed, Apr 8, 2020 at 4:30 PM Kuntal Ghosh <kuntalghosh.2007@gmail.com>
wrote:

I'm getting the following warning during compilation.

partbounds.c: In function ‘partition_bounds_merge’:
partbounds.c:1024:21: warning: unused variable ‘inner_binfo’

[-Wunused-variable]

PartitionBoundInfo inner_binfo = inner_rel->boundinfo;
^
For fixing the same, we can declare inner_binfo as
PG_USED_FOR_ASSERTS_ONLY as it is not used for any other purpose.

I'd propose to remove an assertion causing this (and the
outer_binfo/inner_binfo variables) from partition_bounds_merge(),
rather than doing so, because the assertion is redundant, as we have
the same assertion in merge_list_bounds() and merge_range_bounds().
Please find attached a patch.

Oh, I didn't see this mail before sending my other mail.

I think it's better to have the assertion in all the three places and also
in merge_hash_bounds() whenever that comes along. The assertion in
merge_*_bounds() will be good to in case those functions are called from
places other than partition_bounds_merge(). The assertion in
partition_bounds_merge() will make sure that when the individual
merge_*_bounds() functions are called based on one of the bounds both of
the bounds have same strategy.
--
Best Wishes,
Ashutosh

#142Kuntal Ghosh
kuntalghosh.2007@gmail.com
In reply to: Ashutosh Bapat (#141)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Hello Ashutosh, Fujita,

On Wed, Apr 8, 2020 at 3:49 PM Ashutosh Bapat
<ashutosh.bapat@2ndquadrant.com> wrote:

On Wed, 8 Apr 2020 at 15:42, Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

On Wed, Apr 8, 2020 at 4:30 PM Kuntal Ghosh <kuntalghosh.2007@gmail.com> wrote:

I'm getting the following warning during compilation.

partbounds.c: In function ‘partition_bounds_merge’:
partbounds.c:1024:21: warning: unused variable ‘inner_binfo’ [-Wunused-variable]
PartitionBoundInfo inner_binfo = inner_rel->boundinfo;
^
For fixing the same, we can declare inner_binfo as
PG_USED_FOR_ASSERTS_ONLY as it is not used for any other purpose.

I'd propose to remove an assertion causing this (and the
outer_binfo/inner_binfo variables) from partition_bounds_merge(),
rather than doing so, because the assertion is redundant, as we have
the same assertion in merge_list_bounds() and merge_range_bounds().
Please find attached a patch.

I think it's better to have the assertion in all the three places and also in merge_hash_bounds() whenever that comes along. The assertion in merge_*_bounds() will be good to in case those functions are called from places other than partition_bounds_merge(). The assertion in partition_bounds_merge() will make sure that when the individual merge_*_bounds() functions are called based on one of the bounds both of the bounds have same strategy.

Both of your patches fix the problem. I don't have much exposure in
this area to comment on whether we should keep/remove the assertion
from the code. But, here is my opinion:

The code structure looks like following:
Assert(condition A);
if (Condition B)
merge_*_bounds(....);

Inside merge_*_bounds(), you have both the above assert and the if
condition as another assert:
Assert(condition A and Condition B);

And, merge_*_bounds() are called from only one place. So, something is
redundant here and I'm inclined towards removal of the assert
condition. Another thing I noticed:

/* The partitioning strategies should be the same. */
Assert(outer_binfo->strategy == inner_binfo->strategy);

The comment just reads the assertion aloud which looks unnecessary.

--
Thanks & Regards,
Kuntal Ghosh
EnterpriseDB: http://www.enterprisedb.com

#143Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Kuntal Ghosh (#142)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Hi,

On Thu, Apr 9, 2020 at 12:06 AM Kuntal Ghosh <kuntalghosh.2007@gmail.com> wrote:

Both of your patches fix the problem. I don't have much exposure in
this area to comment on whether we should keep/remove the assertion
from the code. But, here is my opinion:

The code structure looks like following:
Assert(condition A);
if (Condition B)
merge_*_bounds(....);

Inside merge_*_bounds(), you have both the above assert and the if
condition as another assert:
Assert(condition A and Condition B);

And, merge_*_bounds() are called from only one place. So, something is
redundant here and I'm inclined towards removal of the assert
condition. Another thing I noticed:

/* The partitioning strategies should be the same. */
Assert(outer_binfo->strategy == inner_binfo->strategy);

The comment just reads the assertion aloud which looks unnecessary.

Yeah, partition_bounds_merge() is currently called only from
try_partitionwise_join(), which guarantees that the strategies are the
same. The assertion cost would be cheap, but not zero, so I still
think it would be better to remove the assertion from
partition_bounds_merge().

Best regards,
Etsuro Fujita

#144Tom Lane
tgl@sss.pgh.pa.us
In reply to: Etsuro Fujita (#143)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Etsuro Fujita <etsuro.fujita@gmail.com> writes:

Yeah, partition_bounds_merge() is currently called only from
try_partitionwise_join(), which guarantees that the strategies are the
same. The assertion cost would be cheap, but not zero, so I still
think it would be better to remove the assertion from
partition_bounds_merge().

FWIW, our general policy is that assertion costs should be ignored
in any performance considerations. If you're concerned about
performance you should be running a non-assert build, so it doesn't
matter. (And certainly, there are lots of assertions in the backend
that cost FAR more than this one.) The thing to evaluate an assertion
on is how likely it is that it would catch a foreseeable sort of coding
error in some future patch. Maybe this one carries its weight on that
score or maybe it doesn't, but that's how to think about it.

If there's only one caller and there's not likely to ever be more,
then I tend to agree that you don't need the assertion.

regards, tom lane

#145Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Tom Lane (#144)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Thu, Apr 9, 2020 at 2:36 PM Tom Lane <tgl@sss.pgh.pa.us> wrote:

Etsuro Fujita <etsuro.fujita@gmail.com> writes:

Yeah, partition_bounds_merge() is currently called only from
try_partitionwise_join(), which guarantees that the strategies are the
same.

If there's only one caller and there's not likely to ever be more,
then I tend to agree that you don't need the assertion.

It seems unlikely that partition_bounds_merge() will be called from
more places in the foreseeable future, so I'd still vote for removing
the assertion.

Best regards,
Etsuro Fujita

#146Ashutosh Bapat
ashutosh.bapat.oss@gmail.com
In reply to: Etsuro Fujita (#145)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Thu, Apr 9, 2020 at 12:03 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

On Thu, Apr 9, 2020 at 2:36 PM Tom Lane <tgl@sss.pgh.pa.us> wrote:

Etsuro Fujita <etsuro.fujita@gmail.com> writes:

Yeah, partition_bounds_merge() is currently called only from
try_partitionwise_join(), which guarantees that the strategies are the
same.

If there's only one caller and there's not likely to ever be more,
then I tend to agree that you don't need the assertion.

It seems unlikely that partition_bounds_merge() will be called from
more places in the foreseeable future, so I'd still vote for removing
the assertion.

When I wrote that function, I had UNION also in mind. A UNION across
multiple partitioned relations will be partitioned if we can merge the
partition bounds in a sensible manner. Of course the current structure
of that function looks more purposed for join, but it's not difficult
to convert it to be used for UNION as well. In that case those set of
functions will have many more callers. So, I will vote to keep that
assertion now that we have it there.
--
Best Wishes,
Ashutosh Bapat

#147Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: Ashutosh Bapat (#146)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Thu, Apr 09, 2020 at 07:34:01PM +0530, Ashutosh Bapat wrote:

On Thu, Apr 9, 2020 at 12:03 PM Etsuro Fujita <etsuro.fujita@gmail.com> wrote:

On Thu, Apr 9, 2020 at 2:36 PM Tom Lane <tgl@sss.pgh.pa.us> wrote:

Etsuro Fujita <etsuro.fujita@gmail.com> writes:

Yeah, partition_bounds_merge() is currently called only from
try_partitionwise_join(), which guarantees that the strategies are the
same.

If there's only one caller and there's not likely to ever be more,
then I tend to agree that you don't need the assertion.

It seems unlikely that partition_bounds_merge() will be called from
more places in the foreseeable future, so I'd still vote for removing
the assertion.

When I wrote that function, I had UNION also in mind. A UNION across
multiple partitioned relations will be partitioned if we can merge the
partition bounds in a sensible manner. Of course the current structure
of that function looks more purposed for join, but it's not difficult
to convert it to be used for UNION as well. In that case those set of
functions will have many more callers. So, I will vote to keep that
assertion now that we have it there.

Yeah. I really don't see why we should remove an assertion that enforces
something useful, especially when it's just a plain comparions. Had it
been some expensive assert, maybe. But how much slower does this make
an assert-enabled build? 0.000000000001% or something like that?

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#148Jeff Janes
jeff.janes@gmail.com
In reply to: Ashutosh Bapat (#146)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Thu, Apr 9, 2020 at 10:04 AM Ashutosh Bapat <ashutosh.bapat.oss@gmail.com>
wrote:

On Thu, Apr 9, 2020 at 12:03 PM Etsuro Fujita <etsuro.fujita@gmail.com>
wrote:

On Thu, Apr 9, 2020 at 2:36 PM Tom Lane <tgl@sss.pgh.pa.us> wrote:

Etsuro Fujita <etsuro.fujita@gmail.com> writes:

Yeah, partition_bounds_merge() is currently called only from
try_partitionwise_join(), which guarantees that the strategies are

the

same.

If there's only one caller and there's not likely to ever be more,
then I tend to agree that you don't need the assertion.

It seems unlikely that partition_bounds_merge() will be called from
more places in the foreseeable future, so I'd still vote for removing
the assertion.

When I wrote that function, I had UNION also in mind. A UNION across
multiple partitioned relations will be partitioned if we can merge the
partition bounds in a sensible manner. Of course the current structure
of that function looks more purposed for join, but it's not difficult
to convert it to be used for UNION as well. In that case those set of
functions will have many more callers. So, I will vote to keep that
assertion now that we have it there.

In that case, we really should add the PG_USED_FOR_ASSERTS_ONLY to make the
compiler happy.

Cheers,

Jeff

#149Ashutosh Bapat
ashutosh.bapat@2ndquadrant.com
In reply to: Jeff Janes (#148)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Fri, 10 Apr 2020 at 20:44, Jeff Janes <jeff.janes@gmail.com> wrote:

On Thu, Apr 9, 2020 at 10:04 AM Ashutosh Bapat <
ashutosh.bapat.oss@gmail.com> wrote:

On Thu, Apr 9, 2020 at 12:03 PM Etsuro Fujita <etsuro.fujita@gmail.com>
wrote:

On Thu, Apr 9, 2020 at 2:36 PM Tom Lane <tgl@sss.pgh.pa.us> wrote:

Etsuro Fujita <etsuro.fujita@gmail.com> writes:

Yeah, partition_bounds_merge() is currently called only from
try_partitionwise_join(), which guarantees that the strategies are

the

same.

If there's only one caller and there's not likely to ever be more,
then I tend to agree that you don't need the assertion.

It seems unlikely that partition_bounds_merge() will be called from
more places in the foreseeable future, so I'd still vote for removing
the assertion.

When I wrote that function, I had UNION also in mind. A UNION across
multiple partitioned relations will be partitioned if we can merge the
partition bounds in a sensible manner. Of course the current structure
of that function looks more purposed for join, but it's not difficult
to convert it to be used for UNION as well. In that case those set of
functions will have many more callers. So, I will vote to keep that
assertion now that we have it there.

In that case, we really should add the PG_USED_FOR_ASSERTS_ONLY to make
the compiler happy.

Attaching my patch again. It doesn't need PG_USED_FOR_ASSERTS_ONLY as well.
Kuntal has confirmed that this fixes the warning for him.

--
Best Wishes,
Ashutosh

#150Tom Lane
tgl@sss.pgh.pa.us
In reply to: Ashutosh Bapat (#149)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Ashutosh Bapat <ashutosh.bapat@2ndquadrant.com> writes:

On Fri, 10 Apr 2020 at 20:44, Jeff Janes <jeff.janes@gmail.com> wrote:

In that case, we really should add the PG_USED_FOR_ASSERTS_ONLY to make
the compiler happy.

Attaching my patch again. It doesn't need PG_USED_FOR_ASSERTS_ONLY as well.
Kuntal has confirmed that this fixes the warning for him.

I see no patch here ...

regards, tom lane

#151Ashutosh Bapat
ashutosh.bapat.oss@gmail.com
In reply to: Tom Lane (#150)
1 attachment(s)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Fri, Apr 10, 2020 at 9:14 PM Tom Lane <tgl@sss.pgh.pa.us> wrote:

Ashutosh Bapat <ashutosh.bapat@2ndquadrant.com> writes:

On Fri, 10 Apr 2020 at 20:44, Jeff Janes <jeff.janes@gmail.com> wrote:

In that case, we really should add the PG_USED_FOR_ASSERTS_ONLY to make
the compiler happy.

Attaching my patch again. It doesn't need PG_USED_FOR_ASSERTS_ONLY as well.
Kuntal has confirmed that this fixes the warning for him.

I see no patch here ...

Sorry. Here it is

--
Best Wishes,
Ashutosh Bapat

Attachments:

fix_warning_in_adv_pwj.patchtext/x-patch; charset=US-ASCII; name=fix_warning_in_adv_pwj.patchDownload
diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c
index 7607501fe7..4681441dcc 100644
--- a/src/backend/partitioning/partbounds.c
+++ b/src/backend/partitioning/partbounds.c
@@ -1021,7 +1021,6 @@ partition_bounds_merge(int partnatts,
 					   List **outer_parts, List **inner_parts)
 {
 	PartitionBoundInfo outer_binfo = outer_rel->boundinfo;
-	PartitionBoundInfo inner_binfo = inner_rel->boundinfo;
 
 	/*
 	 * Currently, this function is called only from try_partitionwise_join(),
@@ -1032,7 +1031,7 @@ partition_bounds_merge(int partnatts,
 		   jointype == JOIN_ANTI);
 
 	/* The partitioning strategies should be the same. */
-	Assert(outer_binfo->strategy == inner_binfo->strategy);
+	Assert(outer_binfo->strategy == inner_rel->boundinfo->strategy);
 
 	*outer_parts = *inner_parts = NIL;
 	switch (outer_binfo->strategy)
#152Tom Lane
tgl@sss.pgh.pa.us
In reply to: Ashutosh Bapat (#151)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

Ashutosh Bapat <ashutosh.bapat.oss@gmail.com> writes:

On Fri, Apr 10, 2020 at 9:14 PM Tom Lane <tgl@sss.pgh.pa.us> wrote:

I see no patch here ...

Sorry. Here it is

LGTM, will push in a moment.

regards, tom lane

#153Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Tom Lane (#152)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Sat, Apr 11, 2020 at 1:00 AM Tom Lane <tgl@sss.pgh.pa.us> wrote:

Ashutosh Bapat <ashutosh.bapat.oss@gmail.com> writes:

On Fri, Apr 10, 2020 at 9:14 PM Tom Lane <tgl@sss.pgh.pa.us> wrote:

I see no patch here ...

Sorry. Here it is

LGTM, will push in a moment.

Thanks for taking care of this, Tom! Thanks for the patch, Ashutosh!
Thanks for the report, Kuntal!

Best regards,
Etsuro Fujita

#154Etsuro Fujita
etsuro.fujita@gmail.com
In reply to: Ashutosh Bapat (#132)
Re: [HACKERS] advanced partition matching algorithm for partition-wise join

On Mon, Apr 6, 2020 at 8:43 PM Ashutosh Bapat
<ashutosh.bapat@2ndquadrant.com> wrote:

On Fri, 3 Apr 2020 at 20:45, Etsuro Fujita <etsuro.fujita@gmail.com> wrote:
But it will be good to have following addition I suggested in my patches to make sure nparts is set to 0 for an unpartitioned relation as per the comment on nparts in RelOptInfo.
@@ -1653,6 +1663,8 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
jointype, restrictlist))
{
Assert(!IS_PARTITIONED_REL(joinrel));
+ /* Join is not partitioned. */
+ joinrel->nparts = 0;
return;
}

I didn't modified that function as proposed, because I thought that 1)
there would be no need to do so, and that 2) it would be better to set
joinrel->nparts only when we set joinrel->part_schema, for
consistency.

3) I think the for nparts comment is somewhat misleading:

int nparts; /* number of partitions; 0 = not partitioned;
* -1 = not yet set */

which says that nparts=0 means not partitioned. But then there are
conditions like this:

/* Nothing to do, if the join relation is not partitioned. */
if (joinrel->part_scheme == NULL || joinrel->nparts == 0)
return;

which (ignoring the obsolete comment) seems to say we can have nparts==0
even for partitioned tables, no?

Yeah, I think I was a bit hasty. I fixed this.

For a non-join relation, nparts = 0 and nparts = -1 both have the same meaning. Although we never set nparts = 0 for a non-join relation?

I don't think so. Consider this:

create table prt (a int, b int) partition by range (a);
create table prt_p1 partition of prt for values from (0) to (250);
create table prt_p2 partition of prt for values from (250) to (500);
drop table prt_p1;
drop table prt_p2;
select count(*) from prt;

For this query, we would have nparts=0 for the partitioned table prt.

Thanks! Sorry for the delay.

Best regards,
Etsuro Fujita